diff --git a/bin/kafka-cluster.sh b/bin/kafka-cluster.sh new file mode 100755 index 0000000000..574007e9cd --- /dev/null +++ b/bin/kafka-cluster.sh @@ -0,0 +1,17 @@ +#!/bin/bash +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +exec $(dirname $0)/kafka-run-class.sh kafka.tools.ClusterTool "$@" diff --git a/bin/kafka-storage.sh b/bin/kafka-storage.sh new file mode 100755 index 0000000000..eef9342387 --- /dev/null +++ b/bin/kafka-storage.sh @@ -0,0 +1,17 @@ +#!/bin/bash +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +exec $(dirname $0)/kafka-run-class.sh kafka.tools.StorageTool "$@" diff --git a/build.gradle b/build.gradle index d9b185e1bd..790cd11613 100644 --- a/build.gradle +++ b/build.gradle @@ -751,6 +751,7 @@ project(':core') { compile project(':clients') compile project(':metadata') compile project(':raft') + compile libs.argparse4j compile libs.jacksonDatabind compile libs.jacksonModuleScala compile libs.jacksonDataformatCsv @@ -1012,7 +1013,10 @@ project(':core') { } test { java { - srcDirs = ["src/generated/java", "src/test/java"] + srcDirs = [] + } + scala { + srcDirs = ["src/test/java", "src/test/scala"] } } } diff --git a/checkstyle/import-control-core.xml b/checkstyle/import-control-core.xml index 6e5042fd35..e9653ba373 100644 --- a/checkstyle/import-control-core.xml +++ b/checkstyle/import-control-core.xml @@ -58,4 +58,19 @@ + + + + + + + + + + + + + + + diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index c185b72ae8..76690bbc82 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -23,6 +23,7 @@ + + diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java index 7b896ffe01..d1f6e68c65 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java @@ -1550,7 +1550,7 @@ public void handleResponse(AbstractResponse response) { } else if (findCoordinatorResponse.error() == Errors.GROUP_AUTHORIZATION_FAILED) { abortableError(GroupAuthorizationException.forGroupId(builder.data().key())); } else { - fatalError(new KafkaException(String.format("Could not find a coordinator with type %s with key %s due to" + + fatalError(new KafkaException(String.format("Could not find a coordinator with type %s with key %s due to " + "unexpected error: %s", coordinatorType, builder.data().key(), findCoordinatorResponse.data().errorMessage()))); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsRequest.java index 9a1032b8c0..a003c2d3f7 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsRequest.java @@ -65,6 +65,16 @@ public CreateTopicsRequest build(short version) { public String toString() { return data.toString(); } + + @Override + public boolean equals(Object other) { + return other instanceof Builder && this.data.equals(((Builder) other).data); + } + + @Override + public int hashCode() { + return data.hashCode(); + } } private final CreateTopicsRequestData data; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorRequest.java index dc512a554e..f4491ad1ee 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorRequest.java @@ -17,6 +17,7 @@ package org.apache.kafka.common.requests; import org.apache.kafka.common.Node; +import org.apache.kafka.common.errors.InvalidRequestException; import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.message.FindCoordinatorRequestData; import org.apache.kafka.common.message.FindCoordinatorResponseData; @@ -102,7 +103,7 @@ public static CoordinatorType forId(byte id) { case 1: return TRANSACTION; default: - throw new IllegalArgumentException("Unknown coordinator type received: " + id); + throw new InvalidRequestException("Unknown coordinator type received: " + id); } } } diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/unsecured/OAuthBearerUnsecuredValidatorCallbackHandler.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/unsecured/OAuthBearerUnsecuredValidatorCallbackHandler.java index 54e289c2c5..a9dc059128 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/unsecured/OAuthBearerUnsecuredValidatorCallbackHandler.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/unsecured/OAuthBearerUnsecuredValidatorCallbackHandler.java @@ -30,6 +30,7 @@ import org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule; import org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallback; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -171,32 +172,23 @@ private void handleCallback(OAuthBearerValidatorCallback callback) { OAuthBearerValidationUtils.validateTimeConsistency(unsecuredJwt).throwExceptionIfFailed(); OAuthBearerValidationUtils.validateScope(unsecuredJwt, requiredScope).throwExceptionIfFailed(); log.info("Successfully validated token with principal {}: {}", unsecuredJwt.principalName(), - unsecuredJwt.claims().toString()); + unsecuredJwt.claims()); callback.token(unsecuredJwt); } private String principalClaimName() { String principalClaimNameValue = option(PRINCIPAL_CLAIM_NAME_OPTION); - String principalClaimName = principalClaimNameValue != null && !principalClaimNameValue.trim().isEmpty() - ? principalClaimNameValue.trim() - : "sub"; - return principalClaimName; + return Utils.isBlank(principalClaimNameValue) ? "sub" : principalClaimNameValue.trim(); } private String scopeClaimName() { String scopeClaimNameValue = option(SCOPE_CLAIM_NAME_OPTION); - String scopeClaimName = scopeClaimNameValue != null && !scopeClaimNameValue.trim().isEmpty() - ? scopeClaimNameValue.trim() - : "scope"; - return scopeClaimName; + return Utils.isBlank(scopeClaimNameValue) ? "scope" : scopeClaimNameValue.trim(); } private List requiredScope() { String requiredSpaceDelimitedScope = option(REQUIRED_SCOPE_OPTION); - List requiredScope = requiredSpaceDelimitedScope == null || requiredSpaceDelimitedScope.trim().isEmpty() - ? Collections.emptyList() - : OAuthBearerScopeUtils.parseScope(requiredSpaceDelimitedScope.trim()); - return requiredScope; + return Utils.isBlank(requiredSpaceDelimitedScope) ? Collections.emptyList() : OAuthBearerScopeUtils.parseScope(requiredSpaceDelimitedScope.trim()); } private int allowableClockSkewMs() { diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/StringDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/StringDeserializer.java index 68e6c409cb..3d8b7bbd98 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/StringDeserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/StringDeserializer.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.errors.SerializationException; import java.io.UnsupportedEncodingException; +import java.nio.charset.StandardCharsets; import java.util.Map; /** @@ -26,7 +27,7 @@ * value.deserializer.encoding or deserializer.encoding. The first two take precedence over the last. */ public class StringDeserializer implements Deserializer { - private String encoding = "UTF8"; + private String encoding = StandardCharsets.UTF_8.name(); @Override public void configure(Map configs, boolean isKey) { diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/StringSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/StringSerializer.java index e16e19ac75..ee01f1a590 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/StringSerializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/StringSerializer.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.errors.SerializationException; import java.io.UnsupportedEncodingException; +import java.nio.charset.StandardCharsets; import java.util.Map; /** @@ -26,7 +27,7 @@ * value.serializer.encoding or serializer.encoding. The first two take precedence over the last. */ public class StringSerializer implements Serializer { - private String encoding = "UTF8"; + private String encoding = StandardCharsets.UTF_8.name(); @Override public void configure(Map configs, boolean isKey) { diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/UUIDDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/UUIDDeserializer.java index e852fc95af..6b245bf33d 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/UUIDDeserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/UUIDDeserializer.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.errors.SerializationException; import java.io.UnsupportedEncodingException; +import java.nio.charset.StandardCharsets; import java.util.Map; import java.util.UUID; @@ -27,7 +28,7 @@ * the property key.deserializer.encoding, value.deserializer.encoding or deserializer.encoding. The first two take precedence over the last. */ public class UUIDDeserializer implements Deserializer { - private String encoding = "UTF8"; + private String encoding = StandardCharsets.UTF_8.name(); @Override public void configure(Map configs, boolean isKey) { diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/UUIDSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/UUIDSerializer.java index 908c202c74..1477546fdf 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/UUIDSerializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/UUIDSerializer.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.errors.SerializationException; import java.io.UnsupportedEncodingException; +import java.nio.charset.StandardCharsets; import java.util.Map; import java.util.UUID; @@ -27,7 +28,7 @@ * the property key.deserializer.encoding, value.deserializer.encoding or deserializer.encoding. The first two take precedence over the last. */ public class UUIDSerializer implements Serializer { - private String encoding = "UTF8"; + private String encoding = StandardCharsets.UTF_8.name(); @Override public void configure(Map configs, boolean isKey) { diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java index da89181f7b..7e46d51c32 100755 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -1334,4 +1334,13 @@ public static long getDateTime(String timestamp) throws ParseException, IllegalA public static Iterator covariantCast(Iterator iterator) { return (Iterator) iterator; } + + /** + * Checks if a string is null, empty or whitespace only. + * @param str a string to be checked + * @return true if the string is null, empty or whitespace only; otherwise, return false. + */ + public static boolean isBlank(String str) { + return str == null || str.trim().isEmpty(); + } } 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 a7dd619a07..c647e9f80c 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 @@ -38,6 +38,7 @@ import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.TopicExistsException; import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; +import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.internals.KafkaFutureImpl; import org.apache.kafka.common.requests.DescribeLogDirsResponse; import org.apache.kafka.common.quota.ClientQuotaAlteration; @@ -69,6 +70,7 @@ public class MockAdminClient extends AdminClient { new HashMap<>(); private final Map beginningOffsets; private final Map endOffsets; + private final boolean usingRaftController; private final String clusterId; private final List> brokerLogDirs; private final List> brokerConfigs; @@ -90,6 +92,7 @@ public static class Builder { private Node controller = null; private List> brokerLogDirs = new ArrayList<>(); private Short defaultPartitions; + private boolean usingRaftController = false; private Integer defaultReplicationFactor; public Builder() { @@ -135,6 +138,11 @@ public Builder defaultReplicationFactor(int defaultReplicationFactor) { return this; } + public Builder usingRaftController(boolean usingRaftController) { + this.usingRaftController = usingRaftController; + return this; + } + public Builder defaultPartitions(short numPartitions) { this.defaultPartitions = numPartitions; return this; @@ -146,7 +154,8 @@ public MockAdminClient build() { clusterId, defaultPartitions != null ? defaultPartitions.shortValue() : 1, defaultReplicationFactor != null ? defaultReplicationFactor.shortValue() : Math.min(brokers.size(), 3), - brokerLogDirs); + brokerLogDirs, + usingRaftController); } } @@ -156,7 +165,7 @@ public MockAdminClient() { public MockAdminClient(List brokers, Node controller) { this(brokers, controller, DEFAULT_CLUSTER_ID, 1, brokers.size(), - Collections.nCopies(brokers.size(), DEFAULT_LOG_DIRS)); + Collections.nCopies(brokers.size(), DEFAULT_LOG_DIRS), false); } private MockAdminClient(List brokers, @@ -164,7 +173,8 @@ private MockAdminClient(List brokers, String clusterId, int defaultPartitions, int defaultReplicationFactor, - List> brokerLogDirs) { + List> brokerLogDirs, + boolean usingRaftController) { this.brokers = brokers; controller(controller); this.clusterId = clusterId; @@ -177,6 +187,7 @@ private MockAdminClient(List brokers, } this.beginningOffsets = new HashMap<>(); this.endOffsets = new HashMap<>(); + this.usingRaftController = usingRaftController; } synchronized public void controller(Node controller) { @@ -889,7 +900,13 @@ public UpdateFeaturesResult updateFeatures(Map featureUpd @Override public DecommissionBrokerResult decommissionBroker(int brokerId, DecommissionBrokerOptions options) { - throw new UnsupportedOperationException("Not implemented yet"); + if (usingRaftController) { + return new DecommissionBrokerResult(KafkaFuture.completedFuture(null)); + } else { + KafkaFutureImpl future = new KafkaFutureImpl<>(); + future.completeExceptionally(new UnsupportedVersionException("")); + return new DecommissionBrokerResult(future); + } } @Override diff --git a/clients/src/test/java/org/apache/kafka/common/requests/FindCoordinatorRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/FindCoordinatorRequestTest.java new file mode 100644 index 0000000000..9393358074 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/requests/FindCoordinatorRequestTest.java @@ -0,0 +1,31 @@ +/* + * 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.requests; + +import org.apache.kafka.common.errors.InvalidRequestException; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertThrows; + +class FindCoordinatorRequestTest { + + @Test + public void getInvalidCoordinatorTypeId() { + assertThrows(InvalidRequestException.class, + () -> FindCoordinatorRequest.CoordinatorType.forId((byte) 10)); + } +} diff --git a/clients/src/test/java/org/apache/kafka/common/requests/StopReplicaRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/StopReplicaRequestTest.java index fd018243ce..50c6974613 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/StopReplicaRequestTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/StopReplicaRequestTest.java @@ -66,7 +66,7 @@ public void testGetErrorResponse() { } } - for (short version = STOP_REPLICA.oldestVersion(); version < STOP_REPLICA.latestVersion(); version++) { + for (short version = STOP_REPLICA.oldestVersion(); version <= STOP_REPLICA.latestVersion(); version++) { StopReplicaRequest.Builder builder = new StopReplicaRequest.Builder(version, 0, 0, 0L, false, topicStates); StopReplicaRequest request = builder.build(); @@ -93,7 +93,7 @@ private void testBuilderNormalization(boolean deletePartitions) { Map expectedPartitionStates = StopReplicaRequestTest.partitionStates(topicStates); - for (short version = STOP_REPLICA.oldestVersion(); version < STOP_REPLICA.latestVersion(); version++) { + for (short version = STOP_REPLICA.oldestVersion(); version <= STOP_REPLICA.latestVersion(); version++) { StopReplicaRequest request = new StopReplicaRequest.Builder(version, 0, 1, 0, deletePartitions, topicStates).build(version); StopReplicaRequestData data = request.data(); @@ -128,7 +128,7 @@ private void testBuilderNormalization(boolean deletePartitions) { public void testTopicStatesNormalization() { List topicStates = topicStates(true); - for (short version = STOP_REPLICA.oldestVersion(); version < STOP_REPLICA.latestVersion(); version++) { + for (short version = STOP_REPLICA.oldestVersion(); version <= STOP_REPLICA.latestVersion(); version++) { // Create a request for version to get its serialized form StopReplicaRequest baseRequest = new StopReplicaRequest.Builder(version, 0, 1, 0, true, topicStates).build(version); @@ -163,7 +163,7 @@ public void testTopicStatesNormalization() { public void testPartitionStatesNormalization() { List topicStates = topicStates(true); - for (short version = STOP_REPLICA.oldestVersion(); version < STOP_REPLICA.latestVersion(); version++) { + for (short version = STOP_REPLICA.oldestVersion(); version <= STOP_REPLICA.latestVersion(); version++) { // Create a request for version to get its serialized form StopReplicaRequest baseRequest = new StopReplicaRequest.Builder(version, 0, 1, 0, true, topicStates).build(version); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/StopReplicaResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/StopReplicaResponseTest.java index e8ea107f1a..c3d049d482 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/StopReplicaResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/StopReplicaResponseTest.java @@ -44,7 +44,7 @@ public void testErrorCountsFromGetErrorResponse() { new StopReplicaPartitionState().setPartitionIndex(0), new StopReplicaPartitionState().setPartitionIndex(1)))); - for (short version = STOP_REPLICA.oldestVersion(); version < STOP_REPLICA.latestVersion(); version++) { + for (short version = STOP_REPLICA.oldestVersion(); version <= STOP_REPLICA.latestVersion(); version++) { StopReplicaRequest request = new StopReplicaRequest.Builder(version, 15, 20, 0, false, topicStates).build(version); StopReplicaResponse response = request diff --git a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java index 2d85f553ca..cc466f7ada 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java @@ -2096,7 +2096,7 @@ private void testUnauthenticatedApiVersionsRequest(SecurityProtocol securityProt private void authenticateUsingSaslPlainAndCheckConnection(String node, boolean enableSaslAuthenticateHeader) throws Exception { // Authenticate using PLAIN username/password String authString = "\u0000" + TestJaasConfig.USERNAME + "\u0000" + TestJaasConfig.PASSWORD; - ByteBuffer authBuf = ByteBuffer.wrap(authString.getBytes("UTF-8")); + ByteBuffer authBuf = ByteBuffer.wrap(Utils.utf8(authString)); if (enableSaslAuthenticateHeader) { SaslAuthenticateRequestData data = new SaslAuthenticateRequestData().setAuthBytes(authBuf.array()); SaslAuthenticateRequest request = new SaslAuthenticateRequest.Builder(data).build(); diff --git a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java index defc53e7bd..0446bafc2f 100644 --- a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java +++ b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java @@ -21,6 +21,7 @@ import org.junit.jupiter.api.Test; import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.HashMap; import java.util.List; @@ -92,7 +93,7 @@ public void testSerdeFromNotNull() { @Test public void stringSerdeShouldSupportDifferentEncodings() { String str = "my string"; - List encodings = Arrays.asList("UTF8", "UTF-16"); + List encodings = Arrays.asList(StandardCharsets.UTF_8.name(), StandardCharsets.UTF_16.name()); for (String encoding : encodings) { try (Serde serDeser = getStringSerde(encoding)) { diff --git a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java index b26988b7a1..ebdc2042cc 100755 --- a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java @@ -851,4 +851,12 @@ private void invokeGetDateTimeMethod(final SimpleDateFormat format) throws Parse Utils.getDateTime(formattedCheckpoint); } + @Test + void testIsBlank() { + assertTrue(Utils.isBlank(null)); + assertTrue(Utils.isBlank("")); + assertTrue(Utils.isBlank(" ")); + assertFalse(Utils.isBlank("bob")); + assertFalse(Utils.isBlank(" bob ")); + } } diff --git a/connect/api/src/main/java/org/apache/kafka/connect/connector/policy/ConnectorClientConfigRequest.java b/connect/api/src/main/java/org/apache/kafka/connect/connector/policy/ConnectorClientConfigRequest.java index 11f756bcb4..4ee0b94898 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/connector/policy/ConnectorClientConfigRequest.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/connector/policy/ConnectorClientConfigRequest.java @@ -96,6 +96,6 @@ public Class connectorClass() { } public enum ClientType { - PRODUCER, CONSUMER, ADMIN; + PRODUCER, CONSUMER, ADMIN } } \ No newline at end of file diff --git a/connect/api/src/main/java/org/apache/kafka/connect/storage/StringConverterConfig.java b/connect/api/src/main/java/org/apache/kafka/connect/storage/StringConverterConfig.java index 26b01332c1..96d0b5bc98 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/storage/StringConverterConfig.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/storage/StringConverterConfig.java @@ -21,6 +21,7 @@ import org.apache.kafka.common.config.ConfigDef.Type; import org.apache.kafka.common.config.ConfigDef.Width; +import java.nio.charset.StandardCharsets; import java.util.Map; /** @@ -29,7 +30,7 @@ public class StringConverterConfig extends ConverterConfig { public static final String ENCODING_CONFIG = "converter.encoding"; - public static final String ENCODING_DEFAULT = "UTF8"; + public static final String ENCODING_DEFAULT = StandardCharsets.UTF_8.name(); private static final String ENCODING_DOC = "The name of the Java character set to use for encoding strings as byte arrays."; private static final String ENCODING_DISPLAY = "Encoding"; diff --git a/connect/api/src/test/java/org/apache/kafka/connect/connector/ConnectorReconfigurationTest.java b/connect/api/src/test/java/org/apache/kafka/connect/connector/ConnectorReconfigurationTest.java index 7a575a46ae..b895ed3985 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/connector/ConnectorReconfigurationTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/connector/ConnectorReconfigurationTest.java @@ -33,7 +33,7 @@ public class ConnectorReconfigurationTest { @Test public void testDefaultReconfigure() { TestConnector conn = new TestConnector(false); - conn.reconfigure(Collections.emptyMap()); + conn.reconfigure(Collections.emptyMap()); assertEquals(conn.stopOrder, 0); assertEquals(conn.configureOrder, 1); } diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/ConnectSchemaTest.java b/connect/api/src/test/java/org/apache/kafka/connect/data/ConnectSchemaTest.java index b0eae0e71f..25e6db3469 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/data/ConnectSchemaTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/data/ConnectSchemaTest.java @@ -69,7 +69,8 @@ public void testFieldsOnStructSchema() { @Test public void testFieldsOnlyValidForStructs() { - assertThrows(DataException.class, Schema.INT8_SCHEMA::fields); + assertThrows(DataException.class, + Schema.INT8_SCHEMA::fields); } @Test diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java b/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java index fca10f18f0..01fbae76a5 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java @@ -845,14 +845,14 @@ public void shouldParseFloatAsFloat32() { ); assertEquals(Schema.FLOAT32_SCHEMA, schemaAndValue.schema()); assertTrue(schemaAndValue.value() instanceof Float); - assertEquals(value.floatValue(), ((Float) schemaAndValue.value()).floatValue(), 0); + assertEquals(value, (Float) schemaAndValue.value(), 0); value = -Float.MAX_VALUE; schemaAndValue = Values.parseString( String.valueOf(value) ); assertEquals(Schema.FLOAT32_SCHEMA, schemaAndValue.schema()); assertTrue(schemaAndValue.value() instanceof Float); - assertEquals(value.floatValue(), ((Float) schemaAndValue.value()).floatValue(), 0); + assertEquals(value, (Float) schemaAndValue.value(), 0); } @Test @@ -863,14 +863,14 @@ public void shouldParseDoubleAsFloat64() { ); assertEquals(Schema.FLOAT64_SCHEMA, schemaAndValue.schema()); assertTrue(schemaAndValue.value() instanceof Double); - assertEquals(value.doubleValue(), ((Double) schemaAndValue.value()).doubleValue(), 0); + assertEquals(value, (Double) schemaAndValue.value(), 0); value = -Double.MAX_VALUE; schemaAndValue = Values.parseString( String.valueOf(value) ); assertEquals(Schema.FLOAT64_SCHEMA, schemaAndValue.schema()); assertTrue(schemaAndValue.value() instanceof Double); - assertEquals(value.doubleValue(), ((Double) schemaAndValue.value()).doubleValue(), 0); + assertEquals(value, (Double) schemaAndValue.value(), 0); } protected void assertParsed(String input) { diff --git a/connect/api/src/test/java/org/apache/kafka/connect/header/ConnectHeadersTest.java b/connect/api/src/test/java/org/apache/kafka/connect/header/ConnectHeadersTest.java index 46050f2ece..b9b9174d73 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/header/ConnectHeadersTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/header/ConnectHeadersTest.java @@ -345,24 +345,16 @@ public void shouldTransformAndRemoveHeaders() { } protected HeaderTransform appendToKey(final String suffix) { - return new HeaderTransform() { - @Override - public Header apply(Header header) { - return header.rename(header.key() + suffix); - } - }; + return header -> header.rename(header.key() + suffix); } protected HeaderTransform removeHeadersOfType(final Type type) { - return new HeaderTransform() { - @Override - public Header apply(Header header) { - Schema schema = header.schema(); - if (schema != null && schema.type() == type) { - return null; - } - return header; + return header -> { + Schema schema = header.schema(); + if (schema != null && schema.type() == type) { + return null; } + return header; }; } diff --git a/connect/api/src/test/java/org/apache/kafka/connect/storage/StringConverterTest.java b/connect/api/src/test/java/org/apache/kafka/connect/storage/StringConverterTest.java index 60aee4d33b..648bb7e27f 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/storage/StringConverterTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/storage/StringConverterTest.java @@ -16,15 +16,16 @@ */ package org.apache.kafka.connect.storage; +import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaAndValue; import org.junit.jupiter.api.Test; -import java.io.UnsupportedEncodingException; +import java.nio.charset.StandardCharsets; import java.util.Collections; -import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNull; public class StringConverterTest { @@ -34,13 +35,13 @@ public class StringConverterTest { private StringConverter converter = new StringConverter(); @Test - public void testStringToBytes() throws UnsupportedEncodingException { - assertArrayEquals(SAMPLE_STRING.getBytes("UTF8"), converter.fromConnectData(TOPIC, Schema.STRING_SCHEMA, SAMPLE_STRING)); + public void testStringToBytes() { + assertArrayEquals(Utils.utf8(SAMPLE_STRING), converter.fromConnectData(TOPIC, Schema.STRING_SCHEMA, SAMPLE_STRING)); } @Test - public void testNonStringToBytes() throws UnsupportedEncodingException { - assertArrayEquals("true".getBytes("UTF8"), converter.fromConnectData(TOPIC, Schema.BOOLEAN_SCHEMA, true)); + public void testNonStringToBytes() { + assertArrayEquals(Utils.utf8("true"), converter.fromConnectData(TOPIC, Schema.BOOLEAN_SCHEMA, true)); } @Test @@ -49,14 +50,14 @@ public void testNullToBytes() { } @Test - public void testToBytesIgnoresSchema() throws UnsupportedEncodingException { - assertArrayEquals("true".getBytes("UTF8"), converter.fromConnectData(TOPIC, null, true)); + public void testToBytesIgnoresSchema() { + assertArrayEquals(Utils.utf8("true"), converter.fromConnectData(TOPIC, null, true)); } @Test - public void testToBytesNonUtf8Encoding() throws UnsupportedEncodingException { - converter.configure(Collections.singletonMap("converter.encoding", "UTF-16"), true); - assertArrayEquals(SAMPLE_STRING.getBytes("UTF-16"), converter.fromConnectData(TOPIC, Schema.STRING_SCHEMA, SAMPLE_STRING)); + public void testToBytesNonUtf8Encoding() { + converter.configure(Collections.singletonMap("converter.encoding", StandardCharsets.UTF_16.name()), true); + assertArrayEquals(SAMPLE_STRING.getBytes(StandardCharsets.UTF_16), converter.fromConnectData(TOPIC, Schema.STRING_SCHEMA, SAMPLE_STRING)); } @Test @@ -74,9 +75,9 @@ public void testBytesNullToString() { } @Test - public void testBytesToStringNonUtf8Encoding() throws UnsupportedEncodingException { - converter.configure(Collections.singletonMap("converter.encoding", "UTF-16"), true); - SchemaAndValue data = converter.toConnectData(TOPIC, SAMPLE_STRING.getBytes("UTF-16")); + public void testBytesToStringNonUtf8Encoding() { + converter.configure(Collections.singletonMap("converter.encoding", StandardCharsets.UTF_16.name()), true); + SchemaAndValue data = converter.toConnectData(TOPIC, SAMPLE_STRING.getBytes(StandardCharsets.UTF_16)); assertEquals(Schema.OPTIONAL_STRING_SCHEMA, data.schema()); assertEquals(SAMPLE_STRING, data.value()); } @@ -85,13 +86,13 @@ public void testBytesToStringNonUtf8Encoding() throws UnsupportedEncodingExcepti // The following simply verify that the delegation works. @Test - public void testStringHeaderValueToBytes() throws UnsupportedEncodingException { - assertArrayEquals(SAMPLE_STRING.getBytes("UTF8"), converter.fromConnectHeader(TOPIC, "hdr", Schema.STRING_SCHEMA, SAMPLE_STRING)); + public void testStringHeaderValueToBytes() { + assertArrayEquals(Utils.utf8(SAMPLE_STRING), converter.fromConnectHeader(TOPIC, "hdr", Schema.STRING_SCHEMA, SAMPLE_STRING)); } @Test - public void testNonStringHeaderValueToBytes() throws UnsupportedEncodingException { - assertArrayEquals("true".getBytes("UTF8"), converter.fromConnectHeader(TOPIC, "hdr", Schema.BOOLEAN_SCHEMA, true)); + public void testNonStringHeaderValueToBytes() { + assertArrayEquals(Utils.utf8("true"), converter.fromConnectHeader(TOPIC, "hdr", Schema.BOOLEAN_SCHEMA, true)); } @Test diff --git a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java index 8f2f1c23a6..a7bf464707 100644 --- a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java +++ b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java @@ -66,125 +66,73 @@ public class JsonConverter implements Converter, HeaderConverter { private static final Map TO_CONNECT_CONVERTERS = new EnumMap<>(Schema.Type.class); static { - TO_CONNECT_CONVERTERS.put(Schema.Type.BOOLEAN, new JsonToConnectTypeConverter() { - @Override - public Object convert(Schema schema, JsonNode value) { - return value.booleanValue(); - } - }); - TO_CONNECT_CONVERTERS.put(Schema.Type.INT8, new JsonToConnectTypeConverter() { - @Override - public Object convert(Schema schema, JsonNode value) { - return (byte) value.intValue(); - } - }); - TO_CONNECT_CONVERTERS.put(Schema.Type.INT16, new JsonToConnectTypeConverter() { - @Override - public Object convert(Schema schema, JsonNode value) { - return (short) value.intValue(); - } - }); - TO_CONNECT_CONVERTERS.put(Schema.Type.INT32, new JsonToConnectTypeConverter() { - @Override - public Object convert(Schema schema, JsonNode value) { - return value.intValue(); + TO_CONNECT_CONVERTERS.put(Schema.Type.BOOLEAN, (schema, value) -> value.booleanValue()); + TO_CONNECT_CONVERTERS.put(Schema.Type.INT8, (schema, value) -> (byte) value.intValue()); + TO_CONNECT_CONVERTERS.put(Schema.Type.INT16, (schema, value) -> (short) value.intValue()); + TO_CONNECT_CONVERTERS.put(Schema.Type.INT32, (schema, value) -> value.intValue()); + TO_CONNECT_CONVERTERS.put(Schema.Type.INT64, (schema, value) -> value.longValue()); + TO_CONNECT_CONVERTERS.put(Schema.Type.FLOAT32, (schema, value) -> value.floatValue()); + TO_CONNECT_CONVERTERS.put(Schema.Type.FLOAT64, (schema, value) -> value.doubleValue()); + TO_CONNECT_CONVERTERS.put(Schema.Type.BYTES, (schema, value) -> { + try { + return value.binaryValue(); + } catch (IOException e) { + throw new DataException("Invalid bytes field", e); } }); - TO_CONNECT_CONVERTERS.put(Schema.Type.INT64, new JsonToConnectTypeConverter() { - @Override - public Object convert(Schema schema, JsonNode value) { - return value.longValue(); - } - }); - TO_CONNECT_CONVERTERS.put(Schema.Type.FLOAT32, new JsonToConnectTypeConverter() { - @Override - public Object convert(Schema schema, JsonNode value) { - return value.floatValue(); + TO_CONNECT_CONVERTERS.put(Schema.Type.STRING, (schema, value) -> value.textValue()); + TO_CONNECT_CONVERTERS.put(Schema.Type.ARRAY, (schema, value) -> { + Schema elemSchema = schema == null ? null : schema.valueSchema(); + ArrayList result = new ArrayList<>(); + for (JsonNode elem : value) { + result.add(convertToConnect(elemSchema, elem)); } + return result; }); - TO_CONNECT_CONVERTERS.put(Schema.Type.FLOAT64, new JsonToConnectTypeConverter() { - @Override - public Object convert(Schema schema, JsonNode value) { - return value.doubleValue(); - } - }); - TO_CONNECT_CONVERTERS.put(Schema.Type.BYTES, new JsonToConnectTypeConverter() { - @Override - public Object convert(Schema schema, JsonNode value) { - try { - return value.binaryValue(); - } catch (IOException e) { - throw new DataException("Invalid bytes field", e); - } - } - }); - TO_CONNECT_CONVERTERS.put(Schema.Type.STRING, new JsonToConnectTypeConverter() { - @Override - public Object convert(Schema schema, JsonNode value) { - return value.textValue(); - } - }); - TO_CONNECT_CONVERTERS.put(Schema.Type.ARRAY, new JsonToConnectTypeConverter() { - @Override - public Object convert(Schema schema, JsonNode value) { - Schema elemSchema = schema == null ? null : schema.valueSchema(); - ArrayList result = new ArrayList<>(); - for (JsonNode elem : value) { - result.add(convertToConnect(elemSchema, elem)); + TO_CONNECT_CONVERTERS.put(Schema.Type.MAP, (schema, value) -> { + Schema keySchema = schema == null ? null : schema.keySchema(); + Schema valueSchema = schema == null ? null : schema.valueSchema(); + + // If the map uses strings for keys, it should be encoded in the natural JSON format. If it uses other + // primitive types or a complex type as a key, it will be encoded as a list of pairs. If we don't have a + // schema, we default to encoding in a Map. + Map result = new HashMap<>(); + if (schema == null || keySchema.type() == Schema.Type.STRING) { + if (!value.isObject()) + throw new DataException("Maps with string fields should be encoded as JSON objects, but found " + value.getNodeType()); + Iterator> fieldIt = value.fields(); + while (fieldIt.hasNext()) { + Map.Entry entry = fieldIt.next(); + result.put(entry.getKey(), convertToConnect(valueSchema, entry.getValue())); } - return result; - } - }); - TO_CONNECT_CONVERTERS.put(Schema.Type.MAP, new JsonToConnectTypeConverter() { - @Override - public Object convert(Schema schema, JsonNode value) { - Schema keySchema = schema == null ? null : schema.keySchema(); - Schema valueSchema = schema == null ? null : schema.valueSchema(); - - // If the map uses strings for keys, it should be encoded in the natural JSON format. If it uses other - // primitive types or a complex type as a key, it will be encoded as a list of pairs. If we don't have a - // schema, we default to encoding in a Map. - Map result = new HashMap<>(); - if (schema == null || keySchema.type() == Schema.Type.STRING) { - if (!value.isObject()) - throw new DataException("Maps with string fields should be encoded as JSON objects, but found " + value.getNodeType()); - Iterator> fieldIt = value.fields(); - while (fieldIt.hasNext()) { - Map.Entry entry = fieldIt.next(); - result.put(entry.getKey(), convertToConnect(valueSchema, entry.getValue())); - } - } else { - if (!value.isArray()) - throw new DataException("Maps with non-string fields should be encoded as JSON array of tuples, but found " + value.getNodeType()); - for (JsonNode entry : value) { - if (!entry.isArray()) - throw new DataException("Found invalid map entry instead of array tuple: " + entry.getNodeType()); - if (entry.size() != 2) - throw new DataException("Found invalid map entry, expected length 2 but found :" + entry.size()); - result.put(convertToConnect(keySchema, entry.get(0)), - convertToConnect(valueSchema, entry.get(1))); - } + } else { + if (!value.isArray()) + throw new DataException("Maps with non-string fields should be encoded as JSON array of tuples, but found " + value.getNodeType()); + for (JsonNode entry : value) { + if (!entry.isArray()) + throw new DataException("Found invalid map entry instead of array tuple: " + entry.getNodeType()); + if (entry.size() != 2) + throw new DataException("Found invalid map entry, expected length 2 but found :" + entry.size()); + result.put(convertToConnect(keySchema, entry.get(0)), + convertToConnect(valueSchema, entry.get(1))); } - return result; } + return result; }); - TO_CONNECT_CONVERTERS.put(Schema.Type.STRUCT, new JsonToConnectTypeConverter() { - @Override - public Object convert(Schema schema, JsonNode value) { - if (!value.isObject()) - throw new DataException("Structs should be encoded as JSON objects, but found " + value.getNodeType()); - - // We only have ISchema here but need Schema, so we need to materialize the actual schema. Using ISchema - // avoids having to materialize the schema for non-Struct types but it cannot be avoided for Structs since - // they require a schema to be provided at construction. However, the schema is only a SchemaBuilder during - // translation of schemas to JSON; during the more common translation of data to JSON, the call to schema.schema() - // just returns the schema Object and has no overhead. - Struct result = new Struct(schema.schema()); - for (Field field : schema.fields()) - result.put(field, convertToConnect(field.schema(), value.get(field.name()))); - - return result; - } + TO_CONNECT_CONVERTERS.put(Schema.Type.STRUCT, (schema, value) -> { + if (!value.isObject()) + throw new DataException("Structs should be encoded as JSON objects, but found " + value.getNodeType()); + + // We only have ISchema here but need Schema, so we need to materialize the actual schema. Using ISchema + // avoids having to materialize the schema for non-Struct types but it cannot be avoided for Structs since + // they require a schema to be provided at construction. However, the schema is only a SchemaBuilder during + // translation of schemas to JSON; during the more common translation of data to JSON, the call to schema.schema() + // just returns the schema Object and has no overhead. + Struct result = new Struct(schema.schema()); + for (Field field : schema.fields()) + result.put(field, convertToConnect(field.schema(), value.get(field.name()))); + + return result; }); } diff --git a/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterTest.java b/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterTest.java index be556b8e91..4e4c53b9d9 100644 --- a/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterTest.java +++ b/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterTest.java @@ -36,7 +36,6 @@ import java.io.File; import java.io.IOException; -import java.io.UnsupportedEncodingException; import java.math.BigDecimal; import java.math.BigInteger; import java.net.URISyntaxException; @@ -128,8 +127,8 @@ public void doubleToConnect() { @Test - public void bytesToConnect() throws UnsupportedEncodingException { - ByteBuffer reference = ByteBuffer.wrap("test-string".getBytes("UTF-8")); + public void bytesToConnect() { + ByteBuffer reference = ByteBuffer.wrap(Utils.utf8("test-string")); String msg = "{ \"schema\": { \"type\": \"bytes\" }, \"payload\": \"dGVzdC1zdHJpbmc=\" }"; SchemaAndValue schemaAndValue = converter.toConnectData(TOPIC, msg.getBytes()); ByteBuffer converted = ByteBuffer.wrap((byte[]) schemaAndValue.value()); diff --git a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/Checkpoint.java b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/Checkpoint.java index 74db7461ef..8729005adb 100644 --- a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/Checkpoint.java +++ b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/Checkpoint.java @@ -180,5 +180,5 @@ byte[] recordKey() { byte[] recordValue() { return serializeValue(VERSION).array(); } -}; +} diff --git a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/Heartbeat.java b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/Heartbeat.java index a34ce9efb3..5f70055c10 100644 --- a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/Heartbeat.java +++ b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/Heartbeat.java @@ -141,5 +141,5 @@ private static Schema valueSchema(short version) { assert version == 0; return VALUE_SCHEMA_V0; } -}; +} diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointConnector.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointConnector.java index 37b6d51828..5118ee1818 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointConnector.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointConnector.java @@ -148,7 +148,7 @@ private void loadInitialConsumerGroups() List findConsumerGroups() throws InterruptedException, ExecutionException { return listConsumerGroups().stream() - .map(x -> x.groupId()) + .map(ConsumerGroupListing::groupId) .filter(this::shouldReplicate) .collect(Collectors.toList()); } diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java index 8e9e764e30..e42a78ac95 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java @@ -235,7 +235,7 @@ private void refreshIdleConsumerGroupOffset() { if (consumerGroupState.equals(ConsumerGroupState.EMPTY)) { idleConsumerGroupsOffset.put(group, targetAdminClient.listConsumerGroupOffsets(group) .partitionsToOffsetAndMetadata().get().entrySet().stream().collect( - Collectors.toMap(e -> e.getKey(), e -> e.getValue()))); + Collectors.toMap(Entry::getKey, Entry::getValue))); } // new consumer upstream has state "DEAD" and will be identified during the offset sync-up } catch (InterruptedException | ExecutionException e) { diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMaker.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMaker.java index fa73f8de3a..e13873e263 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMaker.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMaker.java @@ -36,6 +36,7 @@ import org.apache.kafka.connect.connector.policy.AllConnectorClientConfigOverridePolicy; import org.apache.kafka.connect.connector.policy.ConnectorClientConfigOverridePolicy; +import org.apache.kafka.connect.util.SharedTopicAdmin; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -133,7 +134,7 @@ public MirrorMaker(MirrorMakerConfig config, List clusters, Time time) { if (herderPairs.isEmpty()) { throw new IllegalArgumentException("No source->target replication flows."); } - this.herderPairs.forEach(x -> addHerder(x)); + this.herderPairs.forEach(this::addHerder); shutdownHook = new ShutdownHook(); } @@ -173,7 +174,7 @@ public void start() { } } log.info("Configuring connectors..."); - herderPairs.forEach(x -> configureConnectors(x)); + herderPairs.forEach(this::configureConnectors); log.info("Kafka MirrorMaker started"); } @@ -233,20 +234,28 @@ private void addHerder(SourceAndTarget sourceAndTarget) { plugins.compareAndSwapWithDelegatingLoader(); DistributedConfig distributedConfig = new DistributedConfig(workerProps); String kafkaClusterId = ConnectUtils.lookupKafkaClusterId(distributedConfig); - KafkaOffsetBackingStore offsetBackingStore = new KafkaOffsetBackingStore(); + // Create the admin client to be shared by all backing stores for this herder + Map adminProps = new HashMap<>(config.originals()); + ConnectUtils.addMetricsContextProperties(adminProps, distributedConfig, kafkaClusterId); + SharedTopicAdmin sharedAdmin = new SharedTopicAdmin(adminProps); + KafkaOffsetBackingStore offsetBackingStore = new KafkaOffsetBackingStore(sharedAdmin); offsetBackingStore.configure(distributedConfig); Worker worker = new Worker(workerId, time, plugins, distributedConfig, offsetBackingStore, CLIENT_CONFIG_OVERRIDE_POLICY); WorkerConfigTransformer configTransformer = worker.configTransformer(); Converter internalValueConverter = worker.getInternalValueConverter(); - StatusBackingStore statusBackingStore = new KafkaStatusBackingStore(time, internalValueConverter); + StatusBackingStore statusBackingStore = new KafkaStatusBackingStore(time, internalValueConverter, sharedAdmin); statusBackingStore.configure(distributedConfig); ConfigBackingStore configBackingStore = new KafkaConfigBackingStore( internalValueConverter, distributedConfig, - configTransformer); + configTransformer, + sharedAdmin); + // Pass the shared admin to the distributed herder as an additional AutoCloseable object that should be closed when the + // herder is stopped. MirrorMaker has multiple herders, and having the herder own the close responsibility is much easier than + // tracking the various shared admin objects in this class. Herder herder = new DistributedHerder(distributedConfig, time, worker, kafkaClusterId, statusBackingStore, configBackingStore, - advertisedUrl, CLIENT_CONFIG_OVERRIDE_POLICY); + advertisedUrl, CLIENT_CONFIG_OVERRIDE_POLICY, sharedAdmin); herders.put(sourceAndTarget, herder); } @@ -281,7 +290,7 @@ public static void main(String[] args) { Exit.exit(-1); return; } - File configFile = (File) ns.get("config"); + File configFile = ns.get("config"); List clusters = ns.getList("clusters"); try { log.info("Kafka MirrorMaker initializing ..."); diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMakerConfig.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMakerConfig.java index 7c546bc862..298bce34fe 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMakerConfig.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMakerConfig.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.connect.mirror; +import java.util.Map.Entry; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; @@ -266,7 +267,7 @@ Map transform(Map props) { private Map stringsWithPrefixStripped(String prefix) { return originalsStrings().entrySet().stream() .filter(x -> x.getKey().startsWith(prefix)) - .collect(Collectors.toMap(x -> x.getKey().substring(prefix.length()), x -> x.getValue())); + .collect(Collectors.toMap(x -> x.getKey().substring(prefix.length()), Entry::getValue)); } private Map stringsWithPrefix(String prefix) { @@ -278,12 +279,12 @@ private Map stringsWithPrefix(String prefix) { static Map clusterConfigsWithPrefix(String prefix, Map props) { return props.entrySet().stream() .filter(x -> !x.getKey().matches("(^consumer.*|^producer.*|^admin.*)")) - .collect(Collectors.toMap(x -> prefix + x.getKey(), x -> x.getValue())); + .collect(Collectors.toMap(x -> prefix + x.getKey(), Entry::getValue)); } static Map clientConfigsWithPrefix(String prefix, Map props) { return props.entrySet().stream() .filter(x -> x.getKey().matches("(^consumer.*|^producer.*|^admin.*)")) - .collect(Collectors.toMap(x -> prefix + x.getKey(), x -> x.getValue())); + .collect(Collectors.toMap(x -> prefix + x.getKey(), Entry::getValue)); } } diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMetrics.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMetrics.java index 04a92b95a2..4bd03f3e19 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMetrics.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMetrics.java @@ -115,7 +115,7 @@ class MirrorMetrics implements AutoCloseable { ReplicationPolicy replicationPolicy = taskConfig.replicationPolicy(); partitionMetrics = taskConfig.taskTopicPartitions().stream() .map(x -> new TopicPartition(replicationPolicy.formatRemoteTopic(source, x.topic()), x.partition())) - .collect(Collectors.toMap(x -> x, x -> new PartitionMetrics(x))); + .collect(Collectors.toMap(x -> x, PartitionMetrics::new)); } diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java index 0f6eb46dd5..3e7f0c7bea 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.connect.mirror; +import java.util.Map.Entry; import org.apache.kafka.connect.connector.Task; import org.apache.kafka.connect.source.SourceConnector; import org.apache.kafka.common.config.ConfigDef; @@ -269,7 +270,7 @@ private void refreshKnownTargetTopics() private Set topicsBeingReplicated() { Set knownTargetTopics = toTopics(knownTargetTopicPartitions); return knownSourceTopicPartitions.stream() - .map(x -> x.topic()) + .map(TopicPartition::topic) .distinct() .filter(x -> knownTargetTopics.contains(formatRemoteTopic(x))) .collect(Collectors.toSet()); @@ -277,7 +278,7 @@ private Set topicsBeingReplicated() { private Set toTopics(Collection tps) { return tps.stream() - .map(x -> x.topic()) + .map(TopicPartition::topic) .collect(Collectors.toSet()); } @@ -309,8 +310,8 @@ private void createOffsetSyncsTopic() { void computeAndCreateTopicPartitions() throws InterruptedException, ExecutionException { Map partitionCounts = knownSourceTopicPartitions.stream() - .collect(Collectors.groupingBy(x -> x.topic(), Collectors.counting())).entrySet().stream() - .collect(Collectors.toMap(x -> formatRemoteTopic(x.getKey()), x -> x.getValue())); + .collect(Collectors.groupingBy(TopicPartition::topic, Collectors.counting())).entrySet().stream() + .collect(Collectors.toMap(x -> formatRemoteTopic(x.getKey()), Entry::getValue)); Set knownTargetTopics = toTopics(knownTargetTopicPartitions); List newTopics = partitionCounts.entrySet().stream() .filter(x -> !knownTargetTopics.contains(x.getKey())) @@ -318,7 +319,7 @@ void computeAndCreateTopicPartitions() .collect(Collectors.toList()); Map newPartitions = partitionCounts.entrySet().stream() .filter(x -> knownTargetTopics.contains(x.getKey())) - .collect(Collectors.toMap(x -> x.getKey(), x -> NewPartitions.increaseTo(x.getValue().intValue()))); + .collect(Collectors.toMap(Entry::getKey, x -> NewPartitions.increaseTo(x.getValue().intValue()))); createTopicPartitions(partitionCounts, newTopics, newPartitions); } @@ -364,7 +365,7 @@ private void updateTopicConfigs(Map topicConfigs) throws InterruptedException, ExecutionException { Map configs = topicConfigs.entrySet().stream() .collect(Collectors.toMap(x -> - new ConfigResource(ConfigResource.Type.TOPIC, x.getKey()), x -> x.getValue())); + new ConfigResource(ConfigResource.Type.TOPIC, x.getKey()), Entry::getValue)); log.trace("Syncing configs for {} topics.", configs.size()); targetAdminClient.alterConfigs(configs).values().forEach((k, v) -> v.whenComplete((x, e) -> { if (e != null) { @@ -395,7 +396,7 @@ private Map describeTopicConfigs(Set topics) .map(x -> new ConfigResource(ConfigResource.Type.TOPIC, x)) .collect(Collectors.toSet()); return sourceAdminClient.describeConfigs(resources).all().get().entrySet().stream() - .collect(Collectors.toMap(x -> x.getKey().name(), x -> x.getValue())); + .collect(Collectors.toMap(x -> x.getKey().name(), Entry::getValue)); } Config targetConfig(Config sourceConfig) { diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceTask.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceTask.java index 0b864768c9..9ccd4308d9 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceTask.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceTask.java @@ -219,7 +219,7 @@ private void sendOffsetSync(TopicPartition topicPartition, long upstreamOffset, } private Map loadOffsets(Set topicPartitions) { - return topicPartitions.stream().collect(Collectors.toMap(x -> x, x -> loadOffset(x))); + return topicPartitions.stream().collect(Collectors.toMap(x -> x, this::loadOffset)); } private Long loadOffset(TopicPartition topicPartition) { diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointConnectorTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointConnectorTest.java index 47d288c9ac..1ce0de5751 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointConnectorTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointConnectorTest.java @@ -90,7 +90,7 @@ public void testFindConsumerGroups() throws Exception { doReturn(true).when(connector).shouldReplicate(anyString()); List groupFound = connector.findConsumerGroups(); - Set expectedGroups = groups.stream().map(g -> g.groupId()).collect(Collectors.toSet()); + Set expectedGroups = groups.stream().map(ConsumerGroupListing::groupId).collect(Collectors.toSet()); assertEquals(expectedGroups, new HashSet<>(groupFound)); } diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java index 568f914b96..f2cc850a4f 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java @@ -281,15 +281,11 @@ public void testReplication() throws Exception { "Checkpoints were not emitted upstream to primary cluster."); } - waitForCondition(() -> { - return primaryClient.remoteConsumerOffsets(consumerGroupName, BACKUP_CLUSTER_ALIAS, - Duration.ofMillis(CHECKPOINT_DURATION_MS)).containsKey(new TopicPartition("backup.test-topic-1", 0)); - }, CHECKPOINT_DURATION_MS, "Offsets not translated downstream to primary cluster."); + waitForCondition(() -> primaryClient.remoteConsumerOffsets(consumerGroupName, BACKUP_CLUSTER_ALIAS, + Duration.ofMillis(CHECKPOINT_DURATION_MS)).containsKey(new TopicPartition("backup.test-topic-1", 0)), CHECKPOINT_DURATION_MS, "Offsets not translated downstream to primary cluster."); - waitForCondition(() -> { - return primaryClient.remoteConsumerOffsets(consumerGroupName, BACKUP_CLUSTER_ALIAS, - Duration.ofMillis(CHECKPOINT_DURATION_MS)).containsKey(new TopicPartition("test-topic-1", 0)); - }, CHECKPOINT_DURATION_MS, "Offsets not translated upstream to primary cluster."); + waitForCondition(() -> primaryClient.remoteConsumerOffsets(consumerGroupName, BACKUP_CLUSTER_ALIAS, + Duration.ofMillis(CHECKPOINT_DURATION_MS)).containsKey(new TopicPartition("test-topic-1", 0)), CHECKPOINT_DURATION_MS, "Offsets not translated upstream to primary cluster."); Map primaryOffsets = primaryClient.remoteConsumerOffsets(consumerGroupName, BACKUP_CLUSTER_ALIAS, Duration.ofMillis(CHECKPOINT_DURATION_MS)); @@ -522,7 +518,7 @@ private static void waitForConsumerGroupOffsetSync(EmbeddedConnectCluster co Map consumerGroupOffsets = adminClient.listConsumerGroupOffsets(consumerGroupId).partitionsToOffsetAndMetadata().get(); long consumerGroupOffsetTotal = consumerGroupOffsets.values().stream() - .mapToLong(metadata -> metadata.offset()).sum(); + .mapToLong(OffsetAndMetadata::offset).sum(); Map offsets = consumer.endOffsets(tps, CONSUMER_POLL_TIMEOUT_MS); long totalOffsets = offsets.values().stream().mapToLong(l -> l).sum(); diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationSSLTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationSSLTest.java index 00f858856b..0e703fb4c0 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationSSLTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationSSLTest.java @@ -55,10 +55,10 @@ public void startClusters() throws Exception { e -> String.valueOf(e.getKey()), e -> String.valueOf(e.getValue())))); mm2Props.putAll(sslProps.entrySet().stream().collect(Collectors.toMap( - e -> BACKUP_CLUSTER_ALIAS + "." + String.valueOf(e.getKey()), e -> String.valueOf(e.getValue())))); + e -> BACKUP_CLUSTER_ALIAS + "." + e.getKey(), e -> String.valueOf(e.getValue())))); // set SSL config for producer used by source task in MM2 mm2Props.putAll(sslProps.entrySet().stream().collect(Collectors.toMap( - e -> BACKUP_CLUSTER_ALIAS + ".producer." + String.valueOf(e.getKey()), e -> String.valueOf(e.getValue())))); + e -> BACKUP_CLUSTER_ALIAS + ".producer." + e.getKey(), e -> String.valueOf(e.getValue())))); super.startClusters(); } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java index 22c1ad82d6..8d93e79591 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java @@ -36,12 +36,14 @@ import org.apache.kafka.connect.storage.KafkaStatusBackingStore; import org.apache.kafka.connect.storage.StatusBackingStore; import org.apache.kafka.connect.util.ConnectUtils; +import org.apache.kafka.connect.util.SharedTopicAdmin; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.net.URI; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.Map; /** @@ -101,7 +103,12 @@ public Connect startConnect(Map workerProps) { URI advertisedUrl = rest.advertisedUrl(); String workerId = advertisedUrl.getHost() + ":" + advertisedUrl.getPort(); - KafkaOffsetBackingStore offsetBackingStore = new KafkaOffsetBackingStore(); + // Create the admin client to be shared by all backing stores. + Map adminProps = new HashMap<>(config.originals()); + ConnectUtils.addMetricsContextProperties(adminProps, config, kafkaClusterId); + SharedTopicAdmin sharedAdmin = new SharedTopicAdmin(adminProps); + + KafkaOffsetBackingStore offsetBackingStore = new KafkaOffsetBackingStore(sharedAdmin); offsetBackingStore.configure(config); ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy = plugins.newPlugin( @@ -112,17 +119,20 @@ public Connect startConnect(Map workerProps) { WorkerConfigTransformer configTransformer = worker.configTransformer(); Converter internalValueConverter = worker.getInternalValueConverter(); - StatusBackingStore statusBackingStore = new KafkaStatusBackingStore(time, internalValueConverter); + StatusBackingStore statusBackingStore = new KafkaStatusBackingStore(time, internalValueConverter, sharedAdmin); statusBackingStore.configure(config); ConfigBackingStore configBackingStore = new KafkaConfigBackingStore( internalValueConverter, config, - configTransformer); + configTransformer, + sharedAdmin); + // Pass the shared admin to the distributed herder as an additional AutoCloseable object that should be closed when the + // herder is stopped. This is easier than having to track and own the lifecycle ourselves. DistributedHerder herder = new DistributedHerder(config, time, worker, kafkaClusterId, statusBackingStore, configBackingStore, - advertisedUrl.toString(), connectorClientConfigOverridePolicy); + advertisedUrl.toString(), connectorClientConfigOverridePolicy, sharedAdmin); final Connect connect = new Connect(herder, rest); log.info("Kafka Connect distributed worker initialization took {}ms", time.hiResClockMs() - initStart); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectStandalone.java b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectStandalone.java index cf7b93bd7c..19cc115d9d 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectStandalone.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectStandalone.java @@ -32,7 +32,6 @@ import org.apache.kafka.connect.runtime.standalone.StandaloneConfig; import org.apache.kafka.connect.runtime.standalone.StandaloneHerder; import org.apache.kafka.connect.storage.FileOffsetBackingStore; -import org.apache.kafka.connect.util.Callback; import org.apache.kafka.connect.util.ConnectUtils; import org.apache.kafka.connect.util.FutureCallback; import org.slf4j.Logger; @@ -73,7 +72,7 @@ public static void main(String[] args) { String workerPropsFile = args[0]; Map workerProps = !workerPropsFile.isEmpty() ? - Utils.propsToStringMap(Utils.loadProps(workerPropsFile)) : Collections.emptyMap(); + Utils.propsToStringMap(Utils.loadProps(workerPropsFile)) : Collections.emptyMap(); log.info("Scanning for plugin classes. This might take a moment ..."); Plugins plugins = new Plugins(workerProps); @@ -103,14 +102,11 @@ public static void main(String[] args) { connect.start(); for (final String connectorPropsFile : Arrays.copyOfRange(args, 1, args.length)) { Map connectorProps = Utils.propsToStringMap(Utils.loadProps(connectorPropsFile)); - FutureCallback> cb = new FutureCallback<>(new Callback>() { - @Override - public void onCompletion(Throwable error, Herder.Created info) { - if (error != null) - log.error("Failed to create job for {}", connectorPropsFile); - else - log.info("Created connector {}", info.result().name()); - } + FutureCallback> cb = new FutureCallback<>((error, info) -> { + if (error != null) + log.error("Failed to create job for {}", connectorPropsFile); + else + log.info("Created connector {}", info.result().name()); }); herder.putConnectorConfig( connectorProps.get(ConnectorConfig.NAME_CONFIG), diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/connector/policy/AbstractConnectorClientConfigOverridePolicy.java b/connect/runtime/src/main/java/org/apache/kafka/connect/connector/policy/AbstractConnectorClientConfigOverridePolicy.java index 3c310db966..859041a5df 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/connector/policy/AbstractConnectorClientConfigOverridePolicy.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/connector/policy/AbstractConnectorClientConfigOverridePolicy.java @@ -39,7 +39,7 @@ public final List validate(ConnectorClientConfigRequest connectorCl protected ConfigValue configValue(Map.Entry configEntry) { ConfigValue configValue = - new ConfigValue(configEntry.getKey(), configEntry.getValue(), new ArrayList<>(), new ArrayList()); + new ConfigValue(configEntry.getKey(), configEntry.getValue(), new ArrayList<>(), new ArrayList<>()); validate(configValue); return configValue; } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java index a68a0d5649..97d9731b95 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java @@ -265,6 +265,20 @@ public ConnectorInfo connectorInfo(String connector) { ); } + protected Map> buildTasksConfig(String connector) { + final ClusterConfigState configState = configBackingStore.snapshot(); + + if (!configState.contains(connector)) + return Collections.emptyMap(); + + Map> configs = new HashMap<>(); + for (ConnectorTaskId cti : configState.tasks(connector)) { + configs.put(cti, configState.taskConfig(cti)); + } + + return configs; + } + @Override public ConnectorStateInfo connectorStatus(String connName) { ConnectorStatus connector = statusBackingStore.get(connName); @@ -632,7 +646,7 @@ private String trace(Throwable t) { ByteArrayOutputStream output = new ByteArrayOutputStream(); try { t.printStackTrace(new PrintStream(output, false, StandardCharsets.UTF_8.name())); - return output.toString("UTF-8"); + return output.toString(StandardCharsets.UTF_8.name()); } catch (UnsupportedEncodingException e) { return null; } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectMetricsRegistry.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectMetricsRegistry.java index 6996dacb39..87dad2caa6 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectMetricsRegistry.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectMetricsRegistry.java @@ -115,7 +115,7 @@ public class ConnectMetricsRegistry { public Map connectorStatusMetrics; public ConnectMetricsRegistry() { - this(new LinkedHashSet()); + this(new LinkedHashSet<>()); } public ConnectMetricsRegistry(Set tags) { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Herder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Herder.java index 03d462b9ad..5a803f8178 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Herder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Herder.java @@ -86,6 +86,13 @@ public interface Herder { */ void connectorConfig(String connName, Callback> callback); + /** + * Get the configuration for all tasks. + * @param connName name of the connector + * @param callback callback to invoke with the configuration + */ + void tasksConfig(String connName, Callback>> callback); + /** * Set the configuration for a connector. This supports creation and updating. * @param connName name of the connector diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SourceTaskOffsetCommitter.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SourceTaskOffsetCommitter.java index 012c7074ad..c3416be45c 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SourceTaskOffsetCommitter.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SourceTaskOffsetCommitter.java @@ -63,7 +63,7 @@ class SourceTaskOffsetCommitter { public SourceTaskOffsetCommitter(WorkerConfig config) { this(config, Executors.newSingleThreadScheduledExecutor(ThreadUtils.createThreadFactory( SourceTaskOffsetCommitter.class.getSimpleName() + "-%d", false)), - new ConcurrentHashMap>()); + new ConcurrentHashMap<>()); } public void close(long timeoutMs) { @@ -79,12 +79,9 @@ public void close(long timeoutMs) { public void schedule(final ConnectorTaskId id, final WorkerSourceTask workerTask) { long commitIntervalMs = config.getLong(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG); - ScheduledFuture commitFuture = commitExecutorService.scheduleWithFixedDelay(new Runnable() { - @Override - public void run() { - try (LoggingContext loggingContext = LoggingContext.forOffsets(id)) { - commit(workerTask); - } + ScheduledFuture commitFuture = commitExecutorService.scheduleWithFixedDelay(() -> { + try (LoggingContext loggingContext = LoggingContext.forOffsets(id)) { + commit(workerTask); } }, commitIntervalMs, commitIntervalMs, TimeUnit.MILLISECONDS); committers.put(id, commitFuture); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java index a137b2d8f6..58a9ce3d5a 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java @@ -444,7 +444,7 @@ protected Map postProcessParsedConfig(final Map public static List pluginLocations(Map props) { String locationList = props.get(WorkerConfig.PLUGIN_PATH_CONFIG); return locationList == null - ? new ArrayList() + ? new ArrayList<>() : Arrays.asList(COMMA_WITH_WHITESPACE.split(locationList.trim(), -1)); } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfigTransformer.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfigTransformer.java index 318626bd5c..4d9c4c1f74 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfigTransformer.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfigTransformer.java @@ -90,12 +90,9 @@ private void scheduleReload(String connectorName, String path, long ttl) { } } log.info("Scheduling a restart of connector {} in {} ms", connectorName, ttl); - Callback cb = new Callback() { - @Override - public void onCompletion(Throwable error, Void result) { - if (error != null) { - log.error("Unexpected error during connector restart: ", error); - } + Callback cb = (error, result) -> { + if (error != null) { + log.error("Unexpected error during connector restart: ", error); } }; HerderRequest request = worker.herder().restartConnector(ttl, connectorName, cb); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java index f95c6a7175..1229fec720 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java @@ -350,12 +350,7 @@ private void doCommitSync(Map offsets, int se private void doCommitAsync(Map offsets, final int seqno) { log.debug("{} Committing offsets asynchronously using sequence number {}: {}", this, seqno, offsets); - OffsetCommitCallback cb = new OffsetCommitCallback() { - @Override - public void onComplete(Map offsets, Exception error) { - onCommitCompleted(error, seqno, offsets); - } - }; + OffsetCommitCallback cb = (tpOffsets, error) -> onCommitCompleted(error, seqno, tpOffsets); consumer.commitAsync(offsets, cb); } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerTask.java index 0bd76300dc..5cd992052b 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerTask.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerTask.java @@ -18,15 +18,12 @@ import org.apache.kafka.common.MetricName; import org.apache.kafka.common.MetricNameTemplate; -import org.apache.kafka.common.metrics.Measurable; -import org.apache.kafka.common.metrics.MetricConfig; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.metrics.stats.Avg; import org.apache.kafka.common.metrics.stats.Frequencies; import org.apache.kafka.common.metrics.stats.Max; import org.apache.kafka.common.utils.Time; import org.apache.kafka.connect.runtime.AbstractStatus.State; -import org.apache.kafka.connect.runtime.ConnectMetrics.LiteralSupplier; import org.apache.kafka.connect.runtime.ConnectMetrics.MetricGroup; import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator; import org.apache.kafka.connect.runtime.isolation.Plugins; @@ -344,12 +341,9 @@ public TaskMetricsGroup(ConnectorTaskId id, ConnectMetrics connectMetrics, TaskS // prevent collisions by removing any previously created metrics in this group. metricGroup.close(); - metricGroup.addValueMetric(registry.taskStatus, new LiteralSupplier() { - @Override - public String metricValue(long now) { - return taskStateTimer.currentState().toString().toLowerCase(Locale.getDefault()); - } - }); + metricGroup.addValueMetric(registry.taskStatus, now -> + taskStateTimer.currentState().toString().toLowerCase(Locale.getDefault()) + ); addRatioMetric(State.RUNNING, registry.taskRunningRatio); addRatioMetric(State.PAUSED, registry.taskPauseRatio); @@ -372,12 +366,8 @@ public String metricValue(long now) { private void addRatioMetric(final State matchingState, MetricNameTemplate template) { MetricName metricName = metricGroup.metricName(template); if (metricGroup.metrics().metric(metricName) == null) { - metricGroup.metrics().addMetric(metricName, new Measurable() { - @Override - public double measure(MetricConfig config, long now) { - return taskStateTimer.durationRatio(matchingState, now); - } - }); + metricGroup.metrics().addMetric(metricName, (config, now) -> + taskStateTimer.durationRatio(matchingState, now)); } } 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 41b2b4afc9..16dfbf9702 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 @@ -28,6 +28,7 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.ThreadUtils; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.connector.Connector; import org.apache.kafka.connect.connector.policy.ConnectorClientConfigOverridePolicy; import org.apache.kafka.connect.errors.AlreadyExistsException; @@ -36,7 +37,6 @@ import org.apache.kafka.connect.runtime.AbstractHerder; import org.apache.kafka.connect.runtime.CloseableConnectorContext; import org.apache.kafka.connect.runtime.ConnectMetrics; -import org.apache.kafka.connect.runtime.ConnectMetrics.LiteralSupplier; import org.apache.kafka.connect.runtime.ConnectMetrics.MetricGroup; import org.apache.kafka.connect.runtime.ConnectMetricsRegistry; import org.apache.kafka.connect.runtime.ConnectorConfig; @@ -67,6 +67,7 @@ import javax.crypto.SecretKey; import javax.ws.rs.core.Response; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashSet; @@ -139,6 +140,7 @@ public class DistributedHerder extends AbstractHerder implements Runnable { private final Time time; private final HerderMetrics herderMetrics; + private final List uponShutdown; private final String workerGroupId; private final int workerSyncTimeoutMs; @@ -186,6 +188,22 @@ public class DistributedHerder extends AbstractHerder implements Runnable { private final DistributedConfig config; + /** + * Create a herder that will form a Connect cluster with other {@link DistributedHerder} instances (in this or other JVMs) + * that have the same group ID. + * + * @param config the configuration for the worker; may not be null + * @param time the clock to use; may not be null + * @param worker the {@link Worker} instance to use; may not be null + * @param kafkaClusterId the identifier of the Kafka cluster to use for internal topics; may not be null + * @param statusBackingStore the backing store for statuses; may not be null + * @param configBackingStore the backing store for connector configurations; may not be null + * @param restUrl the URL of this herder's REST API; may not be null + * @param connectorClientConfigOverridePolicy the policy specifying the client configuration properties that may be overridden + * in connector configurations; may not be null + * @param uponShutdown any {@link AutoCloseable} objects that should be closed when this herder is {@link #stop() stopped}, + * after all services and resources owned by this herder are stopped + */ public DistributedHerder(DistributedConfig config, Time time, Worker worker, @@ -193,9 +211,10 @@ public DistributedHerder(DistributedConfig config, StatusBackingStore statusBackingStore, ConfigBackingStore configBackingStore, String restUrl, - ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy) { + ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy, + AutoCloseable... uponShutdown) { this(config, worker, worker.workerId(), kafkaClusterId, statusBackingStore, configBackingStore, null, restUrl, worker.metrics(), - time, connectorClientConfigOverridePolicy); + time, connectorClientConfigOverridePolicy, uponShutdown); configBackingStore.setUpdateListener(new ConfigUpdateListener()); } @@ -210,7 +229,8 @@ public DistributedHerder(DistributedConfig config, String restUrl, ConnectMetrics metrics, Time time, - ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy) { + ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy, + AutoCloseable... uponShutdown) { super(worker, workerId, kafkaClusterId, statusBackingStore, configBackingStore, connectorClientConfigOverridePolicy); this.time = time; @@ -224,6 +244,7 @@ public DistributedHerder(DistributedConfig config, this.keySignatureVerificationAlgorithms = config.getList(DistributedConfig.INTER_WORKER_VERIFICATION_ALGORITHMS_CONFIG); this.keyGenerator = config.getInternalRequestKeyGenerator(); this.isTopicTrackingEnabled = config.getBoolean(TOPIC_TRACKING_ENABLE_CONFIG); + this.uponShutdown = Arrays.asList(uponShutdown); String clientIdConfig = config.getString(CommonClientConfigs.CLIENT_ID_CONFIG); String clientId = clientIdConfig.length() <= 0 ? "connect-" + CONNECT_CLIENT_ID_SEQUENCE.getAndIncrement() : clientIdConfig; @@ -237,7 +258,7 @@ public DistributedHerder(DistributedConfig config, this.herderExecutor = new ThreadPoolExecutor(1, 1, 0L, TimeUnit.MILLISECONDS, - new LinkedBlockingDeque(1), + new LinkedBlockingDeque<>(1), ThreadUtils.createThreadFactory( this.getClass().getSimpleName() + "-" + clientId + "-%d", false)); @@ -578,7 +599,7 @@ private synchronized boolean updateConfigsWithIncrementalCooperative(AtomicRefer private void processConnectorConfigUpdates(Set connectorConfigUpdates) { // If we only have connector config updates, we can just bounce the updated connectors that are // currently assigned to this worker. - Set localConnectors = assignment == null ? Collections.emptySet() : new HashSet<>(assignment.connectors()); + Set localConnectors = assignment == null ? Collections.emptySet() : new HashSet<>(assignment.connectors()); log.trace("Processing connector config updates; " + "currently-owned connectors are {}, and to-be-updated connectors are {}", localConnectors, @@ -677,6 +698,15 @@ public void halt() { } } + @Override + protected void stopServices() { + try { + super.stopServices(); + } finally { + this.uponShutdown.forEach(closeable -> Utils.closeQuietly(closeable, closeable != null ? closeable.toString() : "")); + } + } + @Override public void stop() { log.info("Herder stopping"); @@ -708,17 +738,12 @@ public void connectors(final Callback> callback) { log.trace("Submitting connector listing request"); addRequest( - new Callable() { - @Override - public Void call() throws Exception { - if (checkRebalanceNeeded(callback)) - return null; - - callback.onCompletion(null, configState.connectors()); - return null; - } - }, - forwardErrorCallback(callback) + () -> { + if (!checkRebalanceNeeded(callback)) + callback.onCompletion(null, configState.connectors()); + return null; + }, + forwardErrorCallback(callback) ); } @@ -727,21 +752,39 @@ public void connectorInfo(final String connName, final Callback c log.trace("Submitting connector info request {}", connName); addRequest( - new Callable() { - @Override - public Void call() throws Exception { - if (checkRebalanceNeeded(callback)) - return null; + () -> { + if (checkRebalanceNeeded(callback)) + return null; - if (!configState.contains(connName)) { - callback.onCompletion(new NotFoundException("Connector " + connName + " not found"), null); - } else { - callback.onCompletion(null, connectorInfo(connName)); - } - return null; - } - }, - forwardErrorCallback(callback) + if (!configState.contains(connName)) { + callback.onCompletion( + new NotFoundException("Connector " + connName + " not found"), null); + } else { + callback.onCompletion(null, connectorInfo(connName)); + } + return null; + }, + forwardErrorCallback(callback) + ); + } + + @Override + public void tasksConfig(String connName, final Callback>> callback) { + log.trace("Submitting tasks config request {}", connName); + + addRequest( + () -> { + if (checkRebalanceNeeded(callback)) + return null; + + if (!configState.contains(connName)) { + callback.onCompletion(new NotFoundException("Connector " + connName + " not found"), null); + } else { + callback.onCompletion(null, buildTasksConfig(connName)); + } + return null; + }, + forwardErrorCallback(callback) ); } @@ -759,26 +802,23 @@ public void connectorConfig(String connName, final Callback> @Override public void deleteConnectorConfig(final String connName, final Callback> callback) { addRequest( - new Callable() { - @Override - public Void call() throws Exception { - log.trace("Handling connector config request {}", connName); - if (!isLeader()) { - callback.onCompletion(new NotLeaderException("Only the leader can delete connector configs.", leaderUrl()), null); - return null; - } + () -> { + log.trace("Handling connector config request {}", connName); + if (!isLeader()) { + callback.onCompletion(new NotLeaderException("Only the leader can delete connector configs.", leaderUrl()), null); + return null; + } - if (!configState.contains(connName)) { - callback.onCompletion(new NotFoundException("Connector " + connName + " not found"), null); - } else { - log.trace("Removing connector config {} {}", connName, configState.connectors()); - configBackingStore.removeConnectorConfig(connName); - callback.onCompletion(null, new Created(false, null)); - } - return null; - } - }, - forwardErrorCallback(callback) + if (!configState.contains(connName)) { + callback.onCompletion(new NotFoundException("Connector " + connName + " not found"), null); + } else { + log.trace("Removing connector config {} {}", connName, configState.connectors()); + configBackingStore.removeConnectorConfig(connName); + callback.onCompletion(null, new Created<>(false, null)); + } + return null; + }, + forwardErrorCallback(callback) ); } @@ -804,56 +844,50 @@ public void putConnectorConfig(final String connName, final Map final Callback> callback) { log.trace("Submitting connector config write request {}", connName); addRequest( - new Callable() { - @Override - public Void call() throws Exception { - validateConnectorConfig(config, (error, configInfos) -> { - if (error != null) { - callback.onCompletion(error, null); - return; + () -> { + validateConnectorConfig(config, (error, configInfos) -> { + if (error != null) { + callback.onCompletion(error, null); + return; + } + + // Complete the connector config write via another herder request in order to + // perform the write to the backing store (or forward to the leader) during + // the "external request" portion of the tick loop + addRequest( + () -> { + if (maybeAddConfigErrors(configInfos, callback)) { + return null; } - // Complete the connector config write via another herder request in order to - // perform the write to the backing store (or forward to the leader) during - // the "external request" portion of the tick loop - addRequest( - new Callable() { - @Override - public Void call() { - if (maybeAddConfigErrors(configInfos, callback)) { - return null; - } - - log.trace("Handling connector config request {}", connName); - if (!isLeader()) { - callback.onCompletion(new NotLeaderException("Only the leader can set connector configs.", leaderUrl()), null); - return null; - } - boolean exists = configState.contains(connName); - if (!allowReplace && exists) { - callback.onCompletion(new AlreadyExistsException("Connector " + connName + " already exists"), null); - return null; - } - - log.trace("Submitting connector config {} {} {}", connName, allowReplace, configState.connectors()); - configBackingStore.putConnectorConfig(connName, config); - - // Note that we use the updated connector config despite the fact that we don't have an updated - // snapshot yet. The existing task info should still be accurate. - ConnectorInfo info = new ConnectorInfo(connName, config, configState.tasks(connName), - // validateConnectorConfig have checked the existence of CONNECTOR_CLASS_CONFIG - connectorTypeForClass(config.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG))); - callback.onCompletion(null, new Created<>(!exists, info)); - return null; - } - }, - forwardErrorCallback(callback) - ); - }); - return null; - } - }, - forwardErrorCallback(callback) + log.trace("Handling connector config request {}", connName); + if (!isLeader()) { + callback.onCompletion(new NotLeaderException("Only the leader can set connector configs.", leaderUrl()), null); + return null; + } + boolean exists = configState.contains(connName); + if (!allowReplace && exists) { + callback.onCompletion(new AlreadyExistsException("Connector " + connName + " already exists"), null); + return null; + } + + log.trace("Submitting connector config {} {} {}", connName, allowReplace, configState.connectors()); + configBackingStore.putConnectorConfig(connName, config); + + // Note that we use the updated connector config despite the fact that we don't have an updated + // snapshot yet. The existing task info should still be accurate. + ConnectorInfo info = new ConnectorInfo(connName, config, configState.tasks(connName), + // validateConnectorConfig have checked the existence of CONNECTOR_CLASS_CONFIG + connectorTypeForClass(config.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG))); + callback.onCompletion(null, new Created<>(!exists, info)); + return null; + }, + forwardErrorCallback(callback) + ); + }); + return null; + }, + forwardErrorCallback(callback) ); } @@ -862,12 +896,9 @@ public void requestTaskReconfiguration(final String connName) { log.trace("Submitting connector task reconfiguration request {}", connName); addRequest( - new Callable() { - @Override - public Void call() throws Exception { - reconfigureConnectorTasksWithRetry(time.milliseconds(), connName); - return null; - } + () -> { + reconfigureConnectorTasksWithRetry(time.milliseconds(), connName); + return null; }, (error, result) -> { if (error != null) { @@ -883,26 +914,23 @@ public void taskConfigs(final String connName, final Callback> ca log.trace("Submitting get task configuration request {}", connName); addRequest( - new Callable() { - @Override - public Void call() throws Exception { - if (checkRebalanceNeeded(callback)) - return null; + () -> { + if (checkRebalanceNeeded(callback)) + return null; - if (!configState.contains(connName)) { - callback.onCompletion(new NotFoundException("Connector " + connName + " not found"), null); - } else { - List result = new ArrayList<>(); - for (int i = 0; i < configState.taskCount(connName); i++) { - ConnectorTaskId id = new ConnectorTaskId(connName, i); - result.add(new TaskInfo(id, configState.rawTaskConfig(id))); - } - callback.onCompletion(null, result); - } - return null; + if (!configState.contains(connName)) { + callback.onCompletion(new NotFoundException("Connector " + connName + " not found"), null); + } else { + List result = new ArrayList<>(); + for (int i = 0; i < configState.taskCount(connName); i++) { + ConnectorTaskId id = new ConnectorTaskId(connName, i); + result.add(new TaskInfo(id, configState.rawTaskConfig(id))); } - }, - forwardErrorCallback(callback) + callback.onCompletion(null, result); + } + return null; + }, + forwardErrorCallback(callback) ); } @@ -937,21 +965,18 @@ public void putTaskConfigs(final String connName, final List } addRequest( - new Callable() { - @Override - public Void call() throws Exception { - if (!isLeader()) - callback.onCompletion(new NotLeaderException("Only the leader may write task configurations.", leaderUrl()), null); - else if (!configState.contains(connName)) - callback.onCompletion(new NotFoundException("Connector " + connName + " not found"), null); - else { - configBackingStore.putTaskConfigs(connName, configs); - callback.onCompletion(null, null); - } - return null; - } - }, - forwardErrorCallback(callback) + () -> { + if (!isLeader()) + callback.onCompletion(new NotLeaderException("Only the leader may write task configurations.", leaderUrl()), null); + else if (!configState.contains(connName)) + callback.onCompletion(new NotFoundException("Connector " + connName + " not found"), null); + else { + configBackingStore.putTaskConfigs(connName, configs); + callback.onCompletion(null, null); + } + return null; + }, + forwardErrorCallback(callback) ); } @@ -962,9 +987,9 @@ public void restartConnector(final String connName, final Callback callbac @Override public HerderRequest restartConnector(final long delayMs, final String connName, final Callback callback) { - return addRequest(delayMs, new Callable() { - @Override - public Void call() throws Exception { + return addRequest( + delayMs, + () -> { if (checkRebalanceNeeded(callback)) return null; @@ -986,15 +1011,14 @@ public Void call() throws Exception { callback.onCompletion(new NotLeaderException("Cannot restart connector since it is not assigned to this member", leaderUrl()), null); } return null; - } - }, forwardErrorCallback(callback)); + }, + forwardErrorCallback(callback)); } @Override public void restartTask(final ConnectorTaskId id, final Callback callback) { - addRequest(new Callable() { - @Override - public Void call() throws Exception { + addRequest( + () -> { if (checkRebalanceNeeded(callback)) return null; @@ -1024,8 +1048,8 @@ public Void call() throws Exception { callback.onCompletion(new NotLeaderException("Cannot restart task since it is not assigned to this member", leaderUrl()), null); } return null; - } - }, forwardErrorCallback(callback)); + }, + forwardErrorCallback(callback)); } @Override @@ -1261,28 +1285,22 @@ private boolean startTask(ConnectorTaskId taskId) { } private Callable getTaskStartingCallable(final ConnectorTaskId taskId) { - return new Callable() { - @Override - public Void call() throws Exception { - try { - startTask(taskId); - } catch (Throwable t) { - log.error("Couldn't instantiate task {} because it has an invalid task configuration. This task will not execute until reconfigured.", - taskId, t); - onFailure(taskId, t); - } - return null; + return () -> { + try { + startTask(taskId); + } catch (Throwable t) { + log.error("Couldn't instantiate task {} because it has an invalid task configuration. This task will not execute until reconfigured.", + taskId, t); + onFailure(taskId, t); } + return null; }; } private Callable getTaskStoppingCallable(final ConnectorTaskId taskId) { - return new Callable() { - @Override - public Void call() throws Exception { - worker.stopAndAwaitTask(taskId); - return null; - } + return () -> { + worker.stopAndAwaitTask(taskId); + return null; }; } @@ -1303,18 +1321,15 @@ private void startConnector(String connectorName, Callback callback) { // Use newState here in case the connector has been paused right after being created if (newState == TargetState.STARTED) { addRequest( - new Callable() { - @Override - public Void call() { - // Request configuration since this could be a brand new connector. However, also only update those - // task configs if they are actually different from the existing ones to avoid unnecessary updates when this is - // just restoring an existing connector. - reconfigureConnectorTasksWithRetry(time.milliseconds(), connectorName); - callback.onCompletion(null, null); - return null; - } - }, - forwardErrorCallback(callback) + () -> { + // Request configuration since this could be a brand new connector. However, also only update those + // task configs if they are actually different from the existing ones to avoid unnecessary updates when this is + // just restoring an existing connector. + reconfigureConnectorTasksWithRetry(time.milliseconds(), connectorName); + callback.onCompletion(null, null); + return null; + }, + forwardErrorCallback(callback) ); } else { callback.onCompletion(null, null); @@ -1324,70 +1339,55 @@ public Void call() { } private Callable getConnectorStartingCallable(final String connectorName) { - return new Callable() { - @Override - public Void call() throws Exception { - try { - startConnector(connectorName, (error, result) -> { - if (error != null) { - log.error("Failed to start connector '" + connectorName + "'", error); - } - }); - } catch (Throwable t) { - log.error("Unexpected error while trying to start connector " + connectorName, t); - onFailure(connectorName, t); - } - return null; + return () -> { + try { + startConnector(connectorName, (error, result) -> { + if (error != null) { + log.error("Failed to start connector '" + connectorName + "'", error); + } + }); + } catch (Throwable t) { + log.error("Unexpected error while trying to start connector " + connectorName, t); + onFailure(connectorName, t); } + return null; }; } private Callable getConnectorStoppingCallable(final String connectorName) { - return new Callable() { - @Override - public Void call() throws Exception { - try { - worker.stopAndAwaitConnector(connectorName); - } catch (Throwable t) { - log.error("Failed to shut down connector " + connectorName, t); - } - return null; + return () -> { + try { + worker.stopAndAwaitConnector(connectorName); + } catch (Throwable t) { + log.error("Failed to shut down connector " + connectorName, t); } + return null; }; } private void reconfigureConnectorTasksWithRetry(long initialRequestTime, final String connName) { - reconfigureConnector(connName, new Callback() { - @Override - public void onCompletion(Throwable error, Void result) { - // If we encountered an error, we don't have much choice but to just retry. If we don't, we could get - // stuck with a connector that thinks it has generated tasks, but wasn't actually successful and therefore - // never makes progress. The retry has to run through a DistributedHerderRequest since this callback could be happening - // from the HTTP request forwarding thread. - if (error != null) { - if (isPossibleExpiredKeyException(initialRequestTime, error)) { - log.debug("Failed to reconfigure connector's tasks ({}), possibly due to expired session key. Retrying after backoff", connName); - } else { - log.error("Failed to reconfigure connector's tasks ({}), retrying after backoff:", connName, error); - } - addRequest(RECONFIGURE_CONNECTOR_TASKS_BACKOFF_MS, - new Callable() { - @Override - public Void call() throws Exception { - reconfigureConnectorTasksWithRetry(initialRequestTime, connName); - return null; - } - }, new Callback() { - @Override - public void onCompletion(Throwable error, Void result) { - if (error != null) { - log.error("Unexpected error during connector task reconfiguration: ", error); - log.error("Task reconfiguration for {} failed unexpectedly, this connector will not be properly reconfigured unless manually triggered.", connName); - } - } - } - ); + reconfigureConnector(connName, (error, result) -> { + // If we encountered an error, we don't have much choice but to just retry. If we don't, we could get + // stuck with a connector that thinks it has generated tasks, but wasn't actually successful and therefore + // never makes progress. The retry has to run through a DistributedHerderRequest since this callback could be happening + // from the HTTP request forwarding thread. + if (error != null) { + if (isPossibleExpiredKeyException(initialRequestTime, error)) { + log.debug("Failed to reconfigure connector's tasks ({}), possibly due to expired session key. Retrying after backoff", connName); + } else { + log.error("Failed to reconfigure connector's tasks ({}), retrying after backoff:", connName, error); } + addRequest(RECONFIGURE_CONNECTOR_TASKS_BACKOFF_MS, + () -> { + reconfigureConnectorTasksWithRetry(initialRequestTime, connName); + return null; + }, (err, res) -> { + if (err != null) { + log.error("Unexpected error during connector task reconfiguration: ", err); + log.error("Task reconfiguration for {} failed unexpectedly, this connector will not be properly reconfigured unless manually triggered.", connName); + } + } + ); } }); } @@ -1807,30 +1807,12 @@ public HerderMetrics(ConnectMetrics connectMetrics) { ConnectMetricsRegistry registry = connectMetrics.registry(); metricGroup = connectMetrics.group(registry.workerRebalanceGroupName()); - metricGroup.addValueMetric(registry.connectProtocol, new LiteralSupplier() { - @Override - public String metricValue(long now) { - return ConnectProtocolCompatibility.fromProtocolVersion(member.currentProtocolVersion()).name(); - } - }); - metricGroup.addValueMetric(registry.leaderName, new LiteralSupplier() { - @Override - public String metricValue(long now) { - return leaderUrl(); - } - }); - metricGroup.addValueMetric(registry.epoch, new LiteralSupplier() { - @Override - public Double metricValue(long now) { - return (double) generation; - } - }); - metricGroup.addValueMetric(registry.rebalanceMode, new LiteralSupplier() { - @Override - public Double metricValue(long now) { - return rebalancing ? 1.0d : 0.0d; - } - }); + metricGroup.addValueMetric(registry.connectProtocol, now -> + ConnectProtocolCompatibility.fromProtocolVersion(member.currentProtocolVersion()).name() + ); + metricGroup.addValueMetric(registry.leaderName, now -> leaderUrl()); + metricGroup.addValueMetric(registry.epoch, now -> (double) generation); + metricGroup.addValueMetric(registry.rebalanceMode, now -> rebalancing ? 1.0d : 0.0d); rebalanceCompletedCounts = metricGroup.sensor("completed-rebalance-count"); rebalanceCompletedCounts.add(metricGroup.metricName(registry.rebalanceCompletedTotal), new CumulativeSum()); @@ -1839,12 +1821,8 @@ public Double metricValue(long now) { rebalanceTime.add(metricGroup.metricName(registry.rebalanceTimeMax), new Max()); rebalanceTime.add(metricGroup.metricName(registry.rebalanceTimeAvg), new Avg()); - metricGroup.addValueMetric(registry.rebalanceTimeSinceLast, new LiteralSupplier() { - @Override - public Double metricValue(long now) { - return lastRebalanceCompletedAtMillis == Long.MIN_VALUE ? Double.POSITIVE_INFINITY : (double) (now - lastRebalanceCompletedAtMillis); - } - }); + metricGroup.addValueMetric(registry.rebalanceTimeSinceLast, now -> + lastRebalanceCompletedAtMillis == Long.MIN_VALUE ? Double.POSITIVE_INFINITY : (double) (now - lastRebalanceCompletedAtMillis)); } void close() { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/IncrementalCooperativeAssignor.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/IncrementalCooperativeAssignor.java index c3f2f4bc44..2cfaf1d3eb 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/IncrementalCooperativeAssignor.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/IncrementalCooperativeAssignor.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.connect.runtime.distributed; +import java.util.Map.Entry; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.connect.runtime.distributed.WorkerCoordinator.ConnectorsAndTasks; @@ -368,7 +369,7 @@ private ConnectorsAndTasks duplicatedAssignments(Map entry.getValue() > 1L) - .map(entry -> entry.getKey()) + .map(Entry::getKey) .collect(Collectors.toSet()); Set tasks = memberConfigs.values().stream() @@ -376,7 +377,7 @@ private ConnectorsAndTasks duplicatedAssignments(Map entry.getValue() > 1L) - .map(entry -> entry.getKey()) + .map(Entry::getKey) .collect(Collectors.toSet()); return new ConnectorsAndTasks.Builder().with(connectors, tasks).build(); } 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 0e021a8097..110ef00a01 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 @@ -20,7 +20,6 @@ import org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient; import org.apache.kafka.clients.GroupRebalanceConfig; import org.apache.kafka.common.metrics.Measurable; -import org.apache.kafka.common.metrics.MetricConfig; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.requests.JoinGroupRequest; import org.apache.kafka.common.utils.LogContext; @@ -347,26 +346,20 @@ private class WorkerCoordinatorMetrics { public WorkerCoordinatorMetrics(Metrics metrics, String metricGrpPrefix) { this.metricGrpName = metricGrpPrefix + "-coordinator-metrics"; - Measurable numConnectors = new Measurable() { - @Override - public double measure(MetricConfig config, long now) { - final ExtendedAssignment localAssignmentSnapshot = assignmentSnapshot; - if (localAssignmentSnapshot == null) { - return 0.0; - } - return localAssignmentSnapshot.connectors().size(); + Measurable numConnectors = (config, now) -> { + final ExtendedAssignment localAssignmentSnapshot = assignmentSnapshot; + if (localAssignmentSnapshot == null) { + return 0.0; } + return localAssignmentSnapshot.connectors().size(); }; - Measurable numTasks = new Measurable() { - @Override - public double measure(MetricConfig config, long now) { - final ExtendedAssignment localAssignmentSnapshot = assignmentSnapshot; - if (localAssignmentSnapshot == null) { - return 0.0; - } - return localAssignmentSnapshot.tasks().size(); + Measurable numTasks = (config, now) -> { + final ExtendedAssignment localAssignmentSnapshot = assignmentSnapshot; + if (localAssignmentSnapshot == null) { + return 0.0; } + return localAssignmentSnapshot.tasks().size(); }; metrics.addMetric(metrics.metricName("assigned-connectors", diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/errors/DeadLetterQueueReporter.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/errors/DeadLetterQueueReporter.java index e07a5d0118..a4480f4c1c 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/errors/DeadLetterQueueReporter.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/errors/DeadLetterQueueReporter.java @@ -183,7 +183,7 @@ void populateContextHeaders(ProducerRecord producerRecord, Proce private byte[] stacktrace(Throwable error) { ByteArrayOutputStream bos = new ByteArrayOutputStream(); try { - PrintStream stream = new PrintStream(bos, true, "UTF-8"); + PrintStream stream = new PrintStream(bos, true, StandardCharsets.UTF_8.name()); error.printStackTrace(stream); bos.close(); return bos.toByteArray(); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/DelegatingClassLoader.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/DelegatingClassLoader.java index 86a53c75ce..c894dd6232 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/DelegatingClassLoader.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/DelegatingClassLoader.java @@ -293,32 +293,29 @@ private void loadJdbcDrivers(final ClassLoader loader) { // Apply here what java.sql.DriverManager does to discover and register classes // implementing the java.sql.Driver interface. AccessController.doPrivileged( - new PrivilegedAction() { - @Override - public Void run() { - ServiceLoader loadedDrivers = ServiceLoader.load( - Driver.class, - loader + (PrivilegedAction) () -> { + ServiceLoader loadedDrivers = ServiceLoader.load( + Driver.class, + loader + ); + Iterator driversIterator = loadedDrivers.iterator(); + try { + while (driversIterator.hasNext()) { + Driver driver = driversIterator.next(); + log.debug( + "Registered java.sql.Driver: {} to java.sql.DriverManager", + driver ); - Iterator driversIterator = loadedDrivers.iterator(); - try { - while (driversIterator.hasNext()) { - Driver driver = driversIterator.next(); - log.debug( - "Registered java.sql.Driver: {} to java.sql.DriverManager", - driver - ); - } - } catch (Throwable t) { - log.debug( - "Ignoring java.sql.Driver classes listed in resources but not" - + " present in class loader's classpath: ", - t - ); - } - return null; } + } catch (Throwable t) { + log.debug( + "Ignoring java.sql.Driver classes listed in resources but not" + + " present in class loader's classpath: ", + t + ); } + return null; + } ); } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/PluginUtils.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/PluginUtils.java index 8b21e7324a..12cb1867d8 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/PluginUtils.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/PluginUtils.java @@ -142,13 +142,8 @@ public class PluginUtils { + "|common\\.config\\.provider\\.(?!ConfigProvider$).*" + ")$"); - private static final DirectoryStream.Filter PLUGIN_PATH_FILTER = new DirectoryStream - .Filter() { - @Override - public boolean accept(Path path) { - return Files.isDirectory(path) || isArchive(path) || isClassFile(path); - } - }; + private static final DirectoryStream.Filter PLUGIN_PATH_FILTER = path -> + Files.isDirectory(path) || isArchive(path) || isClassFile(path); /** * Return whether the class with the given name should be loaded in isolation using a plugin diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestClient.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestClient.java index b381259716..58d7df0ca4 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestClient.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestClient.java @@ -172,7 +172,7 @@ private static void addHeadersToRequest(HttpHeaders headers, Request req) { * @return */ private static Map convertHttpFieldsToMap(HttpFields httpFields) { - Map headers = new HashMap(); + Map headers = new HashMap<>(); if (httpFields == null || httpFields.size() == 0) return headers; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java index b3219b2204..27c250af6b 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java @@ -188,6 +188,17 @@ public Map getConnectorConfig(final @PathParam("connector") Stri return completeOrForwardRequest(cb, "/connectors/" + connector + "/config", "GET", headers, null, forward); } + @GET + @Path("/{connector}/tasks-config") + public Map> getTasksConfig( + final @PathParam("connector") String connector, + final @Context HttpHeaders headers, + final @QueryParam("forward") Boolean forward) throws Throwable { + FutureCallback>> cb = new FutureCallback<>(); + herder.tasksConfig(connector, cb); + return completeOrForwardRequest(cb, "/connectors/" + connector + "/tasks-config", "GET", headers, null, forward); + } + @GET @Path("/{connector}/status") public ConnectorStateInfo getConnectorStatus(final @PathParam("connector") String connector) { @@ -390,12 +401,12 @@ private T completeOrForwardRequest(FutureCallback cb, private T completeOrForwardRequest(FutureCallback cb, String path, String method, HttpHeaders headers, Object body, TypeReference resultType, Boolean forward) throws Throwable { - return completeOrForwardRequest(cb, path, method, headers, body, resultType, new IdentityTranslator(), forward); + return completeOrForwardRequest(cb, path, method, headers, body, resultType, new IdentityTranslator<>(), forward); } private T completeOrForwardRequest(FutureCallback cb, String path, String method, HttpHeaders headers, Object body, Boolean forward) throws Throwable { - return completeOrForwardRequest(cb, path, method, headers, body, null, new IdentityTranslator(), forward); + return completeOrForwardRequest(cb, path, method, headers, body, null, new IdentityTranslator<>(), forward); } private interface Translator { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java index 8da8c6e93a..ba8de91c7c 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java @@ -166,7 +166,7 @@ public synchronized void deleteConnectorConfig(String connName, Callback(false, null)); + callback.onCompletion(null, new Created<>(false, null)); } catch (ConnectException e) { callback.onCompletion(e, null); } @@ -392,9 +392,7 @@ public void onConnectorTargetStateChange(String connector) { } if (newState == TargetState.STARTED) { - requestExecutorService.submit(() -> { - updateConnectorTasks(connector); - }); + requestExecutorService.submit(() -> updateConnectorTasks(connector)); } }); } @@ -434,4 +432,15 @@ public int hashCode() { return Objects.hash(seq); } } + + @Override + public void tasksConfig(String connName, Callback>> callback) { + Map> tasksConfig = buildTasksConfig(connName); + if (tasksConfig.isEmpty()) { + callback.onCompletion(new NotFoundException("Connector " + connName + " not found"), tasksConfig); + return; + } + callback.onCompletion(null, tasksConfig); + } + } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java index fbcc35bb4b..d4e6358e2e 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java @@ -62,6 +62,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.function.Supplier; /** *

@@ -224,6 +225,7 @@ public static String COMMIT_TASKS_KEY(String connectorName) { // Connector and task configs: name or id -> config map private final Map> connectorConfigs = new HashMap<>(); private final Map> taskConfigs = new HashMap<>(); + private final Supplier topicAdminSupplier; // Set of connectors where we saw a task commit with an incomplete set of task config updates, indicating the data // is in an inconsistent state and we cannot safely use them until they have been refreshed. @@ -241,11 +243,17 @@ public static String COMMIT_TASKS_KEY(String connectorName) { private final WorkerConfigTransformer configTransformer; + @Deprecated public KafkaConfigBackingStore(Converter converter, WorkerConfig config, WorkerConfigTransformer configTransformer) { + this(converter, config, configTransformer, null); + } + + public KafkaConfigBackingStore(Converter converter, WorkerConfig config, WorkerConfigTransformer configTransformer, Supplier adminSupplier) { this.lock = new Object(); this.started = false; this.converter = converter; this.offset = -1; + this.topicAdminSupplier = adminSupplier; this.topic = config.getString(DistributedConfig.CONFIG_TOPIC_CONFIG); if (this.topic == null || this.topic.trim().length() == 0) @@ -471,6 +479,7 @@ KafkaBasedLog setupAndCreateKafkaBasedLog(String topic, final Wo Map adminProps = new HashMap<>(originals); ConnectUtils.addMetricsContextProperties(adminProps, config, clusterId); + Supplier adminSupplier = topicAdminSupplier != null ? topicAdminSupplier : () -> new TopicAdmin(adminProps); Map topicSettings = config instanceof DistributedConfig ? ((DistributedConfig) config).configStorageTopicSettings() : Collections.emptyMap(); @@ -481,30 +490,25 @@ KafkaBasedLog setupAndCreateKafkaBasedLog(String topic, final Wo .replicationFactor(config.getShort(DistributedConfig.CONFIG_STORAGE_REPLICATION_FACTOR_CONFIG)) .build(); - return createKafkaBasedLog(topic, producerProps, consumerProps, new ConsumeCallback(), topicDescription, adminProps); + return createKafkaBasedLog(topic, producerProps, consumerProps, new ConsumeCallback(), topicDescription, adminSupplier); } private KafkaBasedLog createKafkaBasedLog(String topic, Map producerProps, Map consumerProps, Callback> consumedCallback, - final NewTopic topicDescription, final Map adminProps) { - Runnable createTopics = new Runnable() { - @Override - public void run() { - log.debug("Creating admin client to manage Connect internal config topic"); - try (TopicAdmin admin = new TopicAdmin(adminProps)) { - // Create the topic if it doesn't exist - Set newTopics = admin.createTopics(topicDescription); - if (!newTopics.contains(topic)) { - // It already existed, so check that the topic cleanup policy is compact only and not delete - log.debug("Using admin client to check cleanup policy of '{}' topic is '{}'", topic, TopicConfig.CLEANUP_POLICY_COMPACT); - admin.verifyTopicCleanupPolicyOnlyCompact(topic, - DistributedConfig.CONFIG_TOPIC_CONFIG, "connector configurations"); - } - } + final NewTopic topicDescription, Supplier adminSupplier) { + java.util.function.Consumer createTopics = admin -> { + log.debug("Creating admin client to manage Connect internal config topic"); + // Create the topic if it doesn't exist + Set newTopics = admin.createTopics(topicDescription); + if (!newTopics.contains(topic)) { + // It already existed, so check that the topic cleanup policy is compact only and not delete + log.debug("Using admin client to check cleanup policy of '{}' topic is '{}'", topic, TopicConfig.CLEANUP_POLICY_COMPACT); + admin.verifyTopicCleanupPolicyOnlyCompact(topic, + DistributedConfig.CONFIG_TOPIC_CONFIG, "connector configurations"); } }; - return new KafkaBasedLog<>(topic, producerProps, consumerProps, consumedCallback, Time.SYSTEM, createTopics); + return new KafkaBasedLog<>(topic, producerProps, consumerProps, adminSupplier, consumedCallback, Time.SYSTEM, createTopics); } @SuppressWarnings("unchecked") diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStore.java index 8408f99c10..26b47f996b 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStore.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStore.java @@ -41,11 +41,13 @@ import java.util.Collections; import java.util.HashMap; import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.function.Supplier; /** *

@@ -62,6 +64,16 @@ public class KafkaOffsetBackingStore implements OffsetBackingStore { private KafkaBasedLog offsetLog; private HashMap data; + private final Supplier topicAdminSupplier; + + @Deprecated + public KafkaOffsetBackingStore() { + this.topicAdminSupplier = null; + } + + public KafkaOffsetBackingStore(Supplier topicAdmin) { + this.topicAdminSupplier = Objects.requireNonNull(topicAdmin); + } @Override public void configure(final WorkerConfig config) { @@ -86,6 +98,7 @@ public void configure(final WorkerConfig config) { Map adminProps = new HashMap<>(originals); ConnectUtils.addMetricsContextProperties(adminProps, config, clusterId); + Supplier adminSupplier = topicAdminSupplier != null ? topicAdminSupplier : () -> new TopicAdmin(adminProps); Map topicSettings = config instanceof DistributedConfig ? ((DistributedConfig) config).offsetStorageTopicSettings() : Collections.emptyMap(); @@ -96,30 +109,25 @@ public void configure(final WorkerConfig config) { .replicationFactor(config.getShort(DistributedConfig.OFFSET_STORAGE_REPLICATION_FACTOR_CONFIG)) .build(); - offsetLog = createKafkaBasedLog(topic, producerProps, consumerProps, consumedCallback, topicDescription, adminProps); + offsetLog = createKafkaBasedLog(topic, producerProps, consumerProps, consumedCallback, topicDescription, adminSupplier); } private KafkaBasedLog createKafkaBasedLog(String topic, Map producerProps, Map consumerProps, Callback> consumedCallback, - final NewTopic topicDescription, final Map adminProps) { - Runnable createTopics = new Runnable() { - @Override - public void run() { - log.debug("Creating admin client to manage Connect internal offset topic"); - try (TopicAdmin admin = new TopicAdmin(adminProps)) { - // Create the topic if it doesn't exist - Set newTopics = admin.createTopics(topicDescription); - if (!newTopics.contains(topic)) { - // It already existed, so check that the topic cleanup policy is compact only and not delete - log.debug("Using admin client to check cleanup policy for '{}' topic is '{}'", topic, TopicConfig.CLEANUP_POLICY_COMPACT); - admin.verifyTopicCleanupPolicyOnlyCompact(topic, - DistributedConfig.OFFSET_STORAGE_TOPIC_CONFIG, "source connector offsets"); - } - } + final NewTopic topicDescription, Supplier adminSupplier) { + java.util.function.Consumer createTopics = admin -> { + log.debug("Creating admin client to manage Connect internal offset topic"); + // Create the topic if it doesn't exist + Set newTopics = admin.createTopics(topicDescription); + if (!newTopics.contains(topic)) { + // It already existed, so check that the topic cleanup policy is compact only and not delete + log.debug("Using admin client to check cleanup policy for '{}' topic is '{}'", topic, TopicConfig.CLEANUP_POLICY_COMPACT); + admin.verifyTopicCleanupPolicyOnlyCompact(topic, + DistributedConfig.OFFSET_STORAGE_TOPIC_CONFIG, "source connector offsets"); } }; - return new KafkaBasedLog<>(topic, producerProps, consumerProps, consumedCallback, Time.SYSTEM, createTopics); + return new KafkaBasedLog<>(topic, producerProps, consumerProps, adminSupplier, consumedCallback, Time.SYSTEM, createTopics); } @Override diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaStatusBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaStatusBackingStore.java index 5d6057d799..efa405f3a4 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaStatusBackingStore.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaStatusBackingStore.java @@ -61,6 +61,7 @@ import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.function.Supplier; /** * StatusBackingStore implementation which uses a compacted topic for storage @@ -128,17 +129,24 @@ public class KafkaStatusBackingStore implements StatusBackingStore { protected final Table> tasks; protected final Map> connectors; protected final ConcurrentMap> topics; + private final Supplier topicAdminSupplier; private String statusTopic; private KafkaBasedLog kafkaLog; private int generation; + @Deprecated public KafkaStatusBackingStore(Time time, Converter converter) { + this(time, converter, null); + } + + public KafkaStatusBackingStore(Time time, Converter converter, Supplier topicAdminSupplier) { this.time = time; this.converter = converter; this.tasks = new Table<>(); this.connectors = new HashMap<>(); this.topics = new ConcurrentHashMap<>(); + this.topicAdminSupplier = topicAdminSupplier; } // visible for testing @@ -169,6 +177,7 @@ public void configure(final WorkerConfig config) { Map adminProps = new HashMap<>(originals); ConnectUtils.addMetricsContextProperties(adminProps, config, clusterId); + Supplier adminSupplier = topicAdminSupplier != null ? topicAdminSupplier : () -> new TopicAdmin(adminProps); Map topicSettings = config instanceof DistributedConfig ? ((DistributedConfig) config).statusStorageTopicSettings() @@ -180,36 +189,26 @@ public void configure(final WorkerConfig config) { .replicationFactor(config.getShort(DistributedConfig.STATUS_STORAGE_REPLICATION_FACTOR_CONFIG)) .build(); - Callback> readCallback = new Callback>() { - @Override - public void onCompletion(Throwable error, ConsumerRecord record) { - read(record); - } - }; - this.kafkaLog = createKafkaBasedLog(statusTopic, producerProps, consumerProps, readCallback, topicDescription, adminProps); + Callback> readCallback = (error, record) -> read(record); + this.kafkaLog = createKafkaBasedLog(statusTopic, producerProps, consumerProps, readCallback, topicDescription, adminSupplier); } private KafkaBasedLog createKafkaBasedLog(String topic, Map producerProps, Map consumerProps, Callback> consumedCallback, - final NewTopic topicDescription, final Map adminProps) { - Runnable createTopics = new Runnable() { - @Override - public void run() { - log.debug("Creating admin client to manage Connect internal status topic"); - try (TopicAdmin admin = new TopicAdmin(adminProps)) { - // Create the topic if it doesn't exist - Set newTopics = admin.createTopics(topicDescription); - if (!newTopics.contains(topic)) { - // It already existed, so check that the topic cleanup policy is compact only and not delete - log.debug("Using admin client to check cleanup policy of '{}' topic is '{}'", topic, TopicConfig.CLEANUP_POLICY_COMPACT); - admin.verifyTopicCleanupPolicyOnlyCompact(topic, - DistributedConfig.STATUS_STORAGE_TOPIC_CONFIG, "connector and task statuses"); - } - } + final NewTopic topicDescription, Supplier adminSupplier) { + java.util.function.Consumer createTopics = admin -> { + log.debug("Creating admin client to manage Connect internal status topic"); + // Create the topic if it doesn't exist + Set newTopics = admin.createTopics(topicDescription); + if (!newTopics.contains(topic)) { + // It already existed, so check that the topic cleanup policy is compact only and not delete + log.debug("Using admin client to check cleanup policy of '{}' topic is '{}'", topic, TopicConfig.CLEANUP_POLICY_COMPACT); + admin.verifyTopicCleanupPolicyOnlyCompact(topic, + DistributedConfig.STATUS_STORAGE_TOPIC_CONFIG, "connector and task statuses"); } }; - return new KafkaBasedLog<>(topic, producerProps, consumerProps, consumedCallback, time, createTopics); + return new KafkaBasedLog<>(topic, producerProps, consumerProps, adminSupplier, consumedCallback, time, createTopics); } @Override diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryConfigBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryConfigBackingStore.java index 90039b9c01..b89efa432f 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryConfigBackingStore.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryConfigBackingStore.java @@ -74,7 +74,7 @@ public synchronized ClusterConfigState snapshot() { connectorConfigs, connectorTargetStates, taskConfigs, - Collections.emptySet(), + Collections.emptySet(), configTransformer); } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryOffsetBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryOffsetBackingStore.java index ccec12a864..ceefd137d6 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryOffsetBackingStore.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryOffsetBackingStore.java @@ -27,7 +27,6 @@ import java.util.Collection; import java.util.HashMap; import java.util.Map; -import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; @@ -78,33 +77,26 @@ public void stop() { @Override public Future> get(final Collection keys) { - return executor.submit(new Callable>() { - @Override - public Map call() throws Exception { - Map result = new HashMap<>(); - for (ByteBuffer key : keys) { - result.put(key, data.get(key)); - } - return result; + return executor.submit(() -> { + Map result = new HashMap<>(); + for (ByteBuffer key : keys) { + result.put(key, data.get(key)); } + return result; }); - } @Override public Future set(final Map values, final Callback callback) { - return executor.submit(new Callable() { - @Override - public Void call() throws Exception { - for (Map.Entry entry : values.entrySet()) { - data.put(entry.getKey(), entry.getValue()); - } - save(); - if (callback != null) - callback.onCompletion(null, null); - return null; + return executor.submit(() -> { + for (Map.Entry entry : values.entrySet()) { + data.put(entry.getKey(), entry.getValue()); } + save(); + if (callback != null) + callback.onCompletion(null, null); + return null; }); } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetStorageWriter.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetStorageWriter.java index c360d4373b..f8ab6a7f1d 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetStorageWriter.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetStorageWriter.java @@ -166,13 +166,10 @@ public Future doFlush(final Callback callback) { log.debug("Submitting {} entries to backing store. The offsets are: {}", offsetsSerialized.size(), toFlush); } - return backingStore.set(offsetsSerialized, new Callback() { - @Override - public void onCompletion(Throwable error, Void result) { - boolean isCurrent = handleFinishWrite(flushId, error, result); - if (isCurrent && callback != null) { - callback.onCompletion(error, result); - } + return backingStore.set(offsetsSerialized, (error, result) -> { + boolean isCurrent = handleFinishWrite(flushId, error, result); + if (isCurrent && callback != null) { + callback.onCompletion(error, result); } }); } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/MockConnector.java b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/MockConnector.java index 82606b4c0e..c7d24d9872 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/MockConnector.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/MockConnector.java @@ -75,12 +75,9 @@ public void start(Map config) { log.debug("Started MockConnector with failure delay of {} ms", delayMs); executor = Executors.newSingleThreadScheduledExecutor(); - executor.schedule(new Runnable() { - @Override - public void run() { - log.debug("Triggering connector failure"); - context.raiseError(new RuntimeException()); - } + executor.schedule(() -> { + log.debug("Triggering connector failure"); + context.raiseError(new RuntimeException()); }, delayMs, TimeUnit.MILLISECONDS); } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java b/connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java index 5248715aa6..6a2a787578 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java @@ -41,10 +41,12 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Queue; import java.util.Set; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; /** @@ -79,13 +81,15 @@ public class KafkaBasedLog { private final Map producerConfigs; private final Map consumerConfigs; private final Callback> consumedCallback; + private final Supplier topicAdminSupplier; private Consumer consumer; private Producer producer; + private TopicAdmin admin; private Thread thread; private boolean stopRequested; private Queue> readLogEndOffsetCallbacks; - private Runnable initializer; + private java.util.function.Consumer initializer; /** * Create a new KafkaBasedLog object. This does not start reading the log and writing is not permitted until @@ -103,31 +107,63 @@ public class KafkaBasedLog { * @param consumedCallback callback to invoke for each {@link ConsumerRecord} consumed when tailing the log * @param time Time interface * @param initializer the component that should be run when this log is {@link #start() started}; may be null + * @deprecated Replaced by {@link #KafkaBasedLog(String, Map, Map, Supplier, Callback, Time, java.util.function.Consumer)} */ + @Deprecated public KafkaBasedLog(String topic, Map producerConfigs, Map consumerConfigs, Callback> consumedCallback, Time time, Runnable initializer) { + this(topic, producerConfigs, consumerConfigs, () -> null, consumedCallback, time, initializer != null ? admin -> initializer.run() : null); + } + + /** + * Create a new KafkaBasedLog object. This does not start reading the log and writing is not permitted until + * {@link #start()} is invoked. + * + * @param topic the topic to treat as a log + * @param producerConfigs configuration options to use when creating the internal producer. At a minimum this must + * contain compatible serializer settings for the generic types used on this class. Some + * setting, such as the number of acks, will be overridden to ensure correct behavior of this + * class. + * @param consumerConfigs configuration options to use when creating the internal consumer. At a minimum this must + * contain compatible serializer settings for the generic types used on this class. Some + * setting, such as the auto offset reset policy, will be overridden to ensure correct + * behavior of this class. + * @param topicAdminSupplier supplier function for an admin client, the lifecycle of which is expected to be controlled + * by the calling component; may not be null + * @param consumedCallback callback to invoke for each {@link ConsumerRecord} consumed when tailing the log + * @param time Time interface + * @param initializer the function that should be run when this log is {@link #start() started}; may be null + */ + public KafkaBasedLog(String topic, + Map producerConfigs, + Map consumerConfigs, + Supplier topicAdminSupplier, + Callback> consumedCallback, + Time time, + java.util.function.Consumer initializer) { this.topic = topic; this.producerConfigs = producerConfigs; this.consumerConfigs = consumerConfigs; + this.topicAdminSupplier = Objects.requireNonNull(topicAdminSupplier); this.consumedCallback = consumedCallback; this.stopRequested = false; this.readLogEndOffsetCallbacks = new ArrayDeque<>(); this.time = time; - this.initializer = initializer != null ? initializer : new Runnable() { - @Override - public void run() { - } - }; + this.initializer = initializer != null ? initializer : admin -> { }; } public void start() { log.info("Starting KafkaBasedLog with topic " + topic); - initializer.run(); + // Create the topic admin client and initialize the topic ... + admin = topicAdminSupplier.get(); // may be null + initializer.accept(admin); + + // Then create the producer and consumer producer = createProducer(); consumer = createConsumer(); @@ -193,6 +229,9 @@ public void stop() { log.error("Failed to stop KafkaBasedLog consumer", e); } + // do not close the admin client, since we don't own it + admin = null; + log.info("Stopped KafkaBasedLog for topic " + topic); } @@ -282,7 +321,29 @@ private void readToLogEnd() { log.trace("Reading to end of offset log"); Set assignment = consumer.assignment(); - Map endOffsets = consumer.endOffsets(assignment); + Map endOffsets; + // Note that we'd prefer to not use the consumer to find the end offsets for the assigned topic partitions. + // That is because it's possible that the consumer is already blocked waiting for new records to appear, when + // the consumer is already at the end. In such cases, using 'consumer.endOffsets(...)' will block until at least + // one more record becomes available, meaning we can't even check whether we're at the end offset. + // Since all we're trying to do here is get the end offset, we should use the supplied admin client + // (if available) + // (which prevents 'consumer.endOffsets(...)' + // from + + // Deprecated constructors do not provide an admin supplier, so the admin is potentially null. + if (admin != null) { + // Use the admin client to immediately find the end offsets for the assigned topic partitions. + // Unlike using the consumer + endOffsets = admin.endOffsets(assignment); + } else { + // The admin may be null if older deprecated constructor is used, though AK Connect currently always provides an admin client. + // Using the consumer is not ideal, because when the topic has low volume, the 'poll(...)' method called from the + // work thread may have blocked the consumer while waiting for more records (even when there are none). + // In such cases, this call to the consumer to simply find the end offsets will block even though we might already be + // at the end offset. + endOffsets = consumer.endOffsets(assignment); + } log.trace("Reading to end of log offsets {}", endOffsets); while (!endOffsets.isEmpty()) { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/util/SharedTopicAdmin.java b/connect/runtime/src/main/java/org/apache/kafka/connect/util/SharedTopicAdmin.java new file mode 100644 index 0000000000..a99514e2ed --- /dev/null +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/util/SharedTopicAdmin.java @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.util; + +import java.time.Duration; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; +import java.util.function.Supplier; +import java.util.function.UnaryOperator; + +import org.apache.kafka.clients.admin.AdminClientConfig; +import org.apache.kafka.connect.errors.ConnectException; + +/** + * A holder of a {@link TopicAdmin} object that is lazily and atomically created when needed by multiple callers. + * As soon as one of the getters is called, all getters will return the same shared {@link TopicAdmin} + * instance until this SharedAdmin is closed via {@link #close()} or {@link #close(Duration)}. + * + *

The owner of this object is responsible for ensuring that either {@link #close()} or {@link #close(Duration)} + * is called when the {@link TopicAdmin} instance is no longer needed. Consequently, once this + * {@link SharedTopicAdmin} instance has been closed, the {@link #get()} and {@link #topicAdmin()} methods, + * nor any previously returned {@link TopicAdmin} instances may be used. + * + *

This class is thread-safe. It also appears as immutable to callers that obtain the {@link TopicAdmin} object, + * until this object is closed, at which point it cannot be used anymore + */ +public class SharedTopicAdmin implements AutoCloseable, Supplier { + + // Visible for testing + static final Duration DEFAULT_CLOSE_DURATION = Duration.ofMillis(Long.MAX_VALUE); + + private final Map adminProps; + private final AtomicReference admin = new AtomicReference<>(); + private final AtomicBoolean closed = new AtomicBoolean(false); + private final Function, TopicAdmin> factory; + + public SharedTopicAdmin(Map adminProps) { + this(adminProps, TopicAdmin::new); + } + + // Visible for testing + SharedTopicAdmin(Map adminProps, Function, TopicAdmin> factory) { + this.adminProps = Objects.requireNonNull(adminProps); + this.factory = Objects.requireNonNull(factory); + } + + /** + * Get the shared {@link TopicAdmin} instance. + * + * @return the shared instance; never null + * @throws ConnectException if this object has already been closed + */ + @Override + public TopicAdmin get() { + return topicAdmin(); + } + + /** + * Get the shared {@link TopicAdmin} instance. + * + * @return the shared instance; never null + * @throws ConnectException if this object has already been closed + */ + public TopicAdmin topicAdmin() { + return admin.updateAndGet(this::createAdmin); + } + + /** + * Get the string containing the list of bootstrap server addresses to the Kafka broker(s) to which + * the admin client connects. + * + * @return the bootstrap servers as a string; never null + */ + public String bootstrapServers() { + return adminProps.getOrDefault(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, "").toString(); + } + + /** + * Close the underlying {@link TopicAdmin} instance, if one has been created, and prevent new ones from being created. + * + *

Once this method is called, the {@link #get()} and {@link #topicAdmin()} methods, + * nor any previously returned {@link TopicAdmin} instances may be used. + */ + @Override + public void close() { + close(DEFAULT_CLOSE_DURATION); + } + + /** + * Close the underlying {@link TopicAdmin} instance, if one has been created, and prevent new ones from being created. + * + *

Once this method is called, the {@link #get()} and {@link #topicAdmin()} methods, + * nor any previously returned {@link TopicAdmin} instances may be used. + * + * @param timeout the maximum time to wait while the underlying admin client is closed; may not be null + */ + public void close(Duration timeout) { + Objects.requireNonNull(timeout); + if (this.closed.compareAndSet(false, true)) { + TopicAdmin admin = this.admin.getAndSet(null); + if (admin != null) { + admin.close(timeout); + } + } + } + + @Override + public String toString() { + return "admin client for brokers at " + bootstrapServers(); + } + + /** + * Method used to create a {@link TopicAdmin} instance. This method must be side-effect free, since it is called from within + * the {@link AtomicReference#updateAndGet(UnaryOperator)}. + * + * @param existing the existing instance; may be null + * @return the + */ + protected TopicAdmin createAdmin(TopicAdmin existing) { + if (closed.get()) { + throw new ConnectException("The " + this + " has already been closed and cannot be used."); + } + if (existing != null) { + return existing; + } + return factory.apply(adminProps); + } +} diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/util/TopicAdmin.java b/connect/runtime/src/main/java/org/apache/kafka/connect/util/TopicAdmin.java index 428343b099..9a7907bcda 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/util/TopicAdmin.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/util/TopicAdmin.java @@ -23,14 +23,20 @@ import org.apache.kafka.clients.admin.CreateTopicsOptions; import org.apache.kafka.clients.admin.DescribeConfigsOptions; import org.apache.kafka.clients.admin.DescribeTopicsOptions; +import org.apache.kafka.clients.admin.ListOffsetsResult; +import org.apache.kafka.clients.admin.ListOffsetsResult.ListOffsetsResultInfo; import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.admin.OffsetSpec; import org.apache.kafka.clients.admin.TopicDescription; import org.apache.kafka.common.KafkaFuture; +import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.config.TopicConfig; +import org.apache.kafka.common.errors.AuthorizationException; import org.apache.kafka.common.errors.ClusterAuthorizationException; import org.apache.kafka.common.errors.InvalidConfigurationException; +import org.apache.kafka.common.errors.LeaderNotAvailableException; import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.TopicAuthorizationException; import org.apache.kafka.common.errors.TopicExistsException; @@ -53,6 +59,7 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.ExecutionException; +import java.util.function.Function; import java.util.stream.Collectors; /** @@ -278,11 +285,19 @@ public TopicAdmin(Map adminConfig) { // visible for testing TopicAdmin(Map adminConfig, Admin adminClient, boolean logCreation) { this.admin = adminClient; - this.adminConfig = adminConfig != null ? adminConfig : Collections.emptyMap(); + this.adminConfig = adminConfig != null ? adminConfig : Collections.emptyMap(); this.logCreation = logCreation; } /** + * Get the {@link Admin} client used by this topic admin object. + * @return the Kafka admin instance; never null + */ + public Admin admin() { + return admin; + } + + /** * Attempt to create the topic described by the given definition, returning true if the topic was created or false * if the topic already existed. * @@ -630,6 +645,58 @@ public Map describeTopicConfigs(String... topicNames) { return result; } + /** + * Fetch the most recent offset for each of the supplied {@link TopicPartition} objects. + * + * @param partitions the topic partitions + * @return the map of offset for each topic partition, or an empty map if the supplied partitions + * are null or empty + * @throws RetriableException if a retriable error occurs, the operation takes too long, or the + * thread is interrupted while attempting to perform this operation + * @throws ConnectException if a non retriable error occurs + */ + public Map endOffsets(Set partitions) { + if (partitions == null || partitions.isEmpty()) { + return Collections.emptyMap(); + } + Map offsetSpecMap = partitions.stream().collect(Collectors.toMap(Function.identity(), tp -> OffsetSpec.latest())); + ListOffsetsResult resultFuture = admin.listOffsets(offsetSpecMap); + // Get the individual result for each topic partition so we have better error messages + Map result = new HashMap<>(); + for (TopicPartition partition : partitions) { + try { + ListOffsetsResultInfo info = resultFuture.partitionResult(partition).get(); + result.put(partition, info.offset()); + } catch (ExecutionException e) { + Throwable cause = e.getCause(); + String topic = partition.topic(); + if (cause instanceof AuthorizationException) { + String msg = String.format("Not authorized to get the end offsets for topic '%s' on brokers at %s", topic, bootstrapServers()); + throw new ConnectException(msg, e); + } else if (cause instanceof UnsupportedVersionException) { + // Should theoretically never happen, because this method is the same as what the consumer uses and therefore + // should exist in the broker since before the admin client was added + String msg = String.format("API to get the get the end offsets for topic '%s' is unsupported on brokers at %s", topic, bootstrapServers()); + throw new ConnectException(msg, e); + } else if (cause instanceof TimeoutException) { + String msg = String.format("Timed out while waiting to get end offsets for topic '%s' on brokers at %s", topic, bootstrapServers()); + throw new RetriableException(msg, e); + } else if (cause instanceof LeaderNotAvailableException) { + String msg = String.format("Unable to get end offsets during leader election for topic '%s' on brokers at %s", topic, bootstrapServers()); + throw new RetriableException(msg, e); + } else { + String msg = String.format("Error while getting end offsets for topic '%s' on brokers at %s", topic, bootstrapServers()); + throw new ConnectException(msg, e); + } + } catch (InterruptedException e) { + Thread.interrupted(); + String msg = String.format("Interrupted while attempting to read end offsets for topic '%s' on brokers at %s", partition.topic(), bootstrapServers()); + throw new RetriableException(msg, e); + } + } + return result; + } + @Override public void close() { admin.close(); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/TransformationIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/TransformationIntegrationTest.java index 379172ad16..7b71f2f5f7 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/TransformationIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/TransformationIntegrationTest.java @@ -173,7 +173,7 @@ public void testFilterOnTopicNameWithSinkConnector() throws Exception { connectorHandle.awaitCommits(RECORD_TRANSFER_DURATION_MS); // Assert that we didn't see any baz - Map expectedRecordCounts = singletonMap(fooTopic, Long.valueOf(numFooRecords)); + Map expectedRecordCounts = singletonMap(fooTopic, (long) numFooRecords); assertObservedRecords(observedRecords, expectedRecordCounts); // delete connector @@ -258,7 +258,7 @@ public void testFilterOnTombstonesWithSinkConnector() throws Exception { // wait for the connector tasks to commit all records. connectorHandle.awaitCommits(RECORD_TRANSFER_DURATION_MS); - Map expectedRecordCounts = singletonMap(topic, Long.valueOf(numRecords / 2)); + Map expectedRecordCounts = singletonMap(topic, (long) (numRecords / 2)); assertObservedRecords(observedRecords, expectedRecordCounts); // delete connector diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java index d2b7913731..5d33f28504 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java @@ -48,7 +48,6 @@ import org.apache.kafka.connect.util.ConnectorTaskId; import org.easymock.Capture; import org.easymock.EasyMock; -import org.easymock.IAnswer; import org.junit.Test; import org.junit.runner.RunWith; import org.powermock.api.easymock.PowerMock; @@ -125,10 +124,10 @@ public class AbstractHerderTest { } private static final ClusterConfigState SNAPSHOT = new ClusterConfigState(1, null, Collections.singletonMap(CONN1, 3), Collections.singletonMap(CONN1, CONN1_CONFIG), Collections.singletonMap(CONN1, TargetState.STARTED), - TASK_CONFIGS_MAP, Collections.emptySet()); + TASK_CONFIGS_MAP, Collections.emptySet()); private static final ClusterConfigState SNAPSHOT_NO_TASKS = new ClusterConfigState(1, null, Collections.singletonMap(CONN1, 3), Collections.singletonMap(CONN1, CONN1_CONFIG), Collections.singletonMap(CONN1, TargetState.STARTED), - Collections.emptyMap(), Collections.emptySet()); + Collections.emptyMap(), Collections.emptySet()); private final String workerId = "workerId"; private final String kafkaClusterId = "I4ZmrWqfT2e-upky_4fdPA"; @@ -253,12 +252,7 @@ public void taskStatus() { statusStore.putSafe(EasyMock.capture(statusCapture)); EasyMock.expectLastCall(); - EasyMock.expect(statusStore.get(taskId)).andAnswer(new IAnswer() { - @Override - public TaskStatus answer() throws Throwable { - return statusCapture.getValue(); - } - }); + EasyMock.expect(statusStore.get(taskId)).andAnswer(statusCapture::getValue); replayAll(); @@ -364,7 +358,7 @@ public void testConfigValidationTransformsExtendResults() throws Throwable { // 2 transform aliases defined -> 2 plugin lookups Set> transformations = new HashSet<>(); - transformations.add(new PluginDesc(SampleTransformation.class, "1.0", classLoader)); + transformations.add(new PluginDesc<>(SampleTransformation.class, "1.0", classLoader)); EasyMock.expect(plugins.transformations()).andReturn(transformations).times(2); replayAll(); @@ -416,11 +410,11 @@ public void testConfigValidationPredicatesExtendResults() { // 2 transform aliases defined -> 2 plugin lookups Set> transformations = new HashSet<>(); - transformations.add(new PluginDesc(SampleTransformation.class, "1.0", classLoader)); + transformations.add(new PluginDesc<>(SampleTransformation.class, "1.0", classLoader)); EasyMock.expect(plugins.transformations()).andReturn(transformations).times(1); Set> predicates = new HashSet<>(); - predicates.add(new PluginDesc(SamplePredicate.class, "1.0", classLoader)); + predicates.add(new PluginDesc<>(SamplePredicate.class, "1.0", classLoader)); EasyMock.expect(plugins.predicates()).andReturn(predicates).times(2); replayAll(); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ConnectMetricsTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ConnectMetricsTest.java index 600ac80857..31aca97e20 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ConnectMetricsTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ConnectMetricsTest.java @@ -165,6 +165,6 @@ private Sensor addToGroup(ConnectMetrics connectMetrics, boolean shouldClose) { } static MetricName metricName(String name) { - return new MetricName(name, "test_group", "metrics for testing", Collections.emptyMap()); + return new MetricName(name, "test_group", "metrics for testing", Collections.emptyMap()); } } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ConnectorConfigTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ConnectorConfigTest.java index a74db236e0..b7b7ee7772 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ConnectorConfigTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ConnectorConfigTest.java @@ -41,7 +41,7 @@ public class ConnectorConfigTest> { - public static final Plugins MOCK_PLUGINS = new Plugins(new HashMap()) { + public static final Plugins MOCK_PLUGINS = new Plugins(new HashMap<>()) { @Override public Set> transformations() { return Collections.emptySet(); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/MockConnectMetrics.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/MockConnectMetrics.java index 38bd4017fc..fa49358baa 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/MockConnectMetrics.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/MockConnectMetrics.java @@ -89,7 +89,7 @@ public Object currentMetricValue(MetricGroup metricGroup, String name) { */ public double currentMetricValueAsDouble(MetricGroup metricGroup, String name) { Object value = currentMetricValue(metricGroup, name); - return value instanceof Double ? ((Double) value).doubleValue() : Double.NaN; + return value instanceof Double ? (Double) value : Double.NaN; } /** @@ -135,7 +135,7 @@ public static Object currentMetricValue(ConnectMetrics metrics, MetricGroup metr */ public static double currentMetricValueAsDouble(ConnectMetrics metrics, MetricGroup metricGroup, String name) { Object value = currentMetricValue(metrics, metricGroup, name); - return value instanceof Double ? ((Double) value).doubleValue() : Double.NaN; + return value instanceof Double ? (Double) value : Double.NaN; } /** diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java index 9442d103ca..a7d68a6152 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java @@ -55,7 +55,6 @@ import org.easymock.Capture; import org.easymock.CaptureType; import org.easymock.EasyMock; -import org.easymock.IAnswer; import org.easymock.IExpectationSetters; import org.junit.After; import org.junit.Before; @@ -230,7 +229,7 @@ public void testPause() throws Exception { expectConsumerPoll(1); expectConversionAndTransformation(1); - sinkTask.put(EasyMock.>anyObject()); + sinkTask.put(EasyMock.anyObject()); EasyMock.expectLastCall(); Set partitions = new HashSet<>(asList(TOPIC_PARTITION, TOPIC_PARTITION2)); @@ -244,10 +243,10 @@ public void testPause() throws Exception { PowerMock.expectLastCall(); // Offset commit as requested when pausing; No records returned by consumer.poll() - sinkTask.preCommit(EasyMock.>anyObject()); + sinkTask.preCommit(EasyMock.anyObject()); EasyMock.expectLastCall().andStubReturn(Collections.emptyMap()); expectConsumerPoll(0); - sinkTask.put(Collections.emptyList()); + sinkTask.put(Collections.emptyList()); EasyMock.expectLastCall(); // And unpause @@ -262,7 +261,7 @@ public void testPause() throws Exception { expectConsumerPoll(1); expectConversionAndTransformation(1); - sinkTask.put(EasyMock.>anyObject()); + sinkTask.put(EasyMock.anyObject()); EasyMock.expectLastCall(); PowerMock.replayAll(); @@ -329,7 +328,7 @@ public void testShutdown() throws Exception { EasyMock.expect(sinkTask.preCommit(EasyMock.anyObject())).andReturn(Collections.emptyMap()); expectConsumerPoll(1); expectConversionAndTransformation(1); - sinkTask.put(EasyMock.>anyObject()); + sinkTask.put(EasyMock.anyObject()); EasyMock.expectLastCall(); // WorkerSinkTask::stop @@ -340,14 +339,11 @@ public void testShutdown() throws Exception { // WorkerSinkTask::close consumer.close(); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Object answer() throws Throwable { - rebalanceListener.getValue().onPartitionsRevoked( - asList(TOPIC_PARTITION, TOPIC_PARTITION2) - ); - return null; - } + PowerMock.expectLastCall().andAnswer(() -> { + rebalanceListener.getValue().onPartitionsRevoked( + asList(TOPIC_PARTITION, TOPIC_PARTITION2) + ); + return null; }); transformationChain.close(); PowerMock.expectLastCall(); @@ -502,7 +498,7 @@ public void testWakeupInCommitSyncCausesRetry() throws Exception { expectConsumerPoll(1); expectConversionAndTransformation(1); - sinkTask.put(EasyMock.>anyObject()); + sinkTask.put(EasyMock.anyObject()); EasyMock.expectLastCall(); final List partitions = asList(TOPIC_PARTITION, TOPIC_PARTITION2); @@ -531,14 +527,11 @@ public void testWakeupInCommitSyncCausesRetry() throws Exception { EasyMock.expectLastCall(); EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer( - new IAnswer>() { - @Override - public ConsumerRecords answer() throws Throwable { - rebalanceListener.getValue().onPartitionsRevoked(partitions); - rebalanceListener.getValue().onPartitionsAssigned(partitions); - return ConsumerRecords.empty(); - } - }); + () -> { + rebalanceListener.getValue().onPartitionsRevoked(partitions); + rebalanceListener.getValue().onPartitionsAssigned(partitions); + return ConsumerRecords.empty(); + }); EasyMock.expect(consumer.assignment()).andReturn(new HashSet<>(partitions)); @@ -596,16 +589,13 @@ public void testWakeupNotThrownDuringShutdown() throws Exception { expectConsumerPoll(1); expectConversionAndTransformation(1); - sinkTask.put(EasyMock.>anyObject()); + sinkTask.put(EasyMock.anyObject()); EasyMock.expectLastCall(); - EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer(new IAnswer>() { - @Override - public ConsumerRecords answer() throws Throwable { - // stop the task during its second iteration - workerTask.stop(); - return new ConsumerRecords<>(Collections.emptyMap()); - } + EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer(() -> { + // stop the task during its second iteration + workerTask.stop(); + return new ConsumerRecords<>(Collections.emptyMap()); }); consumer.wakeup(); EasyMock.expectLastCall(); @@ -650,7 +640,7 @@ public void testRequestCommit() throws Exception { expectConsumerPoll(1); expectConversionAndTransformation(1); - sinkTask.put(EasyMock.>anyObject()); + sinkTask.put(EasyMock.anyObject()); EasyMock.expectLastCall(); final Map offsets = new HashMap<>(); @@ -661,16 +651,13 @@ public void testRequestCommit() throws Exception { final Capture callback = EasyMock.newCapture(); consumer.commitAsync(EasyMock.eq(offsets), EasyMock.capture(callback)); - EasyMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Void answer() throws Throwable { - callback.getValue().onComplete(offsets, null); - return null; - } + EasyMock.expectLastCall().andAnswer(() -> { + callback.getValue().onComplete(offsets, null); + return null; }); expectConsumerPoll(0); - sinkTask.put(Collections.emptyList()); + sinkTask.put(Collections.emptyList()); EasyMock.expectLastCall(); PowerMock.replayAll(); @@ -759,7 +746,7 @@ public void testPreCommit() throws Exception { // iter 2 expectConsumerPoll(2); expectConversionAndTransformation(2); - sinkTask.put(EasyMock.>anyObject()); + sinkTask.put(EasyMock.anyObject()); EasyMock.expectLastCall(); final Map workerStartingOffsets = new HashMap<>(); @@ -785,15 +772,12 @@ public void testPreCommit() throws Exception { EasyMock.expect(consumer.assignment()).andReturn(workerCurrentOffsets.keySet()); final Capture callback = EasyMock.newCapture(); consumer.commitAsync(EasyMock.eq(committableOffsets), EasyMock.capture(callback)); - EasyMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Void answer() throws Throwable { - callback.getValue().onComplete(committableOffsets, null); - return null; - } + EasyMock.expectLastCall().andAnswer(() -> { + callback.getValue().onComplete(committableOffsets, null); + return null; }); expectConsumerPoll(0); - sinkTask.put(EasyMock.>anyObject()); + sinkTask.put(EasyMock.anyObject()); EasyMock.expectLastCall(); PowerMock.replayAll(); @@ -827,7 +811,7 @@ public void testIgnoredCommit() throws Exception { // iter 2 expectConsumerPoll(1); expectConversionAndTransformation(1); - sinkTask.put(EasyMock.>anyObject()); + sinkTask.put(EasyMock.anyObject()); EasyMock.expectLastCall(); final Map workerStartingOffsets = new HashMap<>(); @@ -843,7 +827,7 @@ public void testIgnoredCommit() throws Exception { EasyMock.expectLastCall().andReturn(workerStartingOffsets); // no actual consumer.commit() triggered expectConsumerPoll(0); - sinkTask.put(EasyMock.>anyObject()); + sinkTask.put(EasyMock.anyObject()); EasyMock.expectLastCall(); PowerMock.replayAll(); @@ -878,7 +862,7 @@ public void testLongRunningCommitWithoutTimeout() throws Exception { // iter 2 expectConsumerPoll(1); expectConversionAndTransformation(1); - sinkTask.put(EasyMock.>anyObject()); + sinkTask.put(EasyMock.anyObject()); EasyMock.expectLastCall(); final Map workerStartingOffsets = new HashMap<>(); @@ -900,37 +884,31 @@ public void testLongRunningCommitWithoutTimeout() throws Exception { final ExecutorService executor = Executors.newSingleThreadExecutor(); final CountDownLatch latch = new CountDownLatch(1); - consumer.commitAsync(EasyMock.eq(workerCurrentOffsets), EasyMock.anyObject()); - EasyMock.expectLastCall().andAnswer(new IAnswer() { + consumer.commitAsync(EasyMock.eq(workerCurrentOffsets), EasyMock.anyObject()); + EasyMock.expectLastCall().andAnswer(() -> { + // Grab the arguments passed to the consumer.commitAsync method + final Object[] args = EasyMock.getCurrentArguments(); @SuppressWarnings("unchecked") - @Override - public Void answer() throws Throwable { - // Grab the arguments passed to the consumer.commitAsync method - final Object[] args = EasyMock.getCurrentArguments(); - final Map offsets = (Map) args[0]; - final OffsetCommitCallback callback = (OffsetCommitCallback) args[1]; - - executor.execute(new Runnable() { - @Override - public void run() { - try { - latch.await(); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - } - - callback.onComplete(offsets, null); - } - }); - - return null; - } + final Map offsets = (Map) args[0]; + final OffsetCommitCallback callback = (OffsetCommitCallback) args[1]; + + executor.execute(() -> { + try { + latch.await(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + + callback.onComplete(offsets, null); + }); + + return null; }); // no actual consumer.commit() triggered expectConsumerPoll(0); - sinkTask.put(EasyMock.>anyObject()); + sinkTask.put(EasyMock.anyObject()); EasyMock.expectLastCall(); PowerMock.replayAll(); @@ -1062,7 +1040,7 @@ public void testCommitWithOutOfOrderCallback() throws Exception { // iter 2 expectConsumerPoll(1); expectConversionAndTransformation(4); - sinkTask.put(EasyMock.>anyObject()); + sinkTask.put(EasyMock.anyObject()); EasyMock.expectLastCall(); final Map workerStartingOffsets = new HashMap<>(); @@ -1096,63 +1074,54 @@ public void testCommitWithOutOfOrderCallback() throws Exception { final AtomicReference asyncCallbackRunner = new AtomicReference<>(); final AtomicBoolean asyncCallbackRan = new AtomicBoolean(); - consumer.commitAsync(EasyMock.eq(workerCurrentOffsets), EasyMock.anyObject()); - EasyMock.expectLastCall().andAnswer(new IAnswer() { + consumer.commitAsync(EasyMock.eq(workerCurrentOffsets), EasyMock.anyObject()); + EasyMock.expectLastCall().andAnswer(() -> { + // Grab the arguments passed to the consumer.commitAsync method + final Object[] args = EasyMock.getCurrentArguments(); @SuppressWarnings("unchecked") - @Override - public Void answer() throws Throwable { - // Grab the arguments passed to the consumer.commitAsync method - final Object[] args = EasyMock.getCurrentArguments(); - final Map offsets = (Map) args[0]; - final OffsetCommitCallback callback = (OffsetCommitCallback) args[1]; - asyncCallbackRunner.set(new Runnable() { - @Override - public void run() { - callback.onComplete(offsets, null); - asyncCallbackRan.set(true); - } - }); - return null; - } + final Map offsets = (Map) args[0]; + final OffsetCommitCallback callback = (OffsetCommitCallback) args[1]; + asyncCallbackRunner.set(() -> { + callback.onComplete(offsets, null); + asyncCallbackRan.set(true); + }); + return null; }); // Expect the next poll to discover and perform the rebalance, THEN complete the previous callback handler, // and then return one record for TP1 and one for TP3. final AtomicBoolean rebalanced = new AtomicBoolean(); EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer( - new IAnswer>() { - @Override - public ConsumerRecords answer() throws Throwable { - // Rebalance always begins with revoking current partitions ... - rebalanceListener.getValue().onPartitionsRevoked(originalPartitions); - // Respond to the rebalance - Map offsets = new HashMap<>(); - offsets.put(TOPIC_PARTITION, rebalanceOffsets.get(TOPIC_PARTITION).offset()); - offsets.put(TOPIC_PARTITION2, rebalanceOffsets.get(TOPIC_PARTITION2).offset()); - offsets.put(TOPIC_PARTITION3, rebalanceOffsets.get(TOPIC_PARTITION3).offset()); - sinkTaskContext.getValue().offset(offsets); - rebalanceListener.getValue().onPartitionsAssigned(rebalancedPartitions); - rebalanced.set(true); - - // Run the previous async commit handler - asyncCallbackRunner.get().run(); - - // And prep the two records to return - long timestamp = RecordBatch.NO_TIMESTAMP; - TimestampType timestampType = TimestampType.NO_TIMESTAMP_TYPE; - List> records = new ArrayList<>(); - records.add(new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturnedTp1 + 1, timestamp, timestampType, 0L, 0, 0, RAW_KEY, RAW_VALUE)); - records.add(new ConsumerRecord<>(TOPIC, PARTITION3, FIRST_OFFSET + recordsReturnedTp3 + 1, timestamp, timestampType, 0L, 0, 0, RAW_KEY, RAW_VALUE)); - recordsReturnedTp1 += 1; - recordsReturnedTp3 += 1; - return new ConsumerRecords<>(Collections.singletonMap(new TopicPartition(TOPIC, PARTITION), records)); - } - }); + () -> { + // Rebalance always begins with revoking current partitions ... + rebalanceListener.getValue().onPartitionsRevoked(originalPartitions); + // Respond to the rebalance + Map offsets = new HashMap<>(); + offsets.put(TOPIC_PARTITION, rebalanceOffsets.get(TOPIC_PARTITION).offset()); + offsets.put(TOPIC_PARTITION2, rebalanceOffsets.get(TOPIC_PARTITION2).offset()); + offsets.put(TOPIC_PARTITION3, rebalanceOffsets.get(TOPIC_PARTITION3).offset()); + sinkTaskContext.getValue().offset(offsets); + rebalanceListener.getValue().onPartitionsAssigned(rebalancedPartitions); + rebalanced.set(true); + + // Run the previous async commit handler + asyncCallbackRunner.get().run(); + + // And prep the two records to return + long timestamp = RecordBatch.NO_TIMESTAMP; + TimestampType timestampType = TimestampType.NO_TIMESTAMP_TYPE; + List> records = new ArrayList<>(); + records.add(new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturnedTp1 + 1, timestamp, timestampType, 0L, 0, 0, RAW_KEY, RAW_VALUE)); + records.add(new ConsumerRecord<>(TOPIC, PARTITION3, FIRST_OFFSET + recordsReturnedTp3 + 1, timestamp, timestampType, 0L, 0, 0, RAW_KEY, RAW_VALUE)); + recordsReturnedTp1 += 1; + recordsReturnedTp3 += 1; + return new ConsumerRecords<>(Collections.singletonMap(new TopicPartition(TOPIC, PARTITION), records)); + }); // onPartitionsRevoked sinkTask.preCommit(workerCurrentOffsets); EasyMock.expectLastCall().andReturn(workerCurrentOffsets); - sinkTask.put(EasyMock.>anyObject()); + sinkTask.put(EasyMock.anyObject()); EasyMock.expectLastCall(); sinkTask.close(workerCurrentOffsets.keySet()); EasyMock.expectLastCall(); @@ -1185,18 +1154,15 @@ public ConsumerRecords answer() throws Throwable { final Capture callback = EasyMock.newCapture(); consumer.commitAsync(EasyMock.eq(postRebalanceCurrentOffsets), EasyMock.capture(callback)); - EasyMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Void answer() throws Throwable { - callback.getValue().onComplete(postRebalanceCurrentOffsets, null); - return null; - } + EasyMock.expectLastCall().andAnswer(() -> { + callback.getValue().onComplete(postRebalanceCurrentOffsets, null); + return null; }); // no actual consumer.commit() triggered expectConsumerPoll(1); - sinkTask.put(EasyMock.>anyObject()); + sinkTask.put(EasyMock.anyObject()); EasyMock.expectLastCall(); PowerMock.replayAll(); @@ -1281,7 +1247,7 @@ public void testDeliveryWithMutatingTransform() throws Exception { expectConsumerPoll(1); expectConversionAndTransformation(1, "newtopic_"); - sinkTask.put(EasyMock.>anyObject()); + sinkTask.put(EasyMock.anyObject()); EasyMock.expectLastCall(); final Map offsets = new HashMap<>(); @@ -1292,16 +1258,13 @@ public void testDeliveryWithMutatingTransform() throws Exception { final Capture callback = EasyMock.newCapture(); consumer.commitAsync(EasyMock.eq(offsets), EasyMock.capture(callback)); - EasyMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Void answer() throws Throwable { - callback.getValue().onComplete(offsets, null); - return null; - } + EasyMock.expectLastCall().andAnswer(() -> { + callback.getValue().onComplete(offsets, null); + return null; }); expectConsumerPoll(0); - sinkTask.put(Collections.emptyList()); + sinkTask.put(Collections.emptyList()); EasyMock.expectLastCall(); PowerMock.replayAll(); @@ -1494,7 +1457,7 @@ public void testHeaders() throws Exception { expectConsumerPoll(1, headers); expectConversionAndTransformation(1, null, headers); - sinkTask.put(EasyMock.>anyObject()); + sinkTask.put(EasyMock.anyObject()); EasyMock.expectLastCall(); PowerMock.replayAll(); @@ -1553,11 +1516,11 @@ public void testHeadersWithCustomConverter() throws Exception { SinkRecord recordA = iterator.next(); assertEquals(keyA, recordA.key()); - assertEquals(valueA, (String) recordA.value()); + assertEquals(valueA, recordA.value()); SinkRecord recordB = iterator.next(); assertEquals(keyB, recordB.key()); - assertEquals(valueB, (String) recordB.value()); + assertEquals(valueB, recordB.value()); PowerMock.verifyAll(); } @@ -1578,17 +1541,14 @@ private void expectRebalanceRevocationError(RuntimeException e) { sinkTask.close(new HashSet<>(partitions)); EasyMock.expectLastCall().andThrow(e); - sinkTask.preCommit(EasyMock.>anyObject()); + sinkTask.preCommit(EasyMock.anyObject()); EasyMock.expectLastCall().andReturn(Collections.emptyMap()); EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer( - new IAnswer>() { - @Override - public ConsumerRecords answer() throws Throwable { - rebalanceListener.getValue().onPartitionsRevoked(partitions); - return ConsumerRecords.empty(); - } - }); + () -> { + rebalanceListener.getValue().onPartitionsRevoked(partitions); + return ConsumerRecords.empty(); + }); } private void expectRebalanceAssignmentError(RuntimeException e) { @@ -1597,7 +1557,7 @@ private void expectRebalanceAssignmentError(RuntimeException e) { sinkTask.close(new HashSet<>(partitions)); EasyMock.expectLastCall(); - sinkTask.preCommit(EasyMock.>anyObject()); + sinkTask.preCommit(EasyMock.anyObject()); EasyMock.expectLastCall().andReturn(Collections.emptyMap()); EasyMock.expect(consumer.position(TOPIC_PARTITION)).andReturn(FIRST_OFFSET); @@ -1607,14 +1567,11 @@ private void expectRebalanceAssignmentError(RuntimeException e) { EasyMock.expectLastCall().andThrow(e); EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer( - new IAnswer>() { - @Override - public ConsumerRecords answer() throws Throwable { - rebalanceListener.getValue().onPartitionsRevoked(partitions); - rebalanceListener.getValue().onPartitionsAssigned(partitions); - return ConsumerRecords.empty(); - } - }); + () -> { + rebalanceListener.getValue().onPartitionsRevoked(partitions); + rebalanceListener.getValue().onPartitionsAssigned(partitions); + return ConsumerRecords.empty(); + }); } private void expectPollInitialAssignment() { @@ -1623,17 +1580,14 @@ private void expectPollInitialAssignment() { sinkTask.open(partitions); EasyMock.expectLastCall(); - EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer(new IAnswer>() { - @Override - public ConsumerRecords answer() throws Throwable { - rebalanceListener.getValue().onPartitionsAssigned(partitions); - return ConsumerRecords.empty(); - } + EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer(() -> { + rebalanceListener.getValue().onPartitionsAssigned(partitions); + return ConsumerRecords.empty(); }); EasyMock.expect(consumer.position(TOPIC_PARTITION)).andReturn(FIRST_OFFSET); EasyMock.expect(consumer.position(TOPIC_PARTITION2)).andReturn(FIRST_OFFSET); - sinkTask.put(Collections.emptyList()); + sinkTask.put(Collections.emptyList()); EasyMock.expectLastCall(); } @@ -1657,34 +1611,26 @@ private void expectConsumerPoll(final int numMessages, final long timestamp, fin private void expectConsumerPoll(final int numMessages, final long timestamp, final TimestampType timestampType, Headers headers) { EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer( - new IAnswer>() { - @Override - public ConsumerRecords answer() throws Throwable { - List> records = new ArrayList<>(); - for (int i = 0; i < numMessages; i++) - records.add(new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturnedTp1 + i, timestamp, timestampType, 0L, 0, 0, RAW_KEY, RAW_VALUE, headers)); - recordsReturnedTp1 += numMessages; - return new ConsumerRecords<>( - numMessages > 0 ? - Collections.singletonMap(new TopicPartition(TOPIC, PARTITION), records) : - Collections.>>emptyMap() - ); - } - }); + () -> { + List> records = new ArrayList<>(); + for (int i = 0; i < numMessages; i++) + records.add(new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturnedTp1 + i, timestamp, timestampType, 0L, 0, 0, RAW_KEY, RAW_VALUE, headers)); + recordsReturnedTp1 += numMessages; + return new ConsumerRecords<>( + numMessages > 0 ? + Collections.singletonMap(new TopicPartition(TOPIC, PARTITION), records) : + Collections.emptyMap() + ); + }); } private void expectConsumerPoll(List> records) { EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer( - new IAnswer>() { - @Override - public ConsumerRecords answer() throws Throwable { - return new ConsumerRecords<>( - records.isEmpty() ? - Collections.>>emptyMap() : - Collections.singletonMap(new TopicPartition(TOPIC, PARTITION), records) - ); - } - }); + () -> new ConsumerRecords<>( + records.isEmpty() ? + Collections.emptyMap() : + Collections.singletonMap(new TopicPartition(TOPIC, PARTITION), records) + )); } private void expectConversionAndTransformation(final int numMessages) { @@ -1709,23 +1655,20 @@ private void expectConversionAndTransformation(final int numMessages, final Stri private void expectTransformation(final int numMessages, final String topicPrefix) { final Capture recordCapture = EasyMock.newCapture(); EasyMock.expect(transformationChain.apply(EasyMock.capture(recordCapture))) - .andAnswer(new IAnswer() { - @Override - public SinkRecord answer() { - SinkRecord origRecord = recordCapture.getValue(); - return topicPrefix != null && !topicPrefix.isEmpty() - ? origRecord.newRecord( - topicPrefix + origRecord.topic(), - origRecord.kafkaPartition(), - origRecord.keySchema(), - origRecord.key(), - origRecord.valueSchema(), - origRecord.value(), - origRecord.timestamp(), - origRecord.headers() - ) - : origRecord; - } + .andAnswer(() -> { + SinkRecord origRecord = recordCapture.getValue(); + return topicPrefix != null && !topicPrefix.isEmpty() + ? origRecord.newRecord( + topicPrefix + origRecord.topic(), + origRecord.kafkaPartition(), + origRecord.keySchema(), + origRecord.key(), + origRecord.valueSchema(), + origRecord.value(), + origRecord.timestamp(), + origRecord.headers() + ) + : origRecord; }).times(numMessages); } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java index 7a82b20de2..336f24b495 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java @@ -45,7 +45,6 @@ import org.easymock.Capture; import org.easymock.CaptureType; import org.easymock.EasyMock; -import org.easymock.IAnswer; import org.easymock.IExpectationSetters; import org.junit.After; import org.junit.Test; @@ -386,47 +385,38 @@ public void testAssignmentPauseResume() throws Exception { expectTaskGetTopic(true); expectPollInitialAssignment(); - expectOnePoll().andAnswer(new IAnswer() { - @Override - public Object answer() throws Throwable { - assertEquals(new HashSet<>(Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2, TOPIC_PARTITION3)), - sinkTaskContext.getValue().assignment()); - return null; - } + expectOnePoll().andAnswer(() -> { + assertEquals(new HashSet<>(Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2, TOPIC_PARTITION3)), + sinkTaskContext.getValue().assignment()); + return null; }); EasyMock.expect(consumer.assignment()).andReturn(new HashSet<>(Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2, TOPIC_PARTITION3))); - expectOnePoll().andAnswer(new IAnswer() { - @Override - public Object answer() throws Throwable { - try { - sinkTaskContext.getValue().pause(UNASSIGNED_TOPIC_PARTITION); - fail("Trying to pause unassigned partition should have thrown an Connect exception"); - } catch (ConnectException e) { - // expected - } - sinkTaskContext.getValue().pause(TOPIC_PARTITION, TOPIC_PARTITION2); - return null; + expectOnePoll().andAnswer(() -> { + try { + sinkTaskContext.getValue().pause(UNASSIGNED_TOPIC_PARTITION); + fail("Trying to pause unassigned partition should have thrown an Connect exception"); + } catch (ConnectException e) { + // expected } + sinkTaskContext.getValue().pause(TOPIC_PARTITION, TOPIC_PARTITION2); + return null; }); consumer.pause(Arrays.asList(UNASSIGNED_TOPIC_PARTITION)); PowerMock.expectLastCall().andThrow(new IllegalStateException("unassigned topic partition")); consumer.pause(Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2)); PowerMock.expectLastCall(); - expectOnePoll().andAnswer(new IAnswer() { - @Override - public Object answer() throws Throwable { - try { - sinkTaskContext.getValue().resume(UNASSIGNED_TOPIC_PARTITION); - fail("Trying to resume unassigned partition should have thrown an Connect exception"); - } catch (ConnectException e) { - // expected - } - - sinkTaskContext.getValue().resume(TOPIC_PARTITION, TOPIC_PARTITION2); - return null; + expectOnePoll().andAnswer(() -> { + try { + sinkTaskContext.getValue().resume(UNASSIGNED_TOPIC_PARTITION); + fail("Trying to resume unassigned partition should have thrown an Connect exception"); + } catch (ConnectException e) { + // expected } + + sinkTaskContext.getValue().resume(TOPIC_PARTITION, TOPIC_PARTITION2); + return null; }); consumer.resume(Arrays.asList(UNASSIGNED_TOPIC_PARTITION)); PowerMock.expectLastCall().andThrow(new IllegalStateException("unassigned topic partition")); @@ -458,25 +448,19 @@ public void testRewind() throws Exception { final long startOffset = 40L; final Map offsets = new HashMap<>(); - expectOnePoll().andAnswer(new IAnswer() { - @Override - public Object answer() throws Throwable { - offsets.put(TOPIC_PARTITION, startOffset); - sinkTaskContext.getValue().offset(offsets); - return null; - } + expectOnePoll().andAnswer(() -> { + offsets.put(TOPIC_PARTITION, startOffset); + sinkTaskContext.getValue().offset(offsets); + return null; }); consumer.seek(TOPIC_PARTITION, startOffset); EasyMock.expectLastCall(); - expectOnePoll().andAnswer(new IAnswer() { - @Override - public Object answer() throws Throwable { - Map offsets = sinkTaskContext.getValue().offsets(); - assertEquals(0, offsets.size()); - return null; - } + expectOnePoll().andAnswer(() -> { + Map offsets1 = sinkTaskContext.getValue().offsets(); + assertEquals(0, offsets1.size()); + return null; }); expectStopTask(); @@ -499,13 +483,10 @@ public void testRewindOnRebalanceDuringPoll() throws Exception { expectTaskGetTopic(true); expectPollInitialAssignment(); - expectRebalanceDuringPoll().andAnswer(new IAnswer() { - @Override - public Object answer() throws Throwable { - Map offsets = sinkTaskContext.getValue().offsets(); - assertEquals(0, offsets.size()); - return null; - } + expectRebalanceDuringPoll().andAnswer(() -> { + Map offsets = sinkTaskContext.getValue().offsets(); + assertEquals(0, offsets.size()); + return null; }); expectStopTask(); @@ -538,18 +519,15 @@ private void expectPollInitialAssignment() throws Exception { sinkTask.open(partitions); EasyMock.expectLastCall(); - EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer(new IAnswer>() { - @Override - public ConsumerRecords answer() throws Throwable { - rebalanceListener.getValue().onPartitionsAssigned(partitions); - return ConsumerRecords.empty(); - } + EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer(() -> { + rebalanceListener.getValue().onPartitionsAssigned(partitions); + return ConsumerRecords.empty(); }); EasyMock.expect(consumer.position(TOPIC_PARTITION)).andReturn(FIRST_OFFSET); EasyMock.expect(consumer.position(TOPIC_PARTITION2)).andReturn(FIRST_OFFSET); EasyMock.expect(consumer.position(TOPIC_PARTITION3)).andReturn(FIRST_OFFSET); - sinkTask.put(Collections.emptyList()); + sinkTask.put(Collections.emptyList()); EasyMock.expectLastCall(); } @@ -571,27 +549,24 @@ private Capture> expectPolls(final long pollDelayMs) thro // Stub out all the consumer stream/iterator responses, which we just want to verify occur, // but don't care about the exact details here. EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andStubAnswer( - new IAnswer>() { - @Override - public ConsumerRecords answer() throws Throwable { - // "Sleep" so time will progress - time.sleep(pollDelayMs); - ConsumerRecords records = new ConsumerRecords<>( - Collections.singletonMap( - new TopicPartition(TOPIC, PARTITION), - Arrays.asList( - new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, TIMESTAMP, TIMESTAMP_TYPE, 0L, 0, 0, RAW_KEY, RAW_VALUE) - ))); - recordsReturned++; - return records; - } - }); + () -> { + // "Sleep" so time will progress + time.sleep(pollDelayMs); + ConsumerRecords records = new ConsumerRecords<>( + Collections.singletonMap( + new TopicPartition(TOPIC, PARTITION), + Arrays.asList( + new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, TIMESTAMP, TIMESTAMP_TYPE, 0L, 0, 0, RAW_KEY, RAW_VALUE) + ))); + recordsReturned++; + return records; + }); EasyMock.expect(keyConverter.toConnectData(TOPIC, emptyHeaders(), RAW_KEY)).andReturn(new SchemaAndValue(KEY_SCHEMA, KEY)).anyTimes(); EasyMock.expect(valueConverter.toConnectData(TOPIC, emptyHeaders(), RAW_VALUE)).andReturn(new SchemaAndValue(VALUE_SCHEMA, VALUE)).anyTimes(); final Capture recordCapture = EasyMock.newCapture(); EasyMock.expect(transformationChain.apply(EasyMock.capture(recordCapture))).andAnswer( - (IAnswer) () -> recordCapture.getValue()).anyTimes(); + recordCapture::getValue).anyTimes(); Capture> capturedRecords = EasyMock.newCapture(CaptureType.ALL); sinkTask.put(EasyMock.capture(capturedRecords)); @@ -605,21 +580,18 @@ private IExpectationSetters expectOnePoll() { // returning empty data, we return one record. The expectation is that the data will be ignored by the // response behavior specified using the return value of this method. EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer( - new IAnswer>() { - @Override - public ConsumerRecords answer() throws Throwable { - // "Sleep" so time will progress - time.sleep(1L); - ConsumerRecords records = new ConsumerRecords<>( - Collections.singletonMap( - new TopicPartition(TOPIC, PARTITION), - Arrays.asList( - new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, TIMESTAMP, TIMESTAMP_TYPE, 0L, 0, 0, RAW_KEY, RAW_VALUE) - ))); - recordsReturned++; - return records; - } - }); + () -> { + // "Sleep" so time will progress + time.sleep(1L); + ConsumerRecords records = new ConsumerRecords<>( + Collections.singletonMap( + new TopicPartition(TOPIC, PARTITION), + Arrays.asList( + new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, TIMESTAMP, TIMESTAMP_TYPE, 0L, 0, 0, RAW_KEY, RAW_VALUE) + ))); + recordsReturned++; + return records; + }); EasyMock.expect(keyConverter.toConnectData(TOPIC, emptyHeaders(), RAW_KEY)).andReturn(new SchemaAndValue(KEY_SCHEMA, KEY)); EasyMock.expect(valueConverter.toConnectData(TOPIC, emptyHeaders(), RAW_VALUE)).andReturn(new SchemaAndValue(VALUE_SCHEMA, VALUE)); sinkTask.put(EasyMock.anyObject(Collection.class)); @@ -635,25 +607,22 @@ private IExpectationSetters expectRebalanceDuringPoll() throws Exception offsets.put(TOPIC_PARTITION, startOffset); EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andAnswer( - new IAnswer>() { - @Override - public ConsumerRecords answer() throws Throwable { - // "Sleep" so time will progress - time.sleep(1L); - - sinkTaskContext.getValue().offset(offsets); - rebalanceListener.getValue().onPartitionsAssigned(partitions); - - ConsumerRecords records = new ConsumerRecords<>( - Collections.singletonMap( - new TopicPartition(TOPIC, PARTITION), - Arrays.asList( - new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, TIMESTAMP, TIMESTAMP_TYPE, 0L, 0, 0, RAW_KEY, RAW_VALUE) - ))); - recordsReturned++; - return records; - } - }); + () -> { + // "Sleep" so time will progress + time.sleep(1L); + + sinkTaskContext.getValue().offset(offsets); + rebalanceListener.getValue().onPartitionsAssigned(partitions); + + ConsumerRecords records = new ConsumerRecords<>( + Collections.singletonMap( + new TopicPartition(TOPIC, PARTITION), + Arrays.asList( + new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, TIMESTAMP, TIMESTAMP_TYPE, 0L, 0, 0, RAW_KEY, RAW_VALUE) + ))); + recordsReturned++; + return records; + }); EasyMock.expect(consumer.position(TOPIC_PARTITION)).andReturn(FIRST_OFFSET); EasyMock.expect(consumer.position(TOPIC_PARTITION2)).andReturn(FIRST_OFFSET); @@ -696,14 +665,11 @@ private Capture expectOffsetCommit(final long expectedMess final Capture capturedCallback = EasyMock.newCapture(); consumer.commitAsync(EasyMock.eq(offsetsToCommit), EasyMock.capture(capturedCallback)); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Object answer() throws Throwable { - time.sleep(consumerCommitDelayMs); - if (invokeCallback) - capturedCallback.getValue().onComplete(offsetsToCommit, consumerCommitError); - return null; - } + PowerMock.expectLastCall().andAnswer(() -> { + time.sleep(consumerCommitDelayMs); + if (invokeCallback) + capturedCallback.getValue().onComplete(offsetsToCommit, consumerCommitError); + return null; }); return capturedCallback; } 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 ea0d54c28e..8c09887353 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 @@ -207,12 +207,9 @@ public void testStartPaused() throws Exception { createWorkerTask(TargetState.PAUSED); statusListener.onPause(taskId); - EasyMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Void answer() throws Throwable { - pauseLatch.countDown(); - return null; - } + EasyMock.expectLastCall().andAnswer(() -> { + pauseLatch.countDown(); + return null; }); expectClose(); @@ -338,12 +335,9 @@ public void testFailureInPoll() throws Exception { final CountDownLatch pollLatch = new CountDownLatch(1); final RuntimeException exception = new RuntimeException(); - EasyMock.expect(sourceTask.poll()).andAnswer(new IAnswer>() { - @Override - public List answer() throws Throwable { - pollLatch.countDown(); - throw exception; - } + EasyMock.expect(sourceTask.poll()).andAnswer(() -> { + pollLatch.countDown(); + throw exception; }); statusListener.onFailure(taskId, exception); @@ -673,13 +667,10 @@ public void testSlowTaskStart() throws Exception { sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class)); EasyMock.expectLastCall(); sourceTask.start(TASK_PROPS); - EasyMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Object answer() throws Throwable { - startupLatch.countDown(); - assertTrue(awaitLatch(finishStartupLatch)); - return null; - } + EasyMock.expectLastCall().andAnswer(() -> { + startupLatch.countDown(); + assertTrue(awaitLatch(finishStartupLatch)); + return null; }); statusListener.onStartup(taskId); @@ -856,14 +847,11 @@ private CountDownLatch expectEmptyPolls(int minimum, final AtomicInteger count) // run. The count passed in + latch returned just makes sure we get *at least* that number of // calls EasyMock.expect(sourceTask.poll()) - .andStubAnswer(new IAnswer>() { - @Override - public List answer() throws Throwable { - count.incrementAndGet(); - latch.countDown(); - Thread.sleep(10); - return Collections.emptyList(); - } + .andStubAnswer(() -> { + count.incrementAndGet(); + latch.countDown(); + Thread.sleep(10); + return Collections.emptyList(); }); return latch; } @@ -874,14 +862,11 @@ private CountDownLatch expectPolls(int minimum, final AtomicInteger count) throw // run. The count passed in + latch returned just makes sure we get *at least* that number of // calls EasyMock.expect(sourceTask.poll()) - .andStubAnswer(new IAnswer>() { - @Override - public List answer() throws Throwable { - count.incrementAndGet(); - latch.countDown(); - Thread.sleep(10); - return RECORDS; - } + .andStubAnswer(() -> { + count.incrementAndGet(); + latch.countDown(); + Thread.sleep(10); + return RECORDS; }); // Fallout of the poll() call expectSendRecordAnyTimes(); @@ -959,22 +944,19 @@ private Capture> expectSendRecord( IExpectationSetters> expect = EasyMock.expect( producer.send(EasyMock.capture(sent), EasyMock.capture(producerCallbacks))); - IAnswer> expectResponse = new IAnswer>() { - @Override - public Future answer() throws Throwable { - synchronized (producerCallbacks) { - for (org.apache.kafka.clients.producer.Callback cb : producerCallbacks.getValues()) { - if (sendSuccess) { - cb.onCompletion(new RecordMetadata(new TopicPartition("foo", 0), 0, 0, - 0L, 0L, 0, 0), null); - } else { - cb.onCompletion(null, new TopicAuthorizationException("foo")); - } + IAnswer> expectResponse = () -> { + synchronized (producerCallbacks) { + for (org.apache.kafka.clients.producer.Callback cb : producerCallbacks.getValues()) { + if (sendSuccess) { + cb.onCompletion(new RecordMetadata(new TopicPartition("foo", 0), 0, 0, + 0L, 0L, 0, 0), null); + } else { + cb.onCompletion(null, new TopicAuthorizationException("foo")); } - producerCallbacks.reset(); } - return sendFuture; + producerCallbacks.reset(); } + return sendFuture; }; if (anyTimes) expect.andStubAnswer(expectResponse); @@ -1018,19 +1000,9 @@ private void expectApplyTransformationChain(boolean anyTimes) { final Capture recordCapture = EasyMock.newCapture(); IExpectationSetters convertKeyExpect = EasyMock.expect(transformationChain.apply(EasyMock.capture(recordCapture))); if (anyTimes) - convertKeyExpect.andStubAnswer(new IAnswer() { - @Override - public SourceRecord answer() { - return recordCapture.getValue(); - } - }); + convertKeyExpect.andStubAnswer(recordCapture::getValue); else - convertKeyExpect.andAnswer(new IAnswer() { - @Override - public SourceRecord answer() { - return recordCapture.getValue(); - } - }); + convertKeyExpect.andAnswer(recordCapture::getValue); } private void expectTaskCommitRecordWithOffset(boolean anyTimes, boolean succeed) throws InterruptedException { diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskWithTopicCreationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskWithTopicCreationTest.java index 9784bd5588..c8fdfa72bf 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskWithTopicCreationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskWithTopicCreationTest.java @@ -232,12 +232,9 @@ public void testStartPaused() throws Exception { createWorkerTask(TargetState.PAUSED); statusListener.onPause(taskId); - EasyMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Void answer() throws Throwable { - pauseLatch.countDown(); - return null; - } + EasyMock.expectLastCall().andAnswer(() -> { + pauseLatch.countDown(); + return null; }); expectClose(); @@ -363,12 +360,9 @@ public void testFailureInPoll() throws Exception { final CountDownLatch pollLatch = new CountDownLatch(1); final RuntimeException exception = new RuntimeException(); - EasyMock.expect(sourceTask.poll()).andAnswer(new IAnswer>() { - @Override - public List answer() throws Throwable { - pollLatch.countDown(); - throw exception; - } + EasyMock.expect(sourceTask.poll()).andAnswer(() -> { + pollLatch.countDown(); + throw exception; }); statusListener.onFailure(taskId, exception); @@ -717,13 +711,10 @@ public void testSlowTaskStart() throws Exception { sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class)); EasyMock.expectLastCall(); sourceTask.start(TASK_PROPS); - EasyMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Object answer() throws Throwable { - startupLatch.countDown(); - assertTrue(awaitLatch(finishStartupLatch)); - return null; - } + EasyMock.expectLastCall().andAnswer(() -> { + startupLatch.countDown(); + assertTrue(awaitLatch(finishStartupLatch)); + return null; }); statusListener.onStartup(taskId); @@ -1189,14 +1180,11 @@ private CountDownLatch expectEmptyPolls(int minimum, final AtomicInteger count) // run. The count passed in + latch returned just makes sure we get *at least* that number of // calls EasyMock.expect(sourceTask.poll()) - .andStubAnswer(new IAnswer>() { - @Override - public List answer() throws Throwable { - count.incrementAndGet(); - latch.countDown(); - Thread.sleep(10); - return Collections.emptyList(); - } + .andStubAnswer(() -> { + count.incrementAndGet(); + latch.countDown(); + Thread.sleep(10); + return Collections.emptyList(); }); return latch; } @@ -1207,14 +1195,11 @@ private CountDownLatch expectPolls(int minimum, final AtomicInteger count) throw // run. The count passed in + latch returned just makes sure we get *at least* that number of // calls EasyMock.expect(sourceTask.poll()) - .andStubAnswer(new IAnswer>() { - @Override - public List answer() throws Throwable { - count.incrementAndGet(); - latch.countDown(); - Thread.sleep(10); - return RECORDS; - } + .andStubAnswer(() -> { + count.incrementAndGet(); + latch.countDown(); + Thread.sleep(10); + return RECORDS; }); // Fallout of the poll() call expectSendRecordAnyTimes(); @@ -1293,22 +1278,19 @@ private Capture> expectSendRecord( IExpectationSetters> expect = EasyMock.expect( producer.send(EasyMock.capture(sent), EasyMock.capture(producerCallbacks))); - IAnswer> expectResponse = new IAnswer>() { - @Override - public Future answer() throws Throwable { - synchronized (producerCallbacks) { - for (org.apache.kafka.clients.producer.Callback cb : producerCallbacks.getValues()) { - if (sendSuccess) { - cb.onCompletion(new RecordMetadata(new TopicPartition("foo", 0), 0, 0, - 0L, 0L, 0, 0), null); - } else { - cb.onCompletion(null, new TopicAuthorizationException("foo")); - } + IAnswer> expectResponse = () -> { + synchronized (producerCallbacks) { + for (org.apache.kafka.clients.producer.Callback cb : producerCallbacks.getValues()) { + if (sendSuccess) { + cb.onCompletion(new RecordMetadata(new TopicPartition("foo", 0), 0, 0, + 0L, 0L, 0, 0), null); + } else { + cb.onCompletion(null, new TopicAuthorizationException("foo")); } - producerCallbacks.reset(); } - return sendFuture; + producerCallbacks.reset(); } + return sendFuture; }; if (anyTimes) expect.andStubAnswer(expectResponse); @@ -1352,19 +1334,9 @@ private void expectApplyTransformationChain(boolean anyTimes) { final Capture recordCapture = EasyMock.newCapture(); IExpectationSetters convertKeyExpect = EasyMock.expect(transformationChain.apply(EasyMock.capture(recordCapture))); if (anyTimes) - convertKeyExpect.andStubAnswer(new IAnswer() { - @Override - public SourceRecord answer() { - return recordCapture.getValue(); - } - }); + convertKeyExpect.andStubAnswer(recordCapture::getValue); else - convertKeyExpect.andAnswer(new IAnswer() { - @Override - public SourceRecord answer() { - return recordCapture.getValue(); - } - }); + convertKeyExpect.andAnswer(recordCapture::getValue); } private void expectTaskCommitRecordWithOffset(boolean anyTimes, boolean succeed) throws InterruptedException { diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTaskTest.java index c26a88c0a6..81912f209a 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTaskTest.java @@ -26,7 +26,6 @@ import org.apache.kafka.connect.util.ConnectorTaskId; import org.apache.kafka.common.utils.MockTime; import org.easymock.EasyMock; -import org.easymock.IAnswer; import org.easymock.Mock; import org.junit.After; import org.junit.Before; @@ -185,26 +184,20 @@ public void cancelBeforeStopping() throws Exception { .createStrictMock(); final CountDownLatch stopped = new CountDownLatch(1); - final Thread thread = new Thread() { - @Override - public void run() { - try { - stopped.await(); - } catch (Exception e) { - } + final Thread thread = new Thread(() -> { + try { + stopped.await(); + } catch (Exception e) { } - }; + }); workerTask.initialize(TASK_CONFIG); EasyMock.expectLastCall(); workerTask.execute(); - expectLastCall().andAnswer(new IAnswer() { - @Override - public Void answer() throws Throwable { - thread.start(); - return null; - } + expectLastCall().andAnswer(() -> { + thread.start(); + return null; }); statusListener.onStartup(taskId); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java index d3bdcbf39d..cbf0802cd4 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java @@ -1072,7 +1072,7 @@ public void testConverterOverrides() throws Exception { @Test public void testProducerConfigsWithoutOverrides() { EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX)).andReturn( - new HashMap()); + new HashMap<>()); PowerMock.replayAll(); Map expectedConfigs = new HashMap<>(defaultProducerConfigs); expectedConfigs.put("client.id", "connector-producer-job-0"); @@ -1096,7 +1096,7 @@ public void testProducerConfigsWithOverrides() { expectedConfigs.put("metrics.context.connect.kafka.cluster.id", CLUSTER_ID); EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX)).andReturn( - new HashMap()); + new HashMap<>()); PowerMock.replayAll(); assertEquals(expectedConfigs, Worker.producerConfigs(TASK_ID, "connector-producer-" + TASK_ID, configWithOverrides, connectorConfig, null, allConnectorClientConfigOverridePolicy, CLUSTER_ID)); @@ -1117,7 +1117,7 @@ public void testProducerConfigsWithClientOverrides() { expectedConfigs.put("client.id", "producer-test-id"); expectedConfigs.put("metrics.context.connect.kafka.cluster.id", CLUSTER_ID); - Map connConfig = new HashMap(); + Map connConfig = new HashMap<>(); connConfig.put("linger.ms", "5000"); connConfig.put("batch.size", "1000"); EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX)) @@ -1177,7 +1177,7 @@ public void testConsumerConfigsWithClientOverrides() { expectedConfigs.put("client.id", "connector-consumer-test-1"); expectedConfigs.put("metrics.context.connect.kafka.cluster.id", CLUSTER_ID); - Map connConfig = new HashMap(); + Map connConfig = new HashMap<>(); connConfig.put("max.poll.records", "5000"); connConfig.put("max.poll.interval.ms", "1000"); EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX)) @@ -1194,7 +1194,7 @@ public void testConsumerConfigsClientOverridesWithNonePolicy() { props.put("consumer.max.poll.records", "5000"); WorkerConfig configWithOverrides = new StandaloneConfig(props); - Map connConfig = new HashMap(); + Map connConfig = new HashMap<>(); connConfig.put("max.poll.records", "5000"); connConfig.put("max.poll.interval.ms", "1000"); EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX)) @@ -1213,7 +1213,7 @@ public void testAdminConfigsClientOverridesWithAllPolicy() { props.put("consumer.bootstrap.servers", "localhost:4761"); WorkerConfig configWithOverrides = new StandaloneConfig(props); - Map connConfig = new HashMap(); + Map connConfig = new HashMap<>(); connConfig.put("metadata.max.age.ms", "10000"); Map expectedConfigs = new HashMap<>(workerProps); @@ -1238,7 +1238,7 @@ public void testAdminConfigsClientOverridesWithNonePolicy() { props.put("admin.metadata.max.age.ms", "5000"); WorkerConfig configWithOverrides = new StandaloneConfig(props); - Map connConfig = new HashMap(); + Map connConfig = new HashMap<>(); connConfig.put("metadata.max.age.ms", "10000"); EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_ADMIN_OVERRIDES_PREFIX)) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerWithTopicCreationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerWithTopicCreationTest.java index 327e45193a..3bf61088ed 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerWithTopicCreationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerWithTopicCreationTest.java @@ -1060,7 +1060,7 @@ public void testConverterOverrides() throws Exception { @Test public void testProducerConfigsWithoutOverrides() { EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX)).andReturn( - new HashMap()); + new HashMap<>()); PowerMock.replayAll(); Map expectedConfigs = new HashMap<>(defaultProducerConfigs); expectedConfigs.put("client.id", "connector-producer-job-0"); @@ -1083,7 +1083,7 @@ public void testProducerConfigsWithOverrides() { expectedConfigs.put("client.id", "producer-test-id"); expectedConfigs.put("metrics.context.connect.kafka.cluster.id", "test-cluster"); EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX)).andReturn( - new HashMap()); + new HashMap<>()); PowerMock.replayAll(); assertEquals(expectedConfigs, Worker.producerConfigs(TASK_ID, "connector-producer-" + TASK_ID, configWithOverrides, connectorConfig, null, allConnectorClientConfigOverridePolicy, CLUSTER_ID)); @@ -1103,7 +1103,7 @@ public void testProducerConfigsWithClientOverrides() { expectedConfigs.put("batch.size", "1000"); expectedConfigs.put("client.id", "producer-test-id"); expectedConfigs.put("metrics.context.connect.kafka.cluster.id", "test-cluster"); - Map connConfig = new HashMap(); + Map connConfig = new HashMap<>(); connConfig.put("linger.ms", "5000"); connConfig.put("batch.size", "1000"); EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX)) @@ -1160,7 +1160,7 @@ public void testConsumerConfigsWithClientOverrides() { expectedConfigs.put("max.poll.interval.ms", "1000"); expectedConfigs.put("client.id", "connector-consumer-test-1"); expectedConfigs.put("metrics.context.connect.kafka.cluster.id", "test-cluster"); - Map connConfig = new HashMap(); + Map connConfig = new HashMap<>(); connConfig.put("max.poll.records", "5000"); connConfig.put("max.poll.interval.ms", "1000"); EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX)) @@ -1196,7 +1196,7 @@ public void testAdminConfigsClientOverridesWithAllPolicy() { props.put("consumer.bootstrap.servers", "localhost:4761"); WorkerConfig configWithOverrides = new StandaloneConfig(props); - Map connConfig = new HashMap(); + Map connConfig = new HashMap<>(); connConfig.put("metadata.max.age.ms", "10000"); Map expectedConfigs = new HashMap<>(workerProps); @@ -1220,7 +1220,7 @@ public void testAdminConfigsClientOverridesWithNonePolicy() { props.put("admin.metadata.max.age.ms", "5000"); WorkerConfig configWithOverrides = new StandaloneConfig(props); - Map connConfig = new HashMap(); + Map connConfig = new HashMap<>(); connConfig.put("metadata.max.age.ms", "10000"); EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_ADMIN_OVERRIDES_PREFIX)) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedConfigTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedConfigTest.java index f176ae0b96..e95232748b 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedConfigTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedConfigTest.java @@ -214,9 +214,7 @@ public void shouldAllowSettingConfigTopicSettings() { topicSettings.put("bar", "bar value"); topicSettings.put("baz.bim", "100"); Map settings = configs(); - topicSettings.entrySet().forEach(e -> { - settings.put(DistributedConfig.CONFIG_STORAGE_PREFIX + e.getKey(), e.getValue()); - }); + topicSettings.forEach((k, v) -> settings.put(DistributedConfig.CONFIG_STORAGE_PREFIX + k, v)); DistributedConfig config = new DistributedConfig(settings); assertEquals(topicSettings, config.configStorageTopicSettings()); } @@ -228,9 +226,7 @@ public void shouldAllowSettingOffsetTopicSettings() { topicSettings.put("bar", "bar value"); topicSettings.put("baz.bim", "100"); Map settings = configs(); - topicSettings.entrySet().forEach(e -> { - settings.put(DistributedConfig.OFFSET_STORAGE_PREFIX + e.getKey(), e.getValue()); - }); + topicSettings.forEach((k, v) -> settings.put(DistributedConfig.OFFSET_STORAGE_PREFIX + k, v)); DistributedConfig config = new DistributedConfig(settings); assertEquals(topicSettings, config.offsetStorageTopicSettings()); } @@ -242,9 +238,7 @@ public void shouldAllowSettingStatusTopicSettings() { topicSettings.put("bar", "bar value"); topicSettings.put("baz.bim", "100"); Map settings = configs(); - topicSettings.entrySet().forEach(e -> { - settings.put(DistributedConfig.STATUS_STORAGE_PREFIX + e.getKey(), e.getValue()); - }); + topicSettings.forEach((k, v) -> settings.put(DistributedConfig.STATUS_STORAGE_PREFIX + k, v)); DistributedConfig config = new DistributedConfig(settings); assertEquals(topicSettings, config.statusStorageTopicSettings()); } @@ -260,9 +254,7 @@ public void shouldRemoveCompactionFromConfigTopicSettings() { topicSettings.put("partitions", "3"); Map settings = configs(); - topicSettings.forEach((k, v) -> { - settings.put(DistributedConfig.CONFIG_STORAGE_PREFIX + k, v); - }); + topicSettings.forEach((k, v) -> settings.put(DistributedConfig.CONFIG_STORAGE_PREFIX + k, v)); DistributedConfig config = new DistributedConfig(settings); Map actual = config.configStorageTopicSettings(); assertEquals(expectedTopicSettings, actual); @@ -279,9 +271,7 @@ public void shouldRemoveCompactionFromOffsetTopicSettings() { topicSettings.put("cleanup.policy", "something-else"); Map settings = configs(); - topicSettings.forEach((k, v) -> { - settings.put(DistributedConfig.OFFSET_STORAGE_PREFIX + k, v); - }); + topicSettings.forEach((k, v) -> settings.put(DistributedConfig.OFFSET_STORAGE_PREFIX + k, v)); DistributedConfig config = new DistributedConfig(settings); Map actual = config.offsetStorageTopicSettings(); assertEquals(expectedTopicSettings, actual); @@ -298,9 +288,7 @@ public void shouldRemoveCompactionFromStatusTopicSettings() { topicSettings.put("cleanup.policy", "something-else"); Map settings = configs(); - topicSettings.forEach((k, v) -> { - settings.put(DistributedConfig.STATUS_STORAGE_PREFIX + k, v); - }); + topicSettings.forEach((k, v) -> settings.put(DistributedConfig.STATUS_STORAGE_PREFIX + k, v)); DistributedConfig config = new DistributedConfig(settings); Map actual = config.statusStorageTopicSettings(); assertEquals(expectedTopicSettings, actual); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java index bea062ef1e..e31a03f11b 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java @@ -24,7 +24,6 @@ import org.apache.kafka.connect.connector.policy.NoneConnectorClientConfigOverridePolicy; import org.apache.kafka.connect.errors.AlreadyExistsException; import org.apache.kafka.connect.errors.NotFoundException; -import org.apache.kafka.connect.runtime.CloseableConnectorContext; import org.apache.kafka.connect.runtime.ConnectMetrics.MetricGroup; import org.apache.kafka.connect.runtime.ConnectorConfig; import org.apache.kafka.connect.runtime.Herder; @@ -57,7 +56,6 @@ import org.apache.kafka.connect.util.FutureCallback; import org.easymock.Capture; import org.easymock.EasyMock; -import org.easymock.IAnswer; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -77,6 +75,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; @@ -166,13 +165,13 @@ public class DistributedHerderTest { } private static final ClusterConfigState SNAPSHOT = new ClusterConfigState(1, null, Collections.singletonMap(CONN1, 3), Collections.singletonMap(CONN1, CONN1_CONFIG), Collections.singletonMap(CONN1, TargetState.STARTED), - TASK_CONFIGS_MAP, Collections.emptySet()); + TASK_CONFIGS_MAP, Collections.emptySet()); private static final ClusterConfigState SNAPSHOT_PAUSED_CONN1 = new ClusterConfigState(1, null, Collections.singletonMap(CONN1, 3), Collections.singletonMap(CONN1, CONN1_CONFIG), Collections.singletonMap(CONN1, TargetState.PAUSED), - TASK_CONFIGS_MAP, Collections.emptySet()); + TASK_CONFIGS_MAP, Collections.emptySet()); private static final ClusterConfigState SNAPSHOT_UPDATED_CONN1_CONFIG = new ClusterConfigState(1, null, Collections.singletonMap(CONN1, 3), Collections.singletonMap(CONN1, CONN1_CONFIG_UPDATED), Collections.singletonMap(CONN1, TargetState.STARTED), - TASK_CONFIGS_MAP, Collections.emptySet()); + TASK_CONFIGS_MAP, Collections.emptySet()); private static final String WORKER_ID = "localhost:8083"; private static final String KAFKA_CLUSTER_ID = "I4ZmrWqfT2e-upky_4fdPA"; @@ -191,6 +190,7 @@ public class DistributedHerderTest { @Mock private Plugins plugins; @Mock private PluginClassLoader pluginLoader; @Mock private DelegatingClassLoader delegatingLoader; + private CountDownLatch shutdownCalled = new CountDownLatch(1); private ConfigBackingStore.UpdateListener configUpdateListener; private WorkerRebalanceListener rebalanceListener; @@ -208,6 +208,7 @@ public void setUp() throws Exception { metrics = new MockConnectMetrics(time); worker = PowerMock.createMock(Worker.class); EasyMock.expect(worker.isSinkConnector(CONN1)).andStubReturn(Boolean.TRUE); + AutoCloseable uponShutdown = () -> shutdownCalled.countDown(); // Default to the old protocol unless specified otherwise connectProtocolVersion = CONNECT_PROTOCOL_V0; @@ -215,7 +216,8 @@ public void setUp() throws Exception { herder = PowerMock.createPartialMock(DistributedHerder.class, new String[]{"connectorTypeForClass", "updateDeletedConnectorStatus", "updateDeletedTaskStatus", "validateConnectorConfig"}, new DistributedConfig(HERDER_CONFIG), worker, WORKER_ID, KAFKA_CLUSTER_ID, - statusBackingStore, configBackingStore, member, MEMBER_URL, metrics, time, noneConnectorClientConfigOverridePolicy); + statusBackingStore, configBackingStore, member, MEMBER_URL, metrics, time, noneConnectorClientConfigOverridePolicy, + new AutoCloseable[]{uponShutdown}); configUpdateListener = herder.new ConfigUpdateListener(); rebalanceListener = herder.new RebalanceListener(time); @@ -244,21 +246,18 @@ public void testJoinAssignment() throws Exception { expectRebalance(1, Arrays.asList(CONN1), Arrays.asList(TASK1)); expectPostRebalanceCatchup(SNAPSHOT); Capture> onStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.>anyObject(), EasyMock.anyObject(), + worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onStart)); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Boolean answer() throws Throwable { - onStart.getValue().onCompletion(null, TargetState.STARTED); - return true; - } + PowerMock.expectLastCall().andAnswer(() -> { + onStart.getValue().onCompletion(null, TargetState.STARTED); + return true; }); member.wakeup(); PowerMock.expectLastCall(); EasyMock.expect(worker.isRunning(CONN1)).andReturn(true); PowerMock.expectLastCall(); EasyMock.expect(worker.connectorTaskConfigs(CONN1, conn1SinkConfig)).andReturn(TASK_CONFIGS); - worker.startTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.>anyObject(), EasyMock.>anyObject(), + worker.startTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall().andReturn(true); member.poll(EasyMock.anyInt()); @@ -282,20 +281,17 @@ public void testRebalance() throws Exception { expectRebalance(1, Arrays.asList(CONN1), Arrays.asList(TASK1)); expectPostRebalanceCatchup(SNAPSHOT); Capture> onFirstStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.>anyObject(), EasyMock.anyObject(), + worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onFirstStart)); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Boolean answer() throws Throwable { - onFirstStart.getValue().onCompletion(null, TargetState.STARTED); - return true; - } + PowerMock.expectLastCall().andAnswer(() -> { + onFirstStart.getValue().onCompletion(null, TargetState.STARTED); + return true; }); member.wakeup(); PowerMock.expectLastCall(); EasyMock.expect(worker.isRunning(CONN1)).andReturn(true); EasyMock.expect(worker.connectorTaskConfigs(CONN1, conn1SinkConfig)).andReturn(TASK_CONFIGS); - worker.startTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.>anyObject(), EasyMock.>anyObject(), + worker.startTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall().andReturn(true); member.poll(EasyMock.anyInt()); @@ -303,18 +299,15 @@ public Boolean answer() throws Throwable { EasyMock.expect(worker.getPlugins()).andReturn(plugins); expectRebalance(Arrays.asList(CONN1), Arrays.asList(TASK1), ConnectProtocol.Assignment.NO_ERROR, - 1, Arrays.asList(CONN1), Arrays.asList()); + 1, Arrays.asList(CONN1), Arrays.asList()); // and the new assignment started Capture> onSecondStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.>anyObject(), EasyMock.anyObject(), + worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onSecondStart)); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Boolean answer() throws Throwable { - onSecondStart.getValue().onCompletion(null, TargetState.STARTED); - return true; - } + PowerMock.expectLastCall().andAnswer(() -> { + onSecondStart.getValue().onCompletion(null, TargetState.STARTED); + return true; }); member.wakeup(); PowerMock.expectLastCall(); @@ -362,12 +355,9 @@ public void testIncrementalCooperativeRebalanceForNewMember() throws Exception { Capture> onStart = newCapture(); worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onStart)); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Boolean answer() throws Throwable { - onStart.getValue().onCompletion(null, TargetState.STARTED); - return true; - } + PowerMock.expectLastCall().andAnswer(() -> { + onStart.getValue().onCompletion(null, TargetState.STARTED); + return true; }); member.wakeup(); PowerMock.expectLastCall(); @@ -470,12 +460,9 @@ public void testIncrementalCooperativeRebalanceWithDelay() throws Exception { Capture> onStart = newCapture(); worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onStart)); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Boolean answer() throws Throwable { - onStart.getValue().onCompletion(null, TargetState.STARTED); - return true; - } + PowerMock.expectLastCall().andAnswer(() -> { + onStart.getValue().onCompletion(null, TargetState.STARTED); + return true; }); member.wakeup(); PowerMock.expectLastCall(); @@ -511,38 +498,32 @@ public void testRebalanceFailedConnector() throws Exception { expectRebalance(1, Arrays.asList(CONN1), Arrays.asList(TASK1)); expectPostRebalanceCatchup(SNAPSHOT); Capture> onFirstStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.>anyObject(), EasyMock.anyObject(), + worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onFirstStart)); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Boolean answer() throws Throwable { - onFirstStart.getValue().onCompletion(null, TargetState.STARTED); - return true; - } + PowerMock.expectLastCall().andAnswer(() -> { + onFirstStart.getValue().onCompletion(null, TargetState.STARTED); + return true; }); member.wakeup(); PowerMock.expectLastCall(); EasyMock.expect(worker.isRunning(CONN1)).andReturn(true); EasyMock.expect(worker.connectorTaskConfigs(CONN1, conn1SinkConfig)).andReturn(TASK_CONFIGS); - worker.startTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.>anyObject(), EasyMock.>anyObject(), + worker.startTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall().andReturn(true); member.poll(EasyMock.anyInt()); PowerMock.expectLastCall(); expectRebalance(Arrays.asList(CONN1), Arrays.asList(TASK1), ConnectProtocol.Assignment.NO_ERROR, - 1, Arrays.asList(CONN1), Arrays.asList()); + 1, Arrays.asList(CONN1), Arrays.asList()); // and the new assignment started Capture> onSecondStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.>anyObject(), EasyMock.anyObject(), + worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onSecondStart)); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Boolean answer() throws Throwable { - onSecondStart.getValue().onCompletion(null, TargetState.STARTED); - return true; - } + PowerMock.expectLastCall().andAnswer(() -> { + onSecondStart.getValue().onCompletion(null, TargetState.STARTED); + return true; }); member.wakeup(); PowerMock.expectLastCall(); @@ -584,23 +565,21 @@ public void revokeAndReassign(boolean incompleteRebalance) throws TimeoutExcepti EasyMock.expect(member.memberId()).andStubReturn("member"); EasyMock.expect(member.currentProtocolVersion()).andStubReturn(connectProtocolVersion); EasyMock.expect(worker.getPlugins()).andReturn(plugins); - expectRebalance(configOffset, Arrays.asList(CONN1), Arrays.asList(TASK1)); + // The lists need to be mutable because assignments might be removed + expectRebalance(configOffset, new ArrayList<>(singletonList(CONN1)), new ArrayList<>(singletonList(TASK1))); expectPostRebalanceCatchup(SNAPSHOT); Capture> onFirstStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.>anyObject(), EasyMock.anyObject(), + worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onFirstStart)); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Boolean answer() throws Throwable { - onFirstStart.getValue().onCompletion(null, TargetState.STARTED); - return true; - } + PowerMock.expectLastCall().andAnswer(() -> { + onFirstStart.getValue().onCompletion(null, TargetState.STARTED); + return true; }); member.wakeup(); PowerMock.expectLastCall(); EasyMock.expect(worker.isRunning(CONN1)).andReturn(true); EasyMock.expect(worker.connectorTaskConfigs(CONN1, conn1SinkConfig)).andReturn(TASK_CONFIGS); - worker.startTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.>anyObject(), EasyMock.>anyObject(), + worker.startTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall().andReturn(true); member.poll(EasyMock.anyInt()); @@ -612,7 +591,7 @@ public Boolean answer() throws Throwable { // Perform a partial re-balance just prior to the revocation // bump the configOffset to trigger reading the config topic to the end configOffset++; - expectRebalance(configOffset, Arrays.asList(), Arrays.asList()); + expectRebalance(configOffset, Arrays.asList(), Arrays.asList()); // give it the wrong snapshot, as if we're out of sync/can't reach the broker expectPostRebalanceCatchup(SNAPSHOT); member.requestRejoin(); @@ -622,9 +601,9 @@ public Boolean answer() throws Throwable { } // Revoke the connector in the next rebalance - expectRebalance(Arrays.asList(CONN1), Arrays.asList(), - ConnectProtocol.Assignment.NO_ERROR, configOffset, Arrays.asList(), - Arrays.asList()); + expectRebalance(Arrays.asList(CONN1), Arrays.asList(), + ConnectProtocol.Assignment.NO_ERROR, configOffset, Arrays.asList(), + Arrays.asList()); if (incompleteRebalance) { // Same as SNAPSHOT, except with an updated offset @@ -632,25 +611,22 @@ public Boolean answer() throws Throwable { ClusterConfigState secondSnapshot = new ClusterConfigState( configOffset, null, Collections.singletonMap(CONN1, 3), Collections.singletonMap(CONN1, CONN1_CONFIG), Collections.singletonMap(CONN1, TargetState.STARTED), - TASK_CONFIGS_MAP, Collections.emptySet()); + TASK_CONFIGS_MAP, Collections.emptySet()); expectPostRebalanceCatchup(secondSnapshot); } member.requestRejoin(); PowerMock.expectLastCall(); // re-assign the connector back to the same worker to ensure state was cleaned up - expectRebalance(configOffset, Arrays.asList(CONN1), Arrays.asList()); + expectRebalance(configOffset, Arrays.asList(CONN1), Arrays.asList()); EasyMock.expect(worker.getPlugins()).andReturn(plugins); Capture> onSecondStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.>anyObject(), - EasyMock.anyObject(), + worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), + EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onSecondStart)); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Boolean answer() throws Throwable { - onSecondStart.getValue().onCompletion(null, TargetState.STARTED); - return true; - } + PowerMock.expectLastCall().andAnswer(() -> { + onSecondStart.getValue().onCompletion(null, TargetState.STARTED); + return true; }); member.wakeup(); PowerMock.expectLastCall(); @@ -700,7 +676,7 @@ public void testHaltCleansUpWorker() { public void testCreateConnector() throws Exception { EasyMock.expect(member.memberId()).andStubReturn("leader"); EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - expectRebalance(1, Collections.emptyList(), Collections.emptyList()); + expectRebalance(1, Collections.emptyList(), Collections.emptyList()); expectPostRebalanceCatchup(SNAPSHOT); member.wakeup(); @@ -710,18 +686,15 @@ public void testCreateConnector() throws Exception { // be covered sufficiently by the unit tests for the AbstractHerder class Capture> validateCallback = newCapture(); herder.validateConnectorConfig(EasyMock.eq(CONN2_CONFIG), capture(validateCallback)); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Void answer() throws Throwable { - validateCallback.getValue().onCompletion(null, CONN2_CONFIG_INFOS); - return null; - } + PowerMock.expectLastCall().andAnswer(() -> { + validateCallback.getValue().onCompletion(null, CONN2_CONFIG_INFOS); + return null; }); // CONN2 is new, should succeed configBackingStore.putConnectorConfig(CONN2, CONN2_CONFIG); PowerMock.expectLastCall(); - ConnectorInfo info = new ConnectorInfo(CONN2, CONN2_CONFIG, Collections.emptyList(), + ConnectorInfo info = new ConnectorInfo(CONN2, CONN2_CONFIG, Collections.emptyList(), ConnectorType.SOURCE); putConnectorCallback.onCompletion(null, new Herder.Created<>(true, info)); PowerMock.expectLastCall(); @@ -758,7 +731,7 @@ public Void answer() throws Throwable { public void testCreateConnectorFailedValidation() throws Exception { EasyMock.expect(member.memberId()).andStubReturn("leader"); EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - expectRebalance(1, Collections.emptyList(), Collections.emptyList()); + expectRebalance(1, Collections.emptyList(), Collections.emptyList()); expectPostRebalanceCatchup(SNAPSHOT); HashMap config = new HashMap<>(CONN2_CONFIG); @@ -771,17 +744,14 @@ public void testCreateConnectorFailedValidation() throws Exception { // be covered sufficiently by the unit tests for the AbstractHerder class Capture> validateCallback = newCapture(); herder.validateConnectorConfig(EasyMock.eq(config), capture(validateCallback)); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Void answer() throws Throwable { - // CONN2 creation should fail - validateCallback.getValue().onCompletion(null, CONN2_INVALID_CONFIG_INFOS); - return null; - } + PowerMock.expectLastCall().andAnswer(() -> { + // CONN2 creation should fail + validateCallback.getValue().onCompletion(null, CONN2_INVALID_CONFIG_INFOS); + return null; }); Capture error = newCapture(); - putConnectorCallback.onCompletion(capture(error), EasyMock.>isNull()); + putConnectorCallback.onCompletion(capture(error), EasyMock.isNull()); PowerMock.expectLastCall(); member.poll(EasyMock.anyInt()); @@ -840,21 +810,18 @@ public void testCreateConnectorAlreadyExists() throws Exception { // be covered sufficiently by the unit tests for the AbstractHerder class Capture> validateCallback = newCapture(); herder.validateConnectorConfig(EasyMock.eq(CONN1_CONFIG), capture(validateCallback)); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Void answer() throws Throwable { - validateCallback.getValue().onCompletion(null, CONN1_CONFIG_INFOS); - return null; - } + PowerMock.expectLastCall().andAnswer(() -> { + validateCallback.getValue().onCompletion(null, CONN1_CONFIG_INFOS); + return null; }); - expectRebalance(1, Collections.emptyList(), Collections.emptyList()); + expectRebalance(1, Collections.emptyList(), Collections.emptyList()); expectPostRebalanceCatchup(SNAPSHOT); member.wakeup(); PowerMock.expectLastCall(); // CONN1 already exists - putConnectorCallback.onCompletion(EasyMock.anyObject(), EasyMock.>isNull()); + putConnectorCallback.onCompletion(EasyMock.anyObject(), EasyMock.isNull()); PowerMock.expectLastCall(); member.poll(EasyMock.anyInt()); PowerMock.expectLastCall(); @@ -887,17 +854,14 @@ public void testDestroyConnector() throws Exception { EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); // Start with one connector EasyMock.expect(worker.getPlugins()).andReturn(plugins); - expectRebalance(1, Arrays.asList(CONN1), Collections.emptyList()); + expectRebalance(1, Arrays.asList(CONN1), Collections.emptyList()); expectPostRebalanceCatchup(SNAPSHOT); Capture> onStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.>anyObject(), EasyMock.anyObject(), + worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onStart)); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Boolean answer() throws Throwable { - onStart.getValue().onCompletion(null, TargetState.STARTED); - return true; - } + PowerMock.expectLastCall().andAnswer(() -> { + onStart.getValue().onCompletion(null, TargetState.STARTED); + return true; }); EasyMock.expect(worker.isRunning(CONN1)).andReturn(true); EasyMock.expect(worker.connectorTaskConfigs(CONN1, conn1SinkConfig)).andReturn(TASK_CONFIGS); @@ -907,7 +871,7 @@ public Boolean answer() throws Throwable { PowerMock.expectLastCall(); configBackingStore.removeConnectorConfig(CONN1); PowerMock.expectLastCall(); - putConnectorCallback.onCompletion(null, new Herder.Created(false, null)); + putConnectorCallback.onCompletion(null, new Herder.Created<>(false, null)); PowerMock.expectLastCall(); member.poll(EasyMock.anyInt()); PowerMock.expectLastCall(); @@ -954,19 +918,16 @@ public void testRestartConnector() throws Exception { EasyMock.expect(member.memberId()).andStubReturn("leader"); EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); EasyMock.expect(worker.getPlugins()).andReturn(plugins); - expectRebalance(1, singletonList(CONN1), Collections.emptyList()); + expectRebalance(1, singletonList(CONN1), Collections.emptyList()); expectPostRebalanceCatchup(SNAPSHOT); member.poll(EasyMock.anyInt()); PowerMock.expectLastCall(); Capture> onFirstStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.>anyObject(), EasyMock.anyObject(), + worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onFirstStart)); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Boolean answer() throws Throwable { - onFirstStart.getValue().onCompletion(null, TargetState.STARTED); - return true; - } + PowerMock.expectLastCall().andAnswer(() -> { + onFirstStart.getValue().onCompletion(null, TargetState.STARTED); + return true; }); member.wakeup(); PowerMock.expectLastCall(); @@ -984,14 +945,11 @@ public Boolean answer() throws Throwable { PowerMock.expectLastCall(); EasyMock.expect(worker.getPlugins()).andReturn(plugins); Capture> onSecondStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.>anyObject(), EasyMock.anyObject(), + worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onSecondStart)); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Boolean answer() throws Throwable { - onSecondStart.getValue().onCompletion(null, TargetState.STARTED); - return true; - } + PowerMock.expectLastCall().andAnswer(() -> { + onSecondStart.getValue().onCompletion(null, TargetState.STARTED); + return true; }); member.wakeup(); PowerMock.expectLastCall(); @@ -1013,7 +971,7 @@ public void testRestartUnknownConnector() throws Exception { // get the initial assignment EasyMock.expect(member.memberId()).andStubReturn("leader"); EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - expectRebalance(1, Collections.emptyList(), Collections.emptyList()); + expectRebalance(1, Collections.emptyList(), Collections.emptyList()); expectPostRebalanceCatchup(SNAPSHOT); member.poll(EasyMock.anyInt()); PowerMock.expectLastCall(); @@ -1047,7 +1005,7 @@ public void testRestartConnectorRedirectToLeader() throws Exception { // get the initial assignment EasyMock.expect(member.memberId()).andStubReturn("member"); EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - expectRebalance(1, Collections.emptyList(), Collections.emptyList()); + expectRebalance(1, Collections.emptyList(), Collections.emptyList()); expectPostRebalanceCatchup(SNAPSHOT); member.poll(EasyMock.anyInt()); PowerMock.expectLastCall(); @@ -1082,7 +1040,7 @@ public void testRestartConnectorRedirectToOwner() throws Exception { // get the initial assignment EasyMock.expect(member.memberId()).andStubReturn("leader"); EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - expectRebalance(1, Collections.emptyList(), Collections.emptyList()); + expectRebalance(1, Collections.emptyList(), Collections.emptyList()); expectPostRebalanceCatchup(SNAPSHOT); member.poll(EasyMock.anyInt()); PowerMock.expectLastCall(); @@ -1130,11 +1088,11 @@ public void testRestartTask() throws Exception { // get the initial assignment EasyMock.expect(member.memberId()).andStubReturn("leader"); EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - expectRebalance(1, Collections.emptyList(), singletonList(TASK0)); + expectRebalance(1, Collections.emptyList(), singletonList(TASK0)); expectPostRebalanceCatchup(SNAPSHOT); member.poll(EasyMock.anyInt()); PowerMock.expectLastCall(); - worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.>anyObject(), EasyMock.>anyObject(), + worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall().andReturn(true); @@ -1148,7 +1106,7 @@ public void testRestartTask() throws Exception { worker.stopAndAwaitTask(TASK0); PowerMock.expectLastCall(); - worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.>anyObject(), EasyMock.>anyObject(), + worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall().andReturn(true); @@ -1168,7 +1126,7 @@ public void testRestartUnknownTask() throws Exception { // get the initial assignment EasyMock.expect(member.memberId()).andStubReturn("member"); EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - expectRebalance(1, Collections.emptyList(), Collections.emptyList()); + expectRebalance(1, Collections.emptyList(), Collections.emptyList()); expectPostRebalanceCatchup(SNAPSHOT); member.poll(EasyMock.anyInt()); PowerMock.expectLastCall(); @@ -1215,7 +1173,7 @@ public void testRestartTaskRedirectToLeader() throws Exception { // get the initial assignment EasyMock.expect(member.memberId()).andStubReturn("member"); EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - expectRebalance(1, Collections.emptyList(), Collections.emptyList()); + expectRebalance(1, Collections.emptyList(), Collections.emptyList()); expectPostRebalanceCatchup(SNAPSHOT); member.poll(EasyMock.anyInt()); PowerMock.expectLastCall(); @@ -1250,7 +1208,7 @@ public void testRestartTaskRedirectToOwner() throws Exception { // get the initial assignment EasyMock.expect(member.memberId()).andStubReturn("leader"); EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - expectRebalance(1, Collections.emptyList(), Collections.emptyList()); + expectRebalance(1, Collections.emptyList(), Collections.emptyList()); expectPostRebalanceCatchup(SNAPSHOT); member.poll(EasyMock.anyInt()); PowerMock.expectLastCall(); @@ -1291,7 +1249,7 @@ public void testConnectorConfigAdded() { EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); // join, no configs so no need to catch up on config topic - expectRebalance(-1, Collections.emptyList(), Collections.emptyList()); + expectRebalance(-1, Collections.emptyList(), Collections.emptyList()); member.poll(EasyMock.anyInt()); PowerMock.expectLastCall(); @@ -1305,18 +1263,15 @@ public void testConnectorConfigAdded() { member.requestRejoin(); PowerMock.expectLastCall(); // Performs rebalance and gets new assignment - expectRebalance(Collections.emptyList(), Collections.emptyList(), - ConnectProtocol.Assignment.NO_ERROR, 1, Arrays.asList(CONN1), Collections.emptyList()); + expectRebalance(Collections.emptyList(), Collections.emptyList(), + ConnectProtocol.Assignment.NO_ERROR, 1, Arrays.asList(CONN1), Collections.emptyList()); EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); Capture> onStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.>anyObject(), EasyMock.anyObject(), + worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onStart)); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Boolean answer() throws Throwable { - onStart.getValue().onCompletion(null, TargetState.STARTED); - return true; - } + PowerMock.expectLastCall().andAnswer(() -> { + onStart.getValue().onCompletion(null, TargetState.STARTED); + return true; }); member.wakeup(); PowerMock.expectLastCall(); @@ -1345,18 +1300,15 @@ public void testConnectorConfigUpdate() throws Exception { EasyMock.expect(worker.connectorNames()).andStubReturn(Collections.singleton(CONN1)); // join - expectRebalance(1, Arrays.asList(CONN1), Collections.emptyList()); + expectRebalance(1, Arrays.asList(CONN1), Collections.emptyList()); expectPostRebalanceCatchup(SNAPSHOT); EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); Capture> onFirstStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.>anyObject(), EasyMock.anyObject(), + worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onFirstStart)); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Boolean answer() throws Throwable { - onFirstStart.getValue().onCompletion(null, TargetState.STARTED); - return true; - } + PowerMock.expectLastCall().andAnswer(() -> { + onFirstStart.getValue().onCompletion(null, TargetState.STARTED); + return true; }); member.wakeup(); PowerMock.expectLastCall(); @@ -1376,14 +1328,11 @@ public Boolean answer() throws Throwable { PowerMock.expectLastCall(); EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); Capture> onSecondStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.>anyObject(), EasyMock.anyObject(), + worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onSecondStart)); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Boolean answer() throws Throwable { - onSecondStart.getValue().onCompletion(null, TargetState.STARTED); - return true; - } + PowerMock.expectLastCall().andAnswer(() -> { + onSecondStart.getValue().onCompletion(null, TargetState.STARTED); + return true; }); member.wakeup(); PowerMock.expectLastCall(); @@ -1418,18 +1367,15 @@ public void testConnectorPaused() throws Exception { EasyMock.expect(worker.connectorNames()).andStubReturn(Collections.singleton(CONN1)); // join - expectRebalance(1, Arrays.asList(CONN1), Collections.emptyList()); + expectRebalance(1, Arrays.asList(CONN1), Collections.emptyList()); expectPostRebalanceCatchup(SNAPSHOT); EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); Capture> onStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.>anyObject(), EasyMock.anyObject(), + worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onStart)); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Boolean answer() throws Throwable { - onStart.getValue().onCompletion(null, TargetState.STARTED); - return true; - } + PowerMock.expectLastCall().andAnswer(() -> { + onStart.getValue().onCompletion(null, TargetState.STARTED); + return true; }); member.wakeup(); PowerMock.expectLastCall(); @@ -1450,12 +1396,9 @@ public Boolean answer() throws Throwable { Capture> onPause = newCapture(); worker.setTargetState(EasyMock.eq(CONN1), EasyMock.eq(TargetState.PAUSED), capture(onPause)); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Void answer() throws Throwable { - onStart.getValue().onCompletion(null, TargetState.PAUSED); - return null; - } + PowerMock.expectLastCall().andAnswer(() -> { + onStart.getValue().onCompletion(null, TargetState.PAUSED); + return null; }); member.poll(EasyMock.anyInt()); @@ -1484,18 +1427,15 @@ public void testConnectorResumed() throws Exception { EasyMock.expect(worker.connectorNames()).andStubReturn(Collections.singleton(CONN1)); // start with the connector paused - expectRebalance(1, Arrays.asList(CONN1), Collections.emptyList()); + expectRebalance(1, Arrays.asList(CONN1), Collections.emptyList()); expectPostRebalanceCatchup(SNAPSHOT_PAUSED_CONN1); EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); Capture> onStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.>anyObject(), EasyMock.anyObject(), + worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.PAUSED), capture(onStart)); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Boolean answer() throws Throwable { - onStart.getValue().onCompletion(null, TargetState.PAUSED); - return true; - } + PowerMock.expectLastCall().andAnswer(() -> { + onStart.getValue().onCompletion(null, TargetState.PAUSED); + return true; }); EasyMock.expect(worker.getPlugins()).andReturn(plugins); @@ -1517,12 +1457,9 @@ public Boolean answer() throws Throwable { Capture> onResume = newCapture(); worker.setTargetState(EasyMock.eq(CONN1), EasyMock.eq(TargetState.STARTED), capture(onResume)); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Void answer() throws Throwable { - onResume.getValue().onCompletion(null, TargetState.STARTED); - return null; - } + PowerMock.expectLastCall().andAnswer(() -> { + onResume.getValue().onCompletion(null, TargetState.STARTED); + return null; }); member.wakeup(); PowerMock.expectLastCall(); @@ -1553,9 +1490,9 @@ public void testUnknownConnectorPaused() throws Exception { EasyMock.expect(worker.connectorNames()).andStubReturn(Collections.singleton(CONN1)); // join - expectRebalance(1, Collections.emptyList(), singletonList(TASK0)); + expectRebalance(1, Collections.emptyList(), singletonList(TASK0)); expectPostRebalanceCatchup(SNAPSHOT); - worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.>anyObject(), EasyMock.>anyObject(), + worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall().andReturn(true); member.poll(EasyMock.anyInt()); @@ -1589,12 +1526,12 @@ public void testConnectorPausedRunningTaskOnly() throws Exception { EasyMock.expect(member.memberId()).andStubReturn("member"); EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - EasyMock.expect(worker.connectorNames()).andStubReturn(Collections.emptySet()); + EasyMock.expect(worker.connectorNames()).andStubReturn(Collections.emptySet()); // join - expectRebalance(1, Collections.emptyList(), singletonList(TASK0)); + expectRebalance(1, Collections.emptyList(), singletonList(TASK0)); expectPostRebalanceCatchup(SNAPSHOT); - worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.>anyObject(), EasyMock.>anyObject(), + worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall().andReturn(true); member.poll(EasyMock.anyInt()); @@ -1611,12 +1548,9 @@ public void testConnectorPausedRunningTaskOnly() throws Exception { Capture> onPause = newCapture(); worker.setTargetState(EasyMock.eq(CONN1), EasyMock.eq(TargetState.PAUSED), capture(onPause)); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Void answer() throws Throwable { - onPause.getValue().onCompletion(null, TargetState.STARTED); - return null; - } + PowerMock.expectLastCall().andAnswer(() -> { + onPause.getValue().onCompletion(null, TargetState.STARTED); + return null; }); member.wakeup(); PowerMock.expectLastCall(); @@ -1640,12 +1574,12 @@ public void testConnectorResumedRunningTaskOnly() throws Exception { EasyMock.expect(member.memberId()).andStubReturn("member"); EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - EasyMock.expect(worker.connectorNames()).andStubReturn(Collections.emptySet()); + EasyMock.expect(worker.connectorNames()).andStubReturn(Collections.emptySet()); // join - expectRebalance(1, Collections.emptyList(), singletonList(TASK0)); + expectRebalance(1, Collections.emptyList(), singletonList(TASK0)); expectPostRebalanceCatchup(SNAPSHOT_PAUSED_CONN1); - worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.>anyObject(), EasyMock.>anyObject(), + worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.PAUSED)); PowerMock.expectLastCall().andReturn(true); member.poll(EasyMock.anyInt()); @@ -1662,12 +1596,9 @@ public void testConnectorResumedRunningTaskOnly() throws Exception { Capture> onStart = newCapture(); worker.setTargetState(EasyMock.eq(CONN1), EasyMock.eq(TargetState.STARTED), capture(onStart)); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Void answer() throws Throwable { - onStart.getValue().onCompletion(null, TargetState.STARTED); - return null; - } + PowerMock.expectLastCall().andAnswer(() -> { + onStart.getValue().onCompletion(null, TargetState.STARTED); + return null; }); member.wakeup(); PowerMock.expectLastCall(); @@ -1700,7 +1631,7 @@ public void testTaskConfigAdded() { EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); // join - expectRebalance(-1, Collections.emptyList(), Collections.emptyList()); + expectRebalance(-1, Collections.emptyList(), Collections.emptyList()); member.poll(EasyMock.anyInt()); PowerMock.expectLastCall(); @@ -1714,10 +1645,10 @@ public void testTaskConfigAdded() { member.requestRejoin(); PowerMock.expectLastCall(); // Performs rebalance and gets new assignment - expectRebalance(Collections.emptyList(), Collections.emptyList(), - ConnectProtocol.Assignment.NO_ERROR, 1, Collections.emptyList(), + expectRebalance(Collections.emptyList(), Collections.emptyList(), + ConnectProtocol.Assignment.NO_ERROR, 1, Collections.emptyList(), Arrays.asList(TASK0)); - worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.>anyObject(), EasyMock.>anyObject(), + worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall().andReturn(true); member.poll(EasyMock.anyInt()); @@ -1738,9 +1669,9 @@ public void testJoinLeaderCatchUpFails() throws Exception { // Join group and as leader fail to do assignment EasyMock.expect(member.memberId()).andStubReturn("leader"); EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); - expectRebalance(Collections.emptyList(), Collections.emptyList(), - ConnectProtocol.Assignment.CONFIG_MISMATCH, 1, Collections.emptyList(), - Collections.emptyList()); + expectRebalance(Collections.emptyList(), Collections.emptyList(), + ConnectProtocol.Assignment.CONFIG_MISMATCH, 1, Collections.emptyList(), + Collections.emptyList()); // Reading to end of log times out configBackingStore.refresh(EasyMock.anyLong(), EasyMock.anyObject(TimeUnit.class)); EasyMock.expectLastCall().andThrow(new TimeoutException()); @@ -1754,20 +1685,17 @@ public void testJoinLeaderCatchUpFails() throws Exception { EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); Capture> onStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.>anyObject(), EasyMock.anyObject(), + worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onStart)); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Boolean answer() throws Throwable { - onStart.getValue().onCompletion(null, TargetState.STARTED); - return true; - } + PowerMock.expectLastCall().andAnswer(() -> { + onStart.getValue().onCompletion(null, TargetState.STARTED); + return true; }); member.wakeup(); PowerMock.expectLastCall(); EasyMock.expect(worker.getPlugins()).andReturn(plugins); EasyMock.expect(worker.connectorTaskConfigs(CONN1, conn1SinkConfig)).andReturn(TASK_CONFIGS); - worker.startTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.>anyObject(), EasyMock.>anyObject(), + worker.startTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall().andReturn(true); EasyMock.expect(worker.isRunning(CONN1)).andReturn(true); @@ -1825,12 +1753,9 @@ public void testJoinLeaderCatchUpRetriesForIncrementalCooperative() throws Excep Capture> onStart = newCapture(); worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onStart)); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Boolean answer() throws Throwable { - onStart.getValue().onCompletion(null, TargetState.STARTED); - return true; - } + PowerMock.expectLastCall().andAnswer(() -> { + onStart.getValue().onCompletion(null, TargetState.STARTED); + return true; }); member.wakeup(); PowerMock.expectLastCall(); @@ -1845,9 +1770,9 @@ public Boolean answer() throws Throwable { // Another rebalance is triggered but this time it fails to read to the max offset and // triggers a re-sync - expectRebalance(Collections.emptyList(), Collections.emptyList(), - ConnectProtocol.Assignment.CONFIG_MISMATCH, 1, Collections.emptyList(), - Collections.emptyList()); + expectRebalance(Collections.emptyList(), Collections.emptyList(), + ConnectProtocol.Assignment.CONFIG_MISMATCH, 1, Collections.emptyList(), + Collections.emptyList()); // The leader will retry a few times to read to the end of the config log int retries = 2; @@ -1920,12 +1845,9 @@ public void testJoinLeaderCatchUpFailsForIncrementalCooperative() throws Excepti Capture> onStart = newCapture(); worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onStart)); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Boolean answer() throws Throwable { - onStart.getValue().onCompletion(null, TargetState.STARTED); - return true; - } + PowerMock.expectLastCall().andAnswer(() -> { + onStart.getValue().onCompletion(null, TargetState.STARTED); + return true; }); member.wakeup(); PowerMock.expectLastCall(); @@ -1940,9 +1862,9 @@ public Boolean answer() throws Throwable { // Another rebalance is triggered but this time it fails to read to the max offset and // triggers a re-sync - expectRebalance(Collections.emptyList(), Collections.emptyList(), - ConnectProtocol.Assignment.CONFIG_MISMATCH, 1, Collections.emptyList(), - Collections.emptyList()); + expectRebalance(Collections.emptyList(), Collections.emptyList(), + ConnectProtocol.Assignment.CONFIG_MISMATCH, 1, Collections.emptyList(), + Collections.emptyList()); // The leader will exhaust the retries while trying to read to the end of the config log int maxRetries = 5; @@ -2003,7 +1925,7 @@ public void testAccessors() throws Exception { EasyMock.expect(member.memberId()).andStubReturn("leader"); EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); EasyMock.expect(worker.getPlugins()).andReturn(plugins).anyTimes(); - expectRebalance(1, Collections.emptyList(), Collections.emptyList()); + expectRebalance(1, Collections.emptyList(), Collections.emptyList()); EasyMock.expect(configBackingStore.snapshot()).andReturn(SNAPSHOT).times(2); WorkerConfigTransformer configTransformer = EasyMock.mock(WorkerConfigTransformer.class); @@ -2012,7 +1934,7 @@ public void testAccessors() throws Exception { EasyMock.replay(configTransformer); ClusterConfigState snapshotWithTransform = new ClusterConfigState(1, null, Collections.singletonMap(CONN1, 3), Collections.singletonMap(CONN1, CONN1_CONFIG), Collections.singletonMap(CONN1, TargetState.STARTED), - TASK_CONFIGS_MAP, Collections.emptySet(), configTransformer); + TASK_CONFIGS_MAP, Collections.emptySet(), configTransformer); expectPostRebalanceCatchup(snapshotWithTransform); @@ -2058,18 +1980,15 @@ public void testAccessors() throws Exception { @Test public void testPutConnectorConfig() throws Exception { EasyMock.expect(member.memberId()).andStubReturn("leader"); - expectRebalance(1, Arrays.asList(CONN1), Collections.emptyList()); + expectRebalance(1, Arrays.asList(CONN1), Collections.emptyList()); expectPostRebalanceCatchup(SNAPSHOT); EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); Capture> onFirstStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.>anyObject(), EasyMock.anyObject(), + worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onFirstStart)); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Boolean answer() throws Throwable { - onFirstStart.getValue().onCompletion(null, TargetState.STARTED); - return true; - } + PowerMock.expectLastCall().andAnswer(() -> { + onFirstStart.getValue().onCompletion(null, TargetState.STARTED); + return true; }); EasyMock.expect(worker.isRunning(CONN1)).andReturn(true); EasyMock.expect(worker.connectorTaskConfigs(CONN1, conn1SinkConfig)).andReturn(TASK_CONFIGS); @@ -2089,21 +2008,15 @@ public Boolean answer() throws Throwable { Capture> validateCallback = newCapture(); herder.validateConnectorConfig(EasyMock.eq(CONN1_CONFIG_UPDATED), capture(validateCallback)); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Void answer() throws Throwable { - validateCallback.getValue().onCompletion(null, CONN1_CONFIG_INFOS); - return null; - } + PowerMock.expectLastCall().andAnswer(() -> { + validateCallback.getValue().onCompletion(null, CONN1_CONFIG_INFOS); + return null; }); configBackingStore.putConnectorConfig(CONN1, CONN1_CONFIG_UPDATED); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Object answer() throws Throwable { - // Simulate response to writing config + waiting until end of log to be read - configUpdateListener.onConnectorConfigUpdate(CONN1); - return null; - } + PowerMock.expectLastCall().andAnswer(() -> { + // Simulate response to writing config + waiting until end of log to be read + configUpdateListener.onConnectorConfigUpdate(CONN1); + return null; }); // As a result of reconfig, should need to update snapshot. With only connector updates, we'll just restart // connector without rebalance @@ -2112,14 +2025,11 @@ public Object answer() throws Throwable { PowerMock.expectLastCall(); EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V0); Capture> onSecondStart = newCapture(); - worker.startConnector(EasyMock.eq(CONN1), EasyMock.>anyObject(), EasyMock.anyObject(), + worker.startConnector(EasyMock.eq(CONN1), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), capture(onSecondStart)); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Boolean answer() throws Throwable { - onSecondStart.getValue().onCompletion(null, TargetState.STARTED); - return true; - } + PowerMock.expectLastCall().andAnswer(() -> { + onSecondStart.getValue().onCompletion(null, TargetState.STARTED); + return true; }); EasyMock.expect(worker.isRunning(CONN1)).andReturn(true); EasyMock.expect(worker.connectorTaskConfigs(CONN1, conn1SinkConfigUpdated)).andReturn(TASK_CONFIGS); @@ -2305,6 +2215,13 @@ public void testThreadNames() { getThreadFactory().newThread(EMPTY_RUNNABLE).getName().startsWith("StartAndStopExecutor")); } + @Test + public void testHerderStopServicesClosesUponShutdown() { + assertEquals(1, shutdownCalled.getCount()); + herder.stopServices(); + assertEquals(0, shutdownCalled.getCount()); + } + private void expectRebalance(final long offset, final List assignedConnectors, final List assignedTasks) { @@ -2331,29 +2248,26 @@ private void expectRebalance(final Collection revokedConnectors, final List assignedTasks, int delay) { member.ensureActive(); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Object answer() throws Throwable { - ExtendedAssignment assignment; - if (!revokedConnectors.isEmpty() || !revokedTasks.isEmpty()) { - rebalanceListener.onRevoked("leader", revokedConnectors, revokedTasks); - } - - if (connectProtocolVersion == CONNECT_PROTOCOL_V0) { - assignment = new ExtendedAssignment( - connectProtocolVersion, error, "leader", "leaderUrl", offset, - assignedConnectors, assignedTasks, - Collections.emptyList(), Collections.emptyList(), 0); - } else { - assignment = new ExtendedAssignment( - connectProtocolVersion, error, "leader", "leaderUrl", offset, - new ArrayList<>(assignedConnectors), new ArrayList<>(assignedTasks), - new ArrayList<>(revokedConnectors), new ArrayList<>(revokedTasks), delay); - } - rebalanceListener.onAssigned(assignment, 3); - time.sleep(100L); - return null; + PowerMock.expectLastCall().andAnswer(() -> { + ExtendedAssignment assignment; + if (!revokedConnectors.isEmpty() || !revokedTasks.isEmpty()) { + rebalanceListener.onRevoked("leader", revokedConnectors, revokedTasks); } + + if (connectProtocolVersion == CONNECT_PROTOCOL_V0) { + assignment = new ExtendedAssignment( + connectProtocolVersion, error, "leader", "leaderUrl", offset, + assignedConnectors, assignedTasks, + Collections.emptyList(), Collections.emptyList(), 0); + } else { + assignment = new ExtendedAssignment( + connectProtocolVersion, error, "leader", "leaderUrl", offset, + assignedConnectors, assignedTasks, + new ArrayList<>(revokedConnectors), new ArrayList<>(revokedTasks), delay); + } + rebalanceListener.onAssigned(assignment, 3); + time.sleep(100L); + return null; }); if (!revokedConnectors.isEmpty()) { diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java index 6ad1b3bddd..9178dd852e 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java @@ -27,7 +27,6 @@ import org.apache.kafka.common.message.SyncGroupResponseData; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.requests.AbstractRequest; import org.apache.kafka.common.requests.FindCoordinatorResponse; import org.apache.kafka.common.requests.JoinGroupResponse; import org.apache.kafka.common.requests.RequestTestUtils; @@ -153,25 +152,25 @@ public void setup() { 1L, null, Collections.singletonMap(connectorId1, 1), - Collections.singletonMap(connectorId1, (Map) new HashMap()), + Collections.singletonMap(connectorId1, new HashMap()), Collections.singletonMap(connectorId1, TargetState.STARTED), - Collections.singletonMap(taskId1x0, (Map) new HashMap()), - Collections.emptySet() + Collections.singletonMap(taskId1x0, new HashMap()), + Collections.emptySet() ); Map configState2ConnectorTaskCounts = new HashMap<>(); configState2ConnectorTaskCounts.put(connectorId1, 2); configState2ConnectorTaskCounts.put(connectorId2, 1); Map> configState2ConnectorConfigs = new HashMap<>(); - configState2ConnectorConfigs.put(connectorId1, new HashMap()); - configState2ConnectorConfigs.put(connectorId2, new HashMap()); + configState2ConnectorConfigs.put(connectorId1, new HashMap<>()); + configState2ConnectorConfigs.put(connectorId2, new HashMap<>()); Map configState2TargetStates = new HashMap<>(); configState2TargetStates.put(connectorId1, TargetState.STARTED); configState2TargetStates.put(connectorId2, TargetState.STARTED); Map> configState2TaskConfigs = new HashMap<>(); - configState2TaskConfigs.put(taskId1x0, new HashMap()); - configState2TaskConfigs.put(taskId1x1, new HashMap()); - configState2TaskConfigs.put(taskId2x0, new HashMap()); + configState2TaskConfigs.put(taskId1x0, new HashMap<>()); + configState2TaskConfigs.put(taskId1x1, new HashMap<>()); + configState2TaskConfigs.put(taskId2x0, new HashMap<>()); configState2 = new ClusterConfigState( 2L, null, @@ -179,7 +178,7 @@ public void setup() { configState2ConnectorConfigs, configState2TargetStates, configState2TaskConfigs, - Collections.emptySet() + Collections.emptySet() ); Map configStateSingleTaskConnectorsConnectorTaskCounts = new HashMap<>(); @@ -187,17 +186,17 @@ public void setup() { configStateSingleTaskConnectorsConnectorTaskCounts.put(connectorId2, 1); configStateSingleTaskConnectorsConnectorTaskCounts.put(connectorId3, 1); Map> configStateSingleTaskConnectorsConnectorConfigs = new HashMap<>(); - configStateSingleTaskConnectorsConnectorConfigs.put(connectorId1, new HashMap()); - configStateSingleTaskConnectorsConnectorConfigs.put(connectorId2, new HashMap()); - configStateSingleTaskConnectorsConnectorConfigs.put(connectorId3, new HashMap()); + configStateSingleTaskConnectorsConnectorConfigs.put(connectorId1, new HashMap<>()); + configStateSingleTaskConnectorsConnectorConfigs.put(connectorId2, new HashMap<>()); + configStateSingleTaskConnectorsConnectorConfigs.put(connectorId3, new HashMap<>()); Map configStateSingleTaskConnectorsTargetStates = new HashMap<>(); configStateSingleTaskConnectorsTargetStates.put(connectorId1, TargetState.STARTED); configStateSingleTaskConnectorsTargetStates.put(connectorId2, TargetState.STARTED); configStateSingleTaskConnectorsTargetStates.put(connectorId3, TargetState.STARTED); Map> configStateSingleTaskConnectorsTaskConfigs = new HashMap<>(); - configStateSingleTaskConnectorsTaskConfigs.put(taskId1x0, new HashMap()); - configStateSingleTaskConnectorsTaskConfigs.put(taskId2x0, new HashMap()); - configStateSingleTaskConnectorsTaskConfigs.put(taskId3x0, new HashMap()); + configStateSingleTaskConnectorsTaskConfigs.put(taskId1x0, new HashMap<>()); + configStateSingleTaskConnectorsTaskConfigs.put(taskId2x0, new HashMap<>()); + configStateSingleTaskConnectorsTaskConfigs.put(taskId3x0, new HashMap<>()); configStateSingleTaskConnectors = new ClusterConfigState( 2L, null, @@ -205,7 +204,7 @@ public void setup() { configStateSingleTaskConnectorsConnectorConfigs, configStateSingleTaskConnectorsTargetStates, configStateSingleTaskConnectorsTaskConfigs, - Collections.emptySet() + Collections.emptySet() ); } @@ -253,16 +252,13 @@ public void testNormalJoinGroupLeader() { memberConfigOffsets.put("leader", 1L); memberConfigOffsets.put("member", 1L); client.prepareResponse(joinGroupLeaderResponse(1, consumerId, memberConfigOffsets, Errors.NONE)); - client.prepareResponse(new MockClient.RequestMatcher() { - @Override - public boolean matches(AbstractRequest body) { - SyncGroupRequest sync = (SyncGroupRequest) body; - return sync.data().memberId().equals(consumerId) && - sync.data().generationId() == 1 && - sync.groupAssignments().containsKey(consumerId); - } + client.prepareResponse(body -> { + SyncGroupRequest sync = (SyncGroupRequest) body; + return sync.data().memberId().equals(consumerId) && + sync.data().generationId() == 1 && + sync.groupAssignments().containsKey(consumerId); }, syncGroupResponse(ConnectProtocol.Assignment.NO_ERROR, "leader", 1L, Collections.singletonList(connectorId1), - Collections.emptyList(), Errors.NONE)); + Collections.emptyList(), Errors.NONE)); coordinator.ensureActiveGroup(); assertFalse(coordinator.rejoinNeededOrPending()); @@ -290,15 +286,12 @@ public void testNormalJoinGroupFollower() { // normal join group client.prepareResponse(joinGroupFollowerResponse(1, memberId, "leader", Errors.NONE)); - client.prepareResponse(new MockClient.RequestMatcher() { - @Override - public boolean matches(AbstractRequest body) { - SyncGroupRequest sync = (SyncGroupRequest) body; - return sync.data().memberId().equals(memberId) && - sync.data().generationId() == 1 && - sync.data().assignments().isEmpty(); - } - }, syncGroupResponse(ConnectProtocol.Assignment.NO_ERROR, "leader", 1L, Collections.emptyList(), + client.prepareResponse(body -> { + SyncGroupRequest sync = (SyncGroupRequest) body; + return sync.data().memberId().equals(memberId) && + sync.data().generationId() == 1 && + sync.data().assignments().isEmpty(); + }, syncGroupResponse(ConnectProtocol.Assignment.NO_ERROR, "leader", 1L, Collections.emptyList(), Collections.singletonList(taskId1x0), Errors.NONE)); coordinator.ensureActiveGroup(); @@ -331,20 +324,17 @@ public void testJoinLeaderCannotAssign() { // config mismatch results in assignment error client.prepareResponse(joinGroupFollowerResponse(1, memberId, "leader", Errors.NONE)); - MockClient.RequestMatcher matcher = new MockClient.RequestMatcher() { - @Override - public boolean matches(AbstractRequest body) { - SyncGroupRequest sync = (SyncGroupRequest) body; - return sync.data().memberId().equals(memberId) && - sync.data().generationId() == 1 && - sync.data().assignments().isEmpty(); - } + MockClient.RequestMatcher matcher = body -> { + SyncGroupRequest sync = (SyncGroupRequest) body; + return sync.data().memberId().equals(memberId) && + sync.data().generationId() == 1 && + sync.data().assignments().isEmpty(); }; client.prepareResponse(matcher, syncGroupResponse(ConnectProtocol.Assignment.CONFIG_MISMATCH, "leader", 10L, - Collections.emptyList(), Collections.emptyList(), Errors.NONE)); + Collections.emptyList(), Collections.emptyList(), Errors.NONE)); client.prepareResponse(joinGroupFollowerResponse(1, memberId, "leader", Errors.NONE)); client.prepareResponse(matcher, syncGroupResponse(ConnectProtocol.Assignment.NO_ERROR, "leader", 1L, - Collections.emptyList(), Collections.singletonList(taskId1x0), Errors.NONE)); + Collections.emptyList(), Collections.singletonList(taskId1x0), Errors.NONE)); coordinator.ensureActiveGroup(); PowerMock.verifyAll(); @@ -362,7 +352,7 @@ public void testRejoinGroup() { // join the group once client.prepareResponse(joinGroupFollowerResponse(1, "consumer", "leader", Errors.NONE)); - client.prepareResponse(syncGroupResponse(ConnectProtocol.Assignment.NO_ERROR, "leader", 1L, Collections.emptyList(), + client.prepareResponse(syncGroupResponse(ConnectProtocol.Assignment.NO_ERROR, "leader", 1L, Collections.emptyList(), Collections.singletonList(taskId1x0), Errors.NONE)); coordinator.ensureActiveGroup(); @@ -377,7 +367,7 @@ public void testRejoinGroup() { coordinator.requestRejoin(); client.prepareResponse(joinGroupFollowerResponse(1, "consumer", "leader", Errors.NONE)); client.prepareResponse(syncGroupResponse(ConnectProtocol.Assignment.NO_ERROR, "leader", 1L, Collections.singletonList(connectorId1), - Collections.emptyList(), Errors.NONE)); + Collections.emptyList(), Errors.NONE)); coordinator.ensureActiveGroup(); assertEquals(1, rebalanceListener.revokedCount); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/health/ConnectClusterStateImplTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/health/ConnectClusterStateImplTest.java index d8a7e49ee8..58eb5a9e97 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/health/ConnectClusterStateImplTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/health/ConnectClusterStateImplTest.java @@ -21,7 +21,6 @@ import org.apache.kafka.connect.util.Callback; import org.easymock.Capture; import org.easymock.EasyMock; -import org.easymock.IAnswer; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -63,12 +62,9 @@ public void setUp() { public void connectors() { Capture>> callback = EasyMock.newCapture(); herder.connectors(EasyMock.capture(callback)); - EasyMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Void answer() { - callback.getValue().onCompletion(null, expectedConnectors); - return null; - } + EasyMock.expectLastCall().andAnswer(() -> { + callback.getValue().onCompletion(null, expectedConnectors); + return null; }); EasyMock.replay(herder); assertEquals(expectedConnectors, connectClusterState.connectors()); @@ -80,12 +76,9 @@ public void connectorConfig() { final Map expectedConfig = Collections.singletonMap("key", "value"); Capture>> callback = EasyMock.newCapture(); herder.connectorConfig(EasyMock.eq(connName), EasyMock.capture(callback)); - EasyMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Void answer() { - callback.getValue().onCompletion(null, expectedConfig); - return null; - } + EasyMock.expectLastCall().andAnswer(() -> { + callback.getValue().onCompletion(null, expectedConfig); + return null; }); EasyMock.replay(herder); Map actualConfig = connectClusterState.connectorConfig(connName); @@ -106,13 +99,10 @@ public void kafkaClusterId() { public void connectorsFailure() { Capture>> callback = EasyMock.newCapture(); herder.connectors(EasyMock.capture(callback)); - EasyMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Void answer() { - Throwable timeout = new TimeoutException(); - callback.getValue().onCompletion(timeout, null); - return null; - } + EasyMock.expectLastCall().andAnswer(() -> { + Throwable timeout = new TimeoutException(); + callback.getValue().onCompletion(timeout, null); + return null; }); EasyMock.replay(herder); assertThrows(ConnectException.class, connectClusterState::connectors); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java index 0c81ddd739..ff3af44aa3 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java @@ -470,7 +470,7 @@ private String executeGet(String host, int port, String endpoint) throws IOExcep private String executePut(String host, int port, String endpoint, String jsonBody) throws IOException { HttpPut request = new HttpPut(endpoint); - StringEntity entity = new StringEntity(jsonBody, "UTF-8"); + StringEntity entity = new StringEntity(jsonBody, StandardCharsets.UTF_8.name()); entity.setContentType("application/json"); request.setEntity(entity); CloseableHttpClient httpClient = HttpClients.createMinimal(); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java index d372252305..e458c5fba1 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java @@ -113,31 +113,31 @@ public class ConnectorPluginsResourceTest { ConfigDef connectorConfigDef = ConnectorConfig.configDef(); List connectorConfigValues = connectorConfigDef.validate(props); List partialConnectorConfigValues = connectorConfigDef.validate(partialProps); - ConfigInfos result = AbstractHerder.generateResult(ConnectorPluginsResourceTestConnector.class.getName(), connectorConfigDef.configKeys(), connectorConfigValues, Collections.emptyList()); - ConfigInfos partialResult = AbstractHerder.generateResult(ConnectorPluginsResourceTestConnector.class.getName(), connectorConfigDef.configKeys(), partialConnectorConfigValues, Collections.emptyList()); + ConfigInfos result = AbstractHerder.generateResult(ConnectorPluginsResourceTestConnector.class.getName(), connectorConfigDef.configKeys(), connectorConfigValues, Collections.emptyList()); + ConfigInfos partialResult = AbstractHerder.generateResult(ConnectorPluginsResourceTestConnector.class.getName(), connectorConfigDef.configKeys(), partialConnectorConfigValues, Collections.emptyList()); configs.addAll(result.values()); partialConfigs.addAll(partialResult.values()); - ConfigKeyInfo configKeyInfo = new ConfigKeyInfo("test.string.config", "STRING", true, null, "HIGH", "Test configuration for string type.", null, -1, "NONE", "test.string.config", Collections.emptyList()); - ConfigValueInfo configValueInfo = new ConfigValueInfo("test.string.config", "testString", Collections.emptyList(), Collections.emptyList(), true); + ConfigKeyInfo configKeyInfo = new ConfigKeyInfo("test.string.config", "STRING", true, null, "HIGH", "Test configuration for string type.", null, -1, "NONE", "test.string.config", Collections.emptyList()); + ConfigValueInfo configValueInfo = new ConfigValueInfo("test.string.config", "testString", Collections.emptyList(), Collections.emptyList(), true); ConfigInfo configInfo = new ConfigInfo(configKeyInfo, configValueInfo); configs.add(configInfo); partialConfigs.add(configInfo); - configKeyInfo = new ConfigKeyInfo("test.int.config", "INT", true, null, "MEDIUM", "Test configuration for integer type.", "Test", 1, "MEDIUM", "test.int.config", Collections.emptyList()); - configValueInfo = new ConfigValueInfo("test.int.config", "1", Arrays.asList("1", "2", "3"), Collections.emptyList(), true); + configKeyInfo = new ConfigKeyInfo("test.int.config", "INT", true, null, "MEDIUM", "Test configuration for integer type.", "Test", 1, "MEDIUM", "test.int.config", Collections.emptyList()); + configValueInfo = new ConfigValueInfo("test.int.config", "1", Arrays.asList("1", "2", "3"), Collections.emptyList(), true); configInfo = new ConfigInfo(configKeyInfo, configValueInfo); configs.add(configInfo); partialConfigs.add(configInfo); - configKeyInfo = new ConfigKeyInfo("test.string.config.default", "STRING", false, "", "LOW", "Test configuration with default value.", null, -1, "NONE", "test.string.config.default", Collections.emptyList()); - configValueInfo = new ConfigValueInfo("test.string.config.default", "", Collections.emptyList(), Collections.emptyList(), true); + configKeyInfo = new ConfigKeyInfo("test.string.config.default", "STRING", false, "", "LOW", "Test configuration with default value.", null, -1, "NONE", "test.string.config.default", Collections.emptyList()); + configValueInfo = new ConfigValueInfo("test.string.config.default", "", Collections.emptyList(), Collections.emptyList(), true); configInfo = new ConfigInfo(configKeyInfo, configValueInfo); configs.add(configInfo); partialConfigs.add(configInfo); - configKeyInfo = new ConfigKeyInfo("test.list.config", "LIST", true, null, "HIGH", "Test configuration for list type.", "Test", 2, "LONG", "test.list.config", Collections.emptyList()); - configValueInfo = new ConfigValueInfo("test.list.config", "a,b", Arrays.asList("a", "b", "c"), Collections.emptyList(), true); + configKeyInfo = new ConfigKeyInfo("test.list.config", "LIST", true, null, "HIGH", "Test configuration for list type.", "Test", 2, "LONG", "test.list.config", Collections.emptyList()); + configValueInfo = new ConfigValueInfo("test.list.config", "a,b", Arrays.asList("a", "b", "c"), Collections.emptyList(), true); configInfo = new ConfigInfo(configKeyInfo, configValueInfo); configs.add(configInfo); partialConfigs.add(configInfo); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java index 9469243725..f3d42a2f58 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java @@ -311,7 +311,7 @@ public void testCreateConnectorNotLeader() throws Throwable { expectAndCallbackNotLeaderException(cb); // Should forward request EasyMock.expect(RestClient.httpRequest(EasyMock.eq("http://leader:8083/connectors?forward=false"), EasyMock.eq("POST"), EasyMock.isNull(), EasyMock.eq(body), EasyMock.anyObject(), EasyMock.anyObject(WorkerConfig.class))) - .andReturn(new RestClient.HttpResponse<>(201, new HashMap(), new ConnectorInfo(CONNECTOR_NAME, CONNECTOR_CONFIG, CONNECTOR_TASK_NAMES, + .andReturn(new RestClient.HttpResponse<>(201, new HashMap<>(), new ConnectorInfo(CONNECTOR_NAME, CONNECTOR_CONFIG, CONNECTOR_TASK_NAMES, ConnectorType.SOURCE))); PowerMock.replayAll(); @@ -519,6 +519,55 @@ public void testGetConnectorConfigConnectorNotFound() { PowerMock.verifyAll(); } + @Test + public void testGetTasksConfig() throws Throwable { + final ConnectorTaskId connectorTask0 = new ConnectorTaskId(CONNECTOR_NAME, 0); + final Map connectorTask0Configs = new HashMap<>(); + connectorTask0Configs.put("connector-task0-config0", "123"); + connectorTask0Configs.put("connector-task0-config1", "456"); + final ConnectorTaskId connectorTask1 = new ConnectorTaskId(CONNECTOR_NAME, 1); + final Map connectorTask1Configs = new HashMap<>(); + connectorTask0Configs.put("connector-task1-config0", "321"); + connectorTask0Configs.put("connector-task1-config1", "654"); + final ConnectorTaskId connector2Task0 = new ConnectorTaskId(CONNECTOR2_NAME, 0); + final Map connector2Task0Configs = Collections.singletonMap("connector2-task0-config0", "789"); + + final Map> expectedTasksConnector = new HashMap<>(); + expectedTasksConnector.put(connectorTask0, connectorTask0Configs); + expectedTasksConnector.put(connectorTask1, connectorTask1Configs); + final Map> expectedTasksConnector2 = new HashMap<>(); + expectedTasksConnector2.put(connector2Task0, connector2Task0Configs); + + final Capture>>> cb1 = Capture.newInstance(); + herder.tasksConfig(EasyMock.eq(CONNECTOR_NAME), EasyMock.capture(cb1)); + expectAndCallbackResult(cb1, expectedTasksConnector); + final Capture>>> cb2 = Capture.newInstance(); + herder.tasksConfig(EasyMock.eq(CONNECTOR2_NAME), EasyMock.capture(cb2)); + expectAndCallbackResult(cb2, expectedTasksConnector2); + + PowerMock.replayAll(); + + Map> tasksConfig = connectorsResource.getTasksConfig(CONNECTOR_NAME, NULL_HEADERS, FORWARD); + assertEquals(expectedTasksConnector, tasksConfig); + Map> tasksConfig2 = connectorsResource.getTasksConfig(CONNECTOR2_NAME, NULL_HEADERS, FORWARD); + assertEquals(expectedTasksConnector2, tasksConfig2); + + PowerMock.verifyAll(); + } + + @Test(expected = NotFoundException.class) + public void testGetTasksConfigConnectorNotFound() throws Throwable { + final Capture>>> cb = Capture.newInstance(); + herder.tasksConfig(EasyMock.eq(CONNECTOR_NAME), EasyMock.capture(cb)); + expectAndCallbackException(cb, new NotFoundException("not found")); + + PowerMock.replayAll(); + + connectorsResource.getTasksConfig(CONNECTOR_NAME, NULL_HEADERS, FORWARD); + + PowerMock.verifyAll(); + } + @Test public void testPutConnectorConfig() throws Throwable { final Capture>> cb = Capture.newInstance(); @@ -743,7 +792,7 @@ public void testRestartConnectorLeaderRedirect() throws Throwable { EasyMock.expect(RestClient.httpRequest(EasyMock.eq("http://leader:8083/connectors/" + CONNECTOR_NAME + "/restart?forward=true"), EasyMock.eq("POST"), EasyMock.isNull(), EasyMock.isNull(), EasyMock.anyObject(), EasyMock.anyObject(WorkerConfig.class))) - .andReturn(new RestClient.HttpResponse<>(202, new HashMap(), null)); + .andReturn(new RestClient.HttpResponse<>(202, new HashMap<>(), null)); PowerMock.replayAll(); @@ -761,7 +810,7 @@ public void testRestartConnectorOwnerRedirect() throws Throwable { EasyMock.expect(RestClient.httpRequest(EasyMock.eq("http://owner:8083/connectors/" + CONNECTOR_NAME + "/restart?forward=false"), EasyMock.eq("POST"), EasyMock.isNull(), EasyMock.isNull(), EasyMock.anyObject(), EasyMock.anyObject(WorkerConfig.class))) - .andReturn(new RestClient.HttpResponse<>(202, new HashMap(), null)); + .andReturn(new RestClient.HttpResponse<>(202, new HashMap<>(), null)); PowerMock.replayAll(); @@ -794,7 +843,7 @@ public void testRestartTaskLeaderRedirect() throws Throwable { EasyMock.expect(RestClient.httpRequest(EasyMock.eq("http://leader:8083/connectors/" + CONNECTOR_NAME + "/tasks/0/restart?forward=true"), EasyMock.eq("POST"), EasyMock.isNull(), EasyMock.isNull(), EasyMock.anyObject(), EasyMock.anyObject(WorkerConfig.class))) - .andReturn(new RestClient.HttpResponse<>(202, new HashMap(), null)); + .andReturn(new RestClient.HttpResponse<>(202, new HashMap<>(), null)); PowerMock.replayAll(); @@ -814,7 +863,7 @@ public void testRestartTaskOwnerRedirect() throws Throwable { EasyMock.expect(RestClient.httpRequest(EasyMock.eq("http://owner:8083/connectors/" + CONNECTOR_NAME + "/tasks/0/restart?forward=false"), EasyMock.eq("POST"), EasyMock.isNull(), EasyMock.isNull(), EasyMock.anyObject(), EasyMock.anyObject(WorkerConfig.class))) - .andReturn(new RestClient.HttpResponse<>(202, new HashMap(), null)); + .andReturn(new RestClient.HttpResponse<>(202, new HashMap<>(), null)); PowerMock.replayAll(); @@ -916,9 +965,8 @@ public void testCompleteOrForwardWithErrorAndNoForwardUrl() throws Throwable { PowerMock.replayAll(); - ConnectRestException e = assertThrows(ConnectRestException.class, () -> { - connectorsResource.destroyConnector(CONNECTOR_NAME, NULL_HEADERS, FORWARD); - }); + ConnectRestException e = assertThrows(ConnectRestException.class, () -> + connectorsResource.destroyConnector(CONNECTOR_NAME, NULL_HEADERS, FORWARD)); assertTrue(e.getMessage().contains("no known leader URL")); PowerMock.verifyAll(); } @@ -928,12 +976,9 @@ private byte[] serializeAsBytes(final T value) throws IOException { } private void expectAndCallbackResult(final Capture> cb, final T value) { - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Void answer() throws Throwable { - cb.getValue().onCompletion(null, value); - return null; - } + PowerMock.expectLastCall().andAnswer(() -> { + cb.getValue().onCompletion(null, value); + return null; }); } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java index 1395410fe0..9649082c8f 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java @@ -27,7 +27,6 @@ import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.errors.NotFoundException; import org.apache.kafka.connect.runtime.AbstractStatus; -import org.apache.kafka.connect.runtime.CloseableConnectorContext; import org.apache.kafka.connect.runtime.ConnectorConfig; import org.apache.kafka.connect.runtime.ConnectorStatus; import org.apache.kafka.connect.runtime.Herder; @@ -59,7 +58,6 @@ import org.apache.kafka.connect.util.FutureCallback; import org.easymock.Capture; import org.easymock.EasyMock; -import org.easymock.IAnswer; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -245,7 +243,7 @@ public void testDestroyConnector() throws Exception { Connector connectorMock = PowerMock.createMock(SourceConnector.class); expectConfigValidation(connectorMock, true, config); - EasyMock.expect(statusBackingStore.getAll(CONNECTOR_NAME)).andReturn(Collections.emptyList()); + EasyMock.expect(statusBackingStore.getAll(CONNECTOR_NAME)).andReturn(Collections.emptyList()); statusBackingStore.put(new ConnectorStatus(CONNECTOR_NAME, AbstractStatus.State.DESTROYED, WORKER_ID, 0)); statusBackingStore.put(new TaskStatus(new ConnectorTaskId(CONNECTOR_NAME, 0), TaskStatus.State.DESTROYED, WORKER_ID, 0)); @@ -288,12 +286,9 @@ public void testRestartConnector() throws Exception { Capture> onStart = EasyMock.newCapture(); worker.startConnector(EasyMock.eq(CONNECTOR_NAME), EasyMock.eq(config), EasyMock.anyObject(HerderConnectorContext.class), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), EasyMock.capture(onStart)); - EasyMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Boolean answer() throws Throwable { - onStart.getValue().onCompletion(null, TargetState.STARTED); - return true; - } + EasyMock.expectLastCall().andAnswer(() -> { + onStart.getValue().onCompletion(null, TargetState.STARTED); + return true; }); PowerMock.replayAll(); @@ -324,12 +319,9 @@ public void testRestartConnectorFailureOnStart() throws Exception { worker.startConnector(EasyMock.eq(CONNECTOR_NAME), EasyMock.eq(config), EasyMock.anyObject(HerderConnectorContext.class), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), EasyMock.capture(onStart)); Exception exception = new ConnectException("Failed to start connector"); - EasyMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Boolean answer() throws Throwable { - onStart.getValue().onCompletion(exception, null); - return true; - } + EasyMock.expectLastCall().andAnswer(() -> { + onStart.getValue().onCompletion(exception, null); + return true; }); PowerMock.replayAll(); @@ -473,11 +465,11 @@ public void testAccessors() throws Exception { // Check accessors with empty worker listConnectorsCb.onCompletion(null, Collections.EMPTY_SET); EasyMock.expectLastCall(); - connectorInfoCb.onCompletion(EasyMock.anyObject(), EasyMock.isNull()); + connectorInfoCb.onCompletion(EasyMock.anyObject(), EasyMock.isNull()); EasyMock.expectLastCall(); - connectorConfigCb.onCompletion(EasyMock.anyObject(), EasyMock.>isNull()); + connectorConfigCb.onCompletion(EasyMock.anyObject(), EasyMock.isNull()); EasyMock.expectLastCall(); - taskConfigsCb.onCompletion(EasyMock.anyObject(), EasyMock.>isNull()); + taskConfigsCb.onCompletion(EasyMock.anyObject(), EasyMock.isNull()); EasyMock.expectLastCall(); // Create connector @@ -549,14 +541,11 @@ public void testPutConnectorConfig() throws Exception { EasyMock.expectLastCall(); Capture> capturedConfig = EasyMock.newCapture(); Capture> onStart = EasyMock.newCapture(); - worker.startConnector(EasyMock.eq(CONNECTOR_NAME), EasyMock.capture(capturedConfig), EasyMock.anyObject(), + worker.startConnector(EasyMock.eq(CONNECTOR_NAME), EasyMock.capture(capturedConfig), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), EasyMock.capture(onStart)); - EasyMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Boolean answer() throws Throwable { - onStart.getValue().onCompletion(null, TargetState.STARTED); - return true; - } + EasyMock.expectLastCall().andAnswer(() -> { + onStart.getValue().onCompletion(null, TargetState.STARTED); + return true; }); EasyMock.expect(worker.isRunning(CONNECTOR_NAME)).andReturn(true); EasyMock.expect(worker.isTopicCreationEnabled()).andReturn(true); @@ -661,12 +650,9 @@ private void expectAdd(SourceSink sourceSink) { worker.startConnector(EasyMock.eq(CONNECTOR_NAME), EasyMock.eq(connectorProps), EasyMock.anyObject(HerderConnectorContext.class), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED), EasyMock.capture(onStart)); // EasyMock.expectLastCall().andReturn(true); - EasyMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Boolean answer() throws Throwable { - onStart.getValue().onCompletion(null, TargetState.STARTED); - return true; - } + EasyMock.expectLastCall().andAnswer(() -> { + onStart.getValue().onCompletion(null, TargetState.STARTED); + return true; }); EasyMock.expect(worker.isRunning(CONNECTOR_NAME)).andReturn(true); if (sourceSink == SourceSink.SOURCE) { @@ -762,7 +748,7 @@ private void expectConfigValidation( EasyMock.expect(connectorMock.config()).andStubReturn(new ConfigDef()); for (Map config : configs) - EasyMock.expect(connectorMock.validate(config)).andReturn(new Config(Collections.emptyList())); + EasyMock.expect(connectorMock.validate(config)).andReturn(new Config(Collections.emptyList())); EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)).andReturn(pluginLoader); } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/FileOffsetBackingStoreTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/FileOffsetBackingStoreTest.java index 530a9bbb88..571844d488 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/FileOffsetBackingStoreTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/FileOffsetBackingStoreTest.java @@ -60,7 +60,7 @@ public class FileOffsetBackingStoreTest { public void setup() throws IOException { store = new FileOffsetBackingStore(); tempFile = File.createTempFile("fileoffsetbackingstore", null); - props = new HashMap(); + props = new HashMap<>(); props.put(StandaloneConfig.OFFSET_STORAGE_FILE_FILENAME_CONFIG, tempFile.getAbsolutePath()); props.put(StandaloneConfig.KEY_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter"); props.put(StandaloneConfig.VALUE_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter"); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreTest.java index a46f6fefcf..4504d39976 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreTest.java @@ -35,9 +35,9 @@ import org.apache.kafka.connect.util.ConnectorTaskId; import org.apache.kafka.connect.util.KafkaBasedLog; import org.apache.kafka.connect.util.TestFuture; +import org.apache.kafka.connect.util.TopicAdmin; import org.easymock.Capture; import org.easymock.EasyMock; -import org.easymock.IAnswer; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -55,8 +55,8 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -142,7 +142,7 @@ public class KafkaConfigBackingStoreTest { private Capture capturedTopic = EasyMock.newCapture(); private Capture> capturedProducerProps = EasyMock.newCapture(); private Capture> capturedConsumerProps = EasyMock.newCapture(); - private Capture> capturedAdminProps = EasyMock.newCapture(); + private Capture> capturedAdminSupplier = EasyMock.newCapture(); private Capture capturedNewTopic = EasyMock.newCapture(); private Capture>> capturedConsumedCallback = EasyMock.newCapture(); @@ -265,7 +265,7 @@ public void testPutTaskConfigs() throws Exception { expectConvertWriteRead( TASK_CONFIG_KEYS.get(1), KafkaConfigBackingStore.TASK_CONFIGURATION_V0, CONFIGS_SERIALIZED.get(1), "properties", SAMPLE_CONFIGS.get(1)); - expectReadToEnd(new LinkedHashMap()); + expectReadToEnd(new LinkedHashMap<>()); expectConvertWriteRead( COMMIT_TASKS_CONFIG_KEYS.get(0), KafkaConfigBackingStore.CONNECTOR_TASKS_COMMIT_V0, CONFIGS_SERIALIZED.get(2), "tasks", 2); // Starts with 0 tasks, after update has 2 @@ -330,7 +330,7 @@ public void testPutTaskConfigsStartsOnlyReconfiguredTasks() throws Exception { expectConvertWriteRead( TASK_CONFIG_KEYS.get(1), KafkaConfigBackingStore.TASK_CONFIGURATION_V0, CONFIGS_SERIALIZED.get(1), "properties", SAMPLE_CONFIGS.get(1)); - expectReadToEnd(new LinkedHashMap()); + expectReadToEnd(new LinkedHashMap<>()); expectConvertWriteRead( COMMIT_TASKS_CONFIG_KEYS.get(0), KafkaConfigBackingStore.CONNECTOR_TASKS_COMMIT_V0, CONFIGS_SERIALIZED.get(2), "tasks", 2); // Starts with 0 tasks, after update has 2 @@ -419,7 +419,7 @@ public void testPutTaskConfigsZeroTasks() throws Exception { COMMIT_TASKS_CONFIG_KEYS.get(0), KafkaConfigBackingStore.CONNECTOR_TASKS_COMMIT_V0, CONFIGS_SERIALIZED.get(0), "tasks", 0); // We have 0 tasks // As soon as root is rewritten, we should see a callback notifying us that we reconfigured some tasks - configUpdateListener.onTaskConfigUpdate(Collections.emptyList()); + configUpdateListener.onTaskConfigUpdate(Collections.emptyList()); EasyMock.expectLastCall(); // Records to be read by consumer as it reads to the end of the log @@ -895,7 +895,7 @@ private void expectConfigure() throws Exception { PowerMock.expectPrivate(configStorage, "createKafkaBasedLog", EasyMock.capture(capturedTopic), EasyMock.capture(capturedProducerProps), EasyMock.capture(capturedConsumerProps), EasyMock.capture(capturedConsumedCallback), - EasyMock.capture(capturedNewTopic), EasyMock.capture(capturedAdminProps)) + EasyMock.capture(capturedNewTopic), EasyMock.capture(capturedAdminSupplier)) .andReturn(storeLog); } @@ -908,13 +908,10 @@ private void expectPartitionCount(int partitionCount) { private void expectStart(final List> preexistingRecords, final Map deserializations) { storeLog.start(); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Object answer() { - for (ConsumerRecord rec : preexistingRecords) - capturedConsumedCallback.getValue().onCompletion(null, rec); - return null; - } + PowerMock.expectLastCall().andAnswer(() -> { + for (ConsumerRecord rec : preexistingRecords) + capturedConsumedCallback.getValue().onCompletion(null, rec); + return null; }); for (Map.Entry deserializationEntry : deserializations.entrySet()) { // Note null schema because default settings for internal serialization are schema-less @@ -956,29 +953,23 @@ private void expectConvertWriteRead(final String configKey, final Schema valueSc storeLog.send(EasyMock.eq(configKey), EasyMock.aryEq(serialized)); PowerMock.expectLastCall(); EasyMock.expect(converter.toConnectData(EasyMock.eq(TOPIC), EasyMock.aryEq(serialized))) - .andAnswer(new IAnswer() { - @Override - public SchemaAndValue answer() throws Throwable { - if (dataFieldName != null) - assertEquals(dataFieldValue, capturedRecord.getValue().get(dataFieldName)); - // Note null schema because default settings for internal serialization are schema-less - return new SchemaAndValue(null, serialized == null ? null : structToMap(capturedRecord.getValue())); - } + .andAnswer(() -> { + if (dataFieldName != null) + assertEquals(dataFieldValue, capturedRecord.getValue().get(dataFieldName)); + // Note null schema because default settings for internal serialization are schema-less + return new SchemaAndValue(null, serialized == null ? null : structToMap(capturedRecord.getValue())); }); } // This map needs to maintain ordering private void expectReadToEnd(final LinkedHashMap serializedConfigs) { EasyMock.expect(storeLog.readToEnd()) - .andAnswer(new IAnswer>() { - @Override - public Future answer() { - TestFuture future = new TestFuture(); - for (Map.Entry entry : serializedConfigs.entrySet()) - capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, logOffset++, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, entry.getKey(), entry.getValue())); - future.resolveOnGet((Void) null); - return future; - } + .andAnswer(() -> { + TestFuture future = new TestFuture<>(); + for (Map.Entry entry : serializedConfigs.entrySet()) + capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, logOffset++, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, entry.getKey(), entry.getValue())); + future.resolveOnGet((Void) null); + return future; }); } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java index 0d0194fc94..d21606809c 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java @@ -27,9 +27,9 @@ import org.apache.kafka.connect.util.Callback; import org.apache.kafka.connect.util.ConnectUtils; import org.apache.kafka.connect.util.KafkaBasedLog; +import org.apache.kafka.connect.util.TopicAdmin; import org.easymock.Capture; import org.easymock.EasyMock; -import org.easymock.IAnswer; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -50,6 +50,7 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Supplier; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -106,7 +107,7 @@ public class KafkaOffsetBackingStoreTest { private Capture capturedTopic = EasyMock.newCapture(); private Capture> capturedProducerProps = EasyMock.newCapture(); private Capture> capturedConsumerProps = EasyMock.newCapture(); - private Capture> capturedAdminProps = EasyMock.newCapture(); + private Capture> capturedAdminSupplier = EasyMock.newCapture(); private Capture capturedNewTopic = EasyMock.newCapture(); private Capture>> capturedConsumedCallback = EasyMock.newCapture(); @@ -194,27 +195,21 @@ public void testGetSet() throws Exception { // Second get() should get the produced data and return the new values final Capture> secondGetReadToEndCallback = EasyMock.newCapture(); storeLog.readToEnd(EasyMock.capture(secondGetReadToEndCallback)); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Object answer() throws Throwable { - capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY.array(), TP0_VALUE.array())); - capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP1_KEY.array(), TP1_VALUE.array())); - secondGetReadToEndCallback.getValue().onCompletion(null, null); - return null; - } + PowerMock.expectLastCall().andAnswer(() -> { + capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY.array(), TP0_VALUE.array())); + capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP1_KEY.array(), TP1_VALUE.array())); + secondGetReadToEndCallback.getValue().onCompletion(null, null); + return null; }); // Third get() should pick up data produced by someone else and return those values final Capture> thirdGetReadToEndCallback = EasyMock.newCapture(); storeLog.readToEnd(EasyMock.capture(thirdGetReadToEndCallback)); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Object answer() throws Throwable { - capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY.array(), TP0_VALUE_NEW.array())); - capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 1, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP1_KEY.array(), TP1_VALUE_NEW.array())); - thirdGetReadToEndCallback.getValue().onCompletion(null, null); - return null; - } + PowerMock.expectLastCall().andAnswer(() -> { + capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY.array(), TP0_VALUE_NEW.array())); + capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 1, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP1_KEY.array(), TP1_VALUE_NEW.array())); + thirdGetReadToEndCallback.getValue().onCompletion(null, null); + return null; }); expectClusterId(); @@ -234,12 +229,7 @@ public Object answer() throws Throwable { toSet.put(TP0_KEY, TP0_VALUE); toSet.put(TP1_KEY, TP1_VALUE); final AtomicBoolean invoked = new AtomicBoolean(false); - Future setFuture = store.set(toSet, new Callback() { - @Override - public void onCompletion(Throwable error, Void result) { - invoked.set(true); - } - }); + Future setFuture = store.set(toSet, (error, result) -> invoked.set(true)); assertFalse(setFuture.isDone()); // Out of order callbacks shouldn't matter, should still require all to be invoked before invoking the callback // for the store's set callback @@ -281,8 +271,8 @@ public void testGetSetNull() throws Exception { final Capture> secondGetReadToEndCallback = EasyMock.newCapture(); storeLog.readToEnd(EasyMock.capture(secondGetReadToEndCallback)); PowerMock.expectLastCall().andAnswer(() -> { - capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, (byte[]) null, TP0_VALUE.array())); - capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP1_KEY.array(), (byte[]) null)); + capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, null, TP0_VALUE.array())); + capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP1_KEY.array(), null)); secondGetReadToEndCallback.getValue().onCompletion(null, null); return null; }); @@ -300,12 +290,7 @@ public void testGetSetNull() throws Exception { toSet.put(null, TP0_VALUE); toSet.put(TP1_KEY, null); final AtomicBoolean invoked = new AtomicBoolean(false); - Future setFuture = store.set(toSet, new Callback() { - @Override - public void onCompletion(Throwable error, Void result) { - invoked.set(true); - } - }); + Future setFuture = store.set(toSet, (error, result) -> invoked.set(true)); assertFalse(setFuture.isDone()); // Out of order callbacks shouldn't matter, should still require all to be invoked before invoking the callback // for the store's set callback @@ -356,13 +341,10 @@ public void testSetFailure() throws Exception { toSet.put(TP2_KEY, TP2_VALUE); final AtomicBoolean invoked = new AtomicBoolean(false); final AtomicBoolean invokedFailure = new AtomicBoolean(false); - Future setFuture = store.set(toSet, new Callback() { - @Override - public void onCompletion(Throwable error, Void result) { - invoked.set(true); - if (error != null) - invokedFailure.set(true); - } + Future setFuture = store.set(toSet, (error, result) -> { + invoked.set(true); + if (error != null) + invokedFailure.set(true); }); assertFalse(setFuture.isDone()); // Out of order callbacks shouldn't matter, should still require all to be invoked before invoking the callback @@ -390,19 +372,16 @@ public void onCompletion(Throwable error, Void result) { private void expectConfigure() throws Exception { PowerMock.expectPrivate(store, "createKafkaBasedLog", EasyMock.capture(capturedTopic), EasyMock.capture(capturedProducerProps), EasyMock.capture(capturedConsumerProps), EasyMock.capture(capturedConsumedCallback), - EasyMock.capture(capturedNewTopic), EasyMock.capture(capturedAdminProps)) + EasyMock.capture(capturedNewTopic), EasyMock.capture(capturedAdminSupplier)) .andReturn(storeLog); } private void expectStart(final List> preexistingRecords) throws Exception { storeLog.start(); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Object answer() throws Throwable { - for (ConsumerRecord rec : preexistingRecords) - capturedConsumedCallback.getValue().onCompletion(null, rec); - return null; - } + PowerMock.expectLastCall().andAnswer(() -> { + for (ConsumerRecord rec : preexistingRecords) + capturedConsumedCallback.getValue().onCompletion(null, rec); + return null; }); } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaStatusBackingStoreTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaStatusBackingStoreTest.java index c4bb3210f0..4c0c7f90a6 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaStatusBackingStoreTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaStatusBackingStoreTest.java @@ -35,7 +35,6 @@ import org.easymock.Capture; import org.easymock.EasyMock; import org.easymock.EasyMockSupport; -import org.easymock.IAnswer; import org.easymock.Mock; import org.junit.Before; import org.junit.Test; @@ -102,12 +101,9 @@ public void putConnectorState() { final Capture callbackCapture = newCapture(); kafkaBasedLog.send(eq("status-connector-conn"), eq(value), capture(callbackCapture)); expectLastCall() - .andAnswer(new IAnswer() { - @Override - public Void answer() throws Throwable { - callbackCapture.getValue().onCompletion(null, null); - return null; - } + .andAnswer(() -> { + callbackCapture.getValue().onCompletion(null, null); + return null; }); replayAll(); @@ -129,19 +125,13 @@ public void putConnectorStateRetriableFailure() { final Capture callbackCapture = newCapture(); kafkaBasedLog.send(eq("status-connector-conn"), eq(value), capture(callbackCapture)); expectLastCall() - .andAnswer(new IAnswer() { - @Override - public Void answer() throws Throwable { - callbackCapture.getValue().onCompletion(null, new TimeoutException()); - return null; - } + .andAnswer(() -> { + callbackCapture.getValue().onCompletion(null, new TimeoutException()); + return null; }) - .andAnswer(new IAnswer() { - @Override - public Void answer() throws Throwable { - callbackCapture.getValue().onCompletion(null, null); - return null; - } + .andAnswer(() -> { + callbackCapture.getValue().onCompletion(null, null); + return null; }); replayAll(); @@ -163,12 +153,9 @@ public void putConnectorStateNonRetriableFailure() { final Capture callbackCapture = newCapture(); kafkaBasedLog.send(eq("status-connector-conn"), eq(value), capture(callbackCapture)); expectLastCall() - .andAnswer(new IAnswer() { - @Override - public Void answer() throws Throwable { - callbackCapture.getValue().onCompletion(null, new UnknownServerException()); - return null; - } + .andAnswer(() -> { + callbackCapture.getValue().onCompletion(null, new UnknownServerException()); + return null; }); replayAll(); @@ -257,13 +244,10 @@ public void putSafeOverridesValueSetBySameWorker() { final Capture callbackCapture = newCapture(); kafkaBasedLog.send(eq("status-connector-conn"), eq(value), capture(callbackCapture)); expectLastCall() - .andAnswer(new IAnswer() { - @Override - public Void answer() throws Throwable { - callbackCapture.getValue().onCompletion(null, null); - store.read(consumerRecord(1, "status-connector-conn", value)); - return null; - } + .andAnswer(() -> { + callbackCapture.getValue().onCompletion(null, null); + store.read(consumerRecord(1, "status-connector-conn", value)); + return null; }); replayAll(); @@ -303,13 +287,10 @@ public void putConnectorStateShouldOverride() { final Capture callbackCapture = newCapture(); kafkaBasedLog.send(eq("status-connector-conn"), eq(value), capture(callbackCapture)); expectLastCall() - .andAnswer(new IAnswer() { - @Override - public Void answer() throws Throwable { - callbackCapture.getValue().onCompletion(null, null); - store.read(consumerRecord(1, "status-connector-conn", value)); - return null; - } + .andAnswer(() -> { + callbackCapture.getValue().onCompletion(null, null); + store.read(consumerRecord(1, "status-connector-conn", value)); + return null; }); replayAll(); @@ -353,12 +334,9 @@ public void putTaskState() { final Capture callbackCapture = newCapture(); kafkaBasedLog.send(eq("status-task-conn-0"), eq(value), capture(callbackCapture)); expectLastCall() - .andAnswer(new IAnswer() { - @Override - public Void answer() throws Throwable { - callbackCapture.getValue().onCompletion(null, null); - return null; - } + .andAnswer(() -> { + callbackCapture.getValue().onCompletion(null, null); + return null; }); replayAll(); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/OffsetStorageWriterTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/OffsetStorageWriterTest.java index d24c64c9ae..38d7a377da 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/OffsetStorageWriterTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/OffsetStorageWriterTest.java @@ -20,7 +20,6 @@ import org.apache.kafka.connect.util.Callback; import org.easymock.Capture; import org.easymock.EasyMock; -import org.easymock.IAnswer; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -39,7 +38,6 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; -import java.util.concurrent.Callable; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThrows; @@ -245,28 +243,22 @@ private void expectStore(Map key, byte[] keySerialized, keySerialized == null ? null : ByteBuffer.wrap(keySerialized), valueSerialized == null ? null : ByteBuffer.wrap(valueSerialized)); EasyMock.expect(store.set(EasyMock.eq(offsetsSerialized), EasyMock.capture(storeCallback))) - .andAnswer(new IAnswer>() { - @Override - public Future answer() throws Throwable { - return service.submit(new Callable() { - @Override - public Void call() throws Exception { - if (waitForCompletion != null) - assertTrue(waitForCompletion.await(10000, TimeUnit.MILLISECONDS)); - - if (fail) { - storeCallback.getValue().onCompletion(exception, null); - } else { - storeCallback.getValue().onCompletion(null, null); - } - return null; - } - }); + .andAnswer(() -> + service.submit(() -> { + if (waitForCompletion != null) + assertTrue(waitForCompletion.await(10000, TimeUnit.MILLISECONDS)); + + if (fail) { + storeCallback.getValue().onCompletion(exception, null); + } else { + storeCallback.getValue().onCompletion(null, null); } - }); + return null; + }) + ); if (callback != null) { if (fail) { - callback.onCompletion(EasyMock.eq(exception), EasyMock.eq((Void) null)); + callback.onCompletion(EasyMock.eq(exception), EasyMock.eq(null)); } else { callback.onCompletion(null, null); } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/KafkaBasedLogTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/KafkaBasedLogTest.java index 080f9434d1..15bf8ca9b4 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/KafkaBasedLogTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/KafkaBasedLogTest.java @@ -119,17 +119,14 @@ public class KafkaBasedLogTest { private MockConsumer consumer; private Map>> consumedRecords = new HashMap<>(); - private Callback> consumedCallback = new Callback>() { - @Override - public void onCompletion(Throwable error, ConsumerRecord record) { - TopicPartition partition = new TopicPartition(record.topic(), record.partition()); - List> records = consumedRecords.get(partition); - if (records == null) { - records = new ArrayList<>(); - consumedRecords.put(partition, records); - } - records.add(record); + private Callback> consumedCallback = (error, record) -> { + TopicPartition partition = new TopicPartition(record.topic(), record.partition()); + List> records = consumedRecords.get(partition); + if (records == null) { + records = new ArrayList<>(); + consumedRecords.put(partition, records); } + records.add(record); }; @SuppressWarnings("unchecked") @@ -178,34 +175,21 @@ public void testReloadOnStart() throws Exception { endOffsets.put(TP1, 1L); consumer.updateEndOffsets(endOffsets); final CountDownLatch finishedLatch = new CountDownLatch(1); - consumer.schedulePollTask(new Runnable() { // Use first poll task to setup sequence of remaining responses to polls - @Override - public void run() { - // Should keep polling until it reaches current log end offset for all partitions. Should handle - // as many empty polls as needed - consumer.scheduleNopPollTask(); - consumer.scheduleNopPollTask(); - consumer.schedulePollTask(new Runnable() { - @Override - public void run() { - consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY, TP0_VALUE)); - } - }); - consumer.scheduleNopPollTask(); - consumer.scheduleNopPollTask(); - consumer.schedulePollTask(new Runnable() { - @Override - public void run() { - consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP1_KEY, TP1_VALUE)); - } - }); - consumer.schedulePollTask(new Runnable() { - @Override - public void run() { - finishedLatch.countDown(); - } - }); - } + consumer.schedulePollTask(() -> { + // Use first poll task to setup sequence of remaining responses to polls + // Should keep polling until it reaches current log end offset for all partitions. Should handle + // as many empty polls as needed + consumer.scheduleNopPollTask(); + consumer.scheduleNopPollTask(); + consumer.schedulePollTask(() -> + consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY, TP0_VALUE)) + ); + consumer.scheduleNopPollTask(); + consumer.scheduleNopPollTask(); + consumer.schedulePollTask(() -> + consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP1_KEY, TP1_VALUE)) + ); + consumer.schedulePollTask(finishedLatch::countDown); }); store.start(); assertTrue(finishedLatch.await(10000, TimeUnit.MILLISECONDS)); @@ -237,14 +221,11 @@ public void testReloadOnStartWithNoNewRecordsPresent() throws Exception { // Better test with an advanced offset other than just 0L consumer.updateBeginningOffsets(endOffsets); - consumer.schedulePollTask(new Runnable() { - @Override - public void run() { - // Throw an exception that will not be ignored or handled by Connect framework. In - // reality a misplaced call to poll blocks indefinitely and connect aborts due to - // time outs (for instance via ConnectRestException) - throw new WakeupException(); - } + consumer.schedulePollTask(() -> { + // Throw an exception that will not be ignored or handled by Connect framework. In + // reality a misplaced call to poll blocks indefinitely and connect aborts due to + // time outs (for instance via ConnectRestException) + throw new WakeupException(); }); store.start(); @@ -291,12 +272,7 @@ public void testSendAndReadToEnd() throws Exception { // Set some keys final AtomicInteger invoked = new AtomicInteger(0); - org.apache.kafka.clients.producer.Callback producerCallback = new org.apache.kafka.clients.producer.Callback() { - @Override - public void onCompletion(RecordMetadata metadata, Exception exception) { - invoked.incrementAndGet(); - } - }; + org.apache.kafka.clients.producer.Callback producerCallback = (metadata, exception) -> invoked.incrementAndGet(); store.send(TP0_KEY, TP0_VALUE, producerCallback); store.send(TP1_KEY, TP1_VALUE, producerCallback); assertEquals(0, invoked.get()); @@ -309,46 +285,31 @@ public void onCompletion(RecordMetadata metadata, Exception exception) { // Now we should have to wait for the records to be read back when we call readToEnd() final AtomicBoolean getInvoked = new AtomicBoolean(false); - final FutureCallback readEndFutureCallback = new FutureCallback<>(new Callback() { - @Override - public void onCompletion(Throwable error, Void result) { - getInvoked.set(true); - } - }); - consumer.schedulePollTask(new Runnable() { - @Override - public void run() { - // Once we're synchronized in a poll, start the read to end and schedule the exact set of poll events - // that should follow. This readToEnd call will immediately wakeup this consumer.poll() call without - // returning any data. - Map newEndOffsets = new HashMap<>(); - newEndOffsets.put(TP0, 2L); - newEndOffsets.put(TP1, 2L); - consumer.updateEndOffsets(newEndOffsets); - store.readToEnd(readEndFutureCallback); - - // Should keep polling until it reaches current log end offset for all partitions - consumer.scheduleNopPollTask(); - consumer.scheduleNopPollTask(); - consumer.scheduleNopPollTask(); - consumer.schedulePollTask(new Runnable() { - @Override - public void run() { - consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY, TP0_VALUE)); - consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY, TP0_VALUE_NEW)); - consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP1_KEY, TP1_VALUE)); - } - }); - - consumer.schedulePollTask(new Runnable() { - @Override - public void run() { - consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP1_KEY, TP1_VALUE_NEW)); - } - }); - - // Already have FutureCallback that should be invoked/awaited, so no need for follow up finishedLatch - } + final FutureCallback readEndFutureCallback = new FutureCallback<>((error, result) -> getInvoked.set(true)); + consumer.schedulePollTask(() -> { + // Once we're synchronized in a poll, start the read to end and schedule the exact set of poll events + // that should follow. This readToEnd call will immediately wakeup this consumer.poll() call without + // returning any data. + Map newEndOffsets = new HashMap<>(); + newEndOffsets.put(TP0, 2L); + newEndOffsets.put(TP1, 2L); + consumer.updateEndOffsets(newEndOffsets); + store.readToEnd(readEndFutureCallback); + + // Should keep polling until it reaches current log end offset for all partitions + consumer.scheduleNopPollTask(); + consumer.scheduleNopPollTask(); + consumer.scheduleNopPollTask(); + consumer.schedulePollTask(() -> { + consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY, TP0_VALUE)); + consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY, TP0_VALUE_NEW)); + consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP1_KEY, TP1_VALUE)); + }); + + consumer.schedulePollTask(() -> + consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP1_KEY, TP1_VALUE_NEW))); + + // Already have FutureCallback that should be invoked/awaited, so no need for follow up finishedLatch }); readEndFutureCallback.get(10000, TimeUnit.MILLISECONDS); assertTrue(getInvoked.get()); @@ -382,35 +343,20 @@ public void testPollConsumerError() throws Exception { endOffsets.put(TP0, 1L); endOffsets.put(TP1, 1L); consumer.updateEndOffsets(endOffsets); - consumer.schedulePollTask(new Runnable() { - @Override - public void run() { - // Trigger exception - consumer.schedulePollTask(new Runnable() { - @Override - public void run() { - consumer.setPollException(Errors.COORDINATOR_NOT_AVAILABLE.exception()); - } - }); - - // Should keep polling until it reaches current log end offset for all partitions - consumer.scheduleNopPollTask(); - consumer.scheduleNopPollTask(); - consumer.schedulePollTask(new Runnable() { - @Override - public void run() { - consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY, TP0_VALUE_NEW)); - consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY, TP0_VALUE_NEW)); - } - }); - - consumer.schedulePollTask(new Runnable() { - @Override - public void run() { - finishedLatch.countDown(); - } - }); - } + consumer.schedulePollTask(() -> { + // Trigger exception + consumer.schedulePollTask(() -> + consumer.setPollException(Errors.COORDINATOR_NOT_AVAILABLE.exception())); + + // Should keep polling until it reaches current log end offset for all partitions + consumer.scheduleNopPollTask(); + consumer.scheduleNopPollTask(); + consumer.schedulePollTask(() -> { + consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY, TP0_VALUE_NEW)); + consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY, TP0_VALUE_NEW)); + }); + + consumer.schedulePollTask(finishedLatch::countDown); }); store.start(); assertTrue(finishedLatch.await(10000, TimeUnit.MILLISECONDS)); @@ -442,45 +388,29 @@ public void testGetOffsetsConsumerErrorOnReadToEnd() throws Exception { consumer.updateEndOffsets(endOffsets); store.start(); final AtomicBoolean getInvoked = new AtomicBoolean(false); - final FutureCallback readEndFutureCallback = new FutureCallback<>(new Callback() { - @Override - public void onCompletion(Throwable error, Void result) { - getInvoked.set(true); - } - }); - consumer.schedulePollTask(new Runnable() { - @Override - public void run() { - // Once we're synchronized in a poll, start the read to end and schedule the exact set of poll events - // that should follow. This readToEnd call will immediately wakeup this consumer.poll() call without - // returning any data. - Map newEndOffsets = new HashMap<>(); - newEndOffsets.put(TP0, 1L); - newEndOffsets.put(TP1, 1L); - consumer.updateEndOffsets(newEndOffsets); - // Set exception to occur when getting offsets to read log to end. It'll be caught in the work thread, - // which will retry and eventually get the correct offsets and read log to end. - consumer.setOffsetsException(new TimeoutException("Failed to get offsets by times")); - store.readToEnd(readEndFutureCallback); - - // Should keep polling until it reaches current log end offset for all partitions - consumer.scheduleNopPollTask(); - consumer.scheduleNopPollTask(); - consumer.schedulePollTask(new Runnable() { - @Override - public void run() { - consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY, TP0_VALUE)); - consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY, TP0_VALUE_NEW)); - } - }); - - consumer.schedulePollTask(new Runnable() { - @Override - public void run() { - finishedLatch.countDown(); - } - }); - } + final FutureCallback readEndFutureCallback = new FutureCallback<>((error, result) -> getInvoked.set(true)); + consumer.schedulePollTask(() -> { + // Once we're synchronized in a poll, start the read to end and schedule the exact set of poll events + // that should follow. This readToEnd call will immediately wakeup this consumer.poll() call without + // returning any data. + Map newEndOffsets = new HashMap<>(); + newEndOffsets.put(TP0, 1L); + newEndOffsets.put(TP1, 1L); + consumer.updateEndOffsets(newEndOffsets); + // Set exception to occur when getting offsets to read log to end. It'll be caught in the work thread, + // which will retry and eventually get the correct offsets and read log to end. + consumer.setOffsetsException(new TimeoutException("Failed to get offsets by times")); + store.readToEnd(readEndFutureCallback); + + // Should keep polling until it reaches current log end offset for all partitions + consumer.scheduleNopPollTask(); + consumer.scheduleNopPollTask(); + consumer.schedulePollTask(() -> { + consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY, TP0_VALUE)); + consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY, TP0_VALUE_NEW)); + }); + + consumer.schedulePollTask(finishedLatch::countDown); }); readEndFutureCallback.get(10000, TimeUnit.MILLISECONDS); assertTrue(getInvoked.get()); @@ -517,12 +447,9 @@ public void testProducerError() throws Exception { assertEquals(0L, consumer.position(TP1)); final AtomicReference setException = new AtomicReference<>(); - store.send(TP0_KEY, TP0_VALUE, new org.apache.kafka.clients.producer.Callback() { - @Override - public void onCompletion(RecordMetadata metadata, Exception exception) { - assertNull(setException.get()); // Should only be invoked once - setException.set(exception); - } + store.send(TP0_KEY, TP0_VALUE, (metadata, exception) -> { + assertNull(setException.get()); // Should only be invoked once + setException.set(exception); }); KafkaException exc = new LeaderNotAvailableException("Error"); tp0Future.resolve(exc); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/SharedTopicAdminTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/SharedTopicAdminTest.java new file mode 100644 index 0000000000..f5ac6a730f --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/SharedTopicAdminTest.java @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.util; + +import java.time.Duration; +import java.util.Collections; +import java.util.Map; +import java.util.function.Function; + +import org.apache.kafka.connect.errors.ConnectException; +import org.junit.Rule; +import org.mockito.Mock; +import org.junit.Before; +import org.junit.Test; +import org.mockito.junit.MockitoJUnit; +import org.mockito.junit.MockitoRule; + +import static org.apache.kafka.connect.util.SharedTopicAdmin.DEFAULT_CLOSE_DURATION; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertThrows; +import static org.mockito.ArgumentMatchers.anyMap; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class SharedTopicAdminTest { + + private static final Map EMPTY_CONFIG = Collections.emptyMap(); + + @Rule + public MockitoRule rule = MockitoJUnit.rule(); + + @Mock private TopicAdmin mockTopicAdmin; + @Mock private Function, TopicAdmin> factory; + private SharedTopicAdmin sharedAdmin; + + @Before + public void beforeEach() { + when(factory.apply(anyMap())).thenReturn(mockTopicAdmin); + sharedAdmin = new SharedTopicAdmin(EMPTY_CONFIG, factory::apply); + } + + @Test + public void shouldCloseWithoutBeingUsed() { + // When closed before being used + sharedAdmin.close(); + // Then should not create or close admin + verifyTopicAdminCreatesAndCloses(0); + } + + @Test + public void shouldCloseAfterTopicAdminUsed() { + // When used and then closed + assertSame(mockTopicAdmin, sharedAdmin.topicAdmin()); + sharedAdmin.close(); + // Then should have created and closed just one admin + verifyTopicAdminCreatesAndCloses(1); + } + + @Test + public void shouldCloseAfterTopicAdminUsedMultipleTimes() { + // When used many times and then closed + for (int i = 0; i != 10; ++i) { + assertSame(mockTopicAdmin, sharedAdmin.topicAdmin()); + } + sharedAdmin.close(); + // Then should have created and closed just one admin + verifyTopicAdminCreatesAndCloses(1); + } + + @Test + public void shouldCloseWithDurationAfterTopicAdminUsed() { + // When used and then closed with a custom timeout + Duration timeout = Duration.ofSeconds(1); + assertSame(mockTopicAdmin, sharedAdmin.topicAdmin()); + sharedAdmin.close(timeout); + // Then should have created and closed just one admin using the supplied timeout + verifyTopicAdminCreatesAndCloses(1, timeout); + } + + @Test + public void shouldFailToGetTopicAdminAfterClose() { + // When closed + sharedAdmin.close(); + // Then using the admin should fail + assertThrows(ConnectException.class, () -> sharedAdmin.topicAdmin()); + } + + private void verifyTopicAdminCreatesAndCloses(int count) { + verifyTopicAdminCreatesAndCloses(count, DEFAULT_CLOSE_DURATION); + } + + private void verifyTopicAdminCreatesAndCloses(int count, Duration expectedDuration) { + verify(factory, times(count)).apply(anyMap()); + verify(mockTopicAdmin, times(count)).close(eq(expectedDuration)); + } +} \ No newline at end of file diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/TopicAdminTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/TopicAdminTest.java index fa18b44fb8..9ba0b1d0aa 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/TopicAdminTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/TopicAdminTest.java @@ -27,6 +27,7 @@ import org.apache.kafka.common.KafkaFuture; import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartitionInfo; import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.config.ConfigResource; @@ -37,19 +38,26 @@ import org.apache.kafka.common.message.CreateTopicsResponseData; import org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult; import org.apache.kafka.common.message.DescribeConfigsResponseData; +import org.apache.kafka.common.message.ListOffsetsResponseData; +import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse; import org.apache.kafka.common.message.MetadataResponseData; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ApiError; import org.apache.kafka.common.requests.CreateTopicsResponse; import org.apache.kafka.common.requests.DescribeConfigsResponse; +import org.apache.kafka.common.requests.ListOffsetsResponse; import org.apache.kafka.common.requests.MetadataResponse; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.errors.RetriableException; import org.junit.Test; +import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -160,7 +168,7 @@ public void shouldNotCreateTopicWhenItAlreadyExists() { NewTopic newTopic = TopicAdmin.defineTopic("myTopic").partitions(1).compacted().build(); Cluster cluster = createCluster(1); try (MockAdminClient mockAdminClient = new MockAdminClient(cluster.nodes(), cluster.nodeById(0))) { - TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), Collections.emptyList()); + TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), Collections.emptyList()); mockAdminClient.addTopic(false, "myTopic", Collections.singletonList(topicPartitionInfo), null); TopicAdmin admin = new TopicAdmin(null, mockAdminClient); assertFalse(admin.createTopic(newTopic)); @@ -272,7 +280,7 @@ public void describeShouldReturnTopicDescriptionWhenTopicExists() { NewTopic newTopic = TopicAdmin.defineTopic(topicName).partitions(1).compacted().build(); Cluster cluster = createCluster(1); try (MockAdminClient mockAdminClient = new MockAdminClient(cluster.nodes(), cluster.nodeById(0))) { - TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), Collections.emptyList()); + TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), Collections.emptyList()); mockAdminClient.addTopic(false, topicName, Collections.singletonList(topicPartitionInfo), null); TopicAdmin admin = new TopicAdmin(null, mockAdminClient); Map desc = admin.describeTopics(newTopic.name()); @@ -352,7 +360,7 @@ public void describeTopicConfigShouldReturnTopicConfigWhenTopicExists() { .build(); Cluster cluster = createCluster(1); try (MockAdminClient mockAdminClient = new MockAdminClient(cluster.nodes(), cluster.nodeById(0))) { - TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), Collections.emptyList()); + TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), Collections.emptyList()); mockAdminClient.addTopic(false, topicName, Collections.singletonList(topicPartitionInfo), null); TopicAdmin admin = new TopicAdmin(null, mockAdminClient); Map result = admin.describeTopicConfigs(newTopic.name()); @@ -360,9 +368,7 @@ public void describeTopicConfigShouldReturnTopicConfigWhenTopicExists() { assertEquals(1, result.size()); Config config = result.get("myTopic"); assertNotNull(config); - config.entries().forEach(entry -> { - assertEquals(newTopic.configs().get(entry.name()), entry.value()); - }); + config.entries().forEach(entry -> assertEquals(newTopic.configs().get(entry.name()), entry.value())); } } @@ -408,7 +414,7 @@ public void verifyingTopicCleanupPolicyShouldReturnTrueWhenTopicHasCorrectPolicy Map topicConfigs = Collections.singletonMap("cleanup.policy", "compact"); Cluster cluster = createCluster(1); try (MockAdminClient mockAdminClient = new MockAdminClient(cluster.nodes(), cluster.nodeById(0))) { - TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), Collections.emptyList()); + TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), Collections.emptyList()); mockAdminClient.addTopic(false, topicName, Collections.singletonList(topicPartitionInfo), topicConfigs); TopicAdmin admin = new TopicAdmin(null, mockAdminClient); boolean result = admin.verifyTopicCleanupPolicyOnlyCompact("myTopic", "worker.topic", "purpose"); @@ -422,12 +428,10 @@ public void verifyingTopicCleanupPolicyShouldFailWhenTopicHasDeletePolicy() { Map topicConfigs = Collections.singletonMap("cleanup.policy", "delete"); Cluster cluster = createCluster(1); try (MockAdminClient mockAdminClient = new MockAdminClient(cluster.nodes(), cluster.nodeById(0))) { - TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), Collections.emptyList()); + TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), Collections.emptyList()); mockAdminClient.addTopic(false, topicName, Collections.singletonList(topicPartitionInfo), topicConfigs); TopicAdmin admin = new TopicAdmin(null, mockAdminClient); - ConfigException e = assertThrows(ConfigException.class, () -> { - admin.verifyTopicCleanupPolicyOnlyCompact("myTopic", "worker.topic", "purpose"); - }); + ConfigException e = assertThrows(ConfigException.class, () -> admin.verifyTopicCleanupPolicyOnlyCompact("myTopic", "worker.topic", "purpose")); assertTrue(e.getMessage().contains("to guarantee consistency and durability")); } } @@ -438,12 +442,10 @@ public void verifyingTopicCleanupPolicyShouldFailWhenTopicHasDeleteAndCompactPol Map topicConfigs = Collections.singletonMap("cleanup.policy", "delete,compact"); Cluster cluster = createCluster(1); try (MockAdminClient mockAdminClient = new MockAdminClient(cluster.nodes(), cluster.nodeById(0))) { - TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), Collections.emptyList()); + TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), Collections.emptyList()); mockAdminClient.addTopic(false, topicName, Collections.singletonList(topicPartitionInfo), topicConfigs); TopicAdmin admin = new TopicAdmin(null, mockAdminClient); - ConfigException e = assertThrows(ConfigException.class, () -> { - admin.verifyTopicCleanupPolicyOnlyCompact("myTopic", "worker.topic", "purpose"); - }); + ConfigException e = assertThrows(ConfigException.class, () -> admin.verifyTopicCleanupPolicyOnlyCompact("myTopic", "worker.topic", "purpose")); assertTrue(e.getMessage().contains("to guarantee consistency and durability")); } } @@ -454,7 +456,7 @@ public void verifyingGettingTopicCleanupPolicies() { Map topicConfigs = Collections.singletonMap("cleanup.policy", "compact"); Cluster cluster = createCluster(1); try (MockAdminClient mockAdminClient = new MockAdminClient(cluster.nodes(), cluster.nodeById(0))) { - TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), Collections.emptyList()); + TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), Collections.emptyList()); mockAdminClient.addTopic(false, topicName, Collections.singletonList(topicPartitionInfo), topicConfigs); TopicAdmin admin = new TopicAdmin(null, mockAdminClient); Set policies = admin.topicCleanupPolicy("myTopic"); @@ -463,17 +465,273 @@ public void verifyingGettingTopicCleanupPolicies() { } } + @Test + public void endOffsetsShouldFailWithNonRetriableWhenAuthorizationFailureOccurs() { + String topicName = "myTopic"; + TopicPartition tp1 = new TopicPartition(topicName, 0); + Set tps = Collections.singleton(tp1); + Long offset = null; // response should use error + Cluster cluster = createCluster(1, topicName, 1); + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(new MockTime(), cluster)) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + env.kafkaClient().prepareResponse(prepareMetadataResponse(cluster, Errors.NONE)); + env.kafkaClient().prepareResponse(listOffsetsResultWithClusterAuthorizationException(tp1, offset)); + TopicAdmin admin = new TopicAdmin(null, env.adminClient()); + ConnectException e = assertThrows(ConnectException.class, () -> { + admin.endOffsets(tps); + }); + assertTrue(e.getMessage().contains("Not authorized to get the end offsets")); + } + } + + @Test + public void endOffsetsShouldFailWithNonRetriableWhenVersionUnsupportedErrorOccurs() { + String topicName = "myTopic"; + TopicPartition tp1 = new TopicPartition(topicName, 0); + Set tps = Collections.singleton(tp1); + Long offset = null; // response should use error + Cluster cluster = createCluster(1, topicName, 1); + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(new MockTime(), cluster)) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + env.kafkaClient().prepareResponse(prepareMetadataResponse(cluster, Errors.NONE)); + env.kafkaClient().prepareResponse(listOffsetsResultWithUnsupportedVersion(tp1, offset)); + TopicAdmin admin = new TopicAdmin(null, env.adminClient()); + ConnectException e = assertThrows(ConnectException.class, () -> { + admin.endOffsets(tps); + }); + assertTrue(e.getMessage().contains("is unsupported on brokers")); + } + } + + @Test + public void endOffsetsShouldFailWithRetriableWhenTimeoutErrorOccurs() { + String topicName = "myTopic"; + TopicPartition tp1 = new TopicPartition(topicName, 0); + Set tps = Collections.singleton(tp1); + Long offset = null; // response should use error + Cluster cluster = createCluster(1, topicName, 1); + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(new MockTime(), cluster)) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + env.kafkaClient().prepareResponse(prepareMetadataResponse(cluster, Errors.NONE)); + env.kafkaClient().prepareResponse(listOffsetsResultWithTimeout(tp1, offset)); + TopicAdmin admin = new TopicAdmin(null, env.adminClient()); + RetriableException e = assertThrows(RetriableException.class, () -> { + admin.endOffsets(tps); + }); + assertTrue(e.getMessage().contains("Timed out while waiting")); + } + } + + @Test + public void endOffsetsShouldFailWithNonRetriableWhenUnknownErrorOccurs() { + String topicName = "myTopic"; + TopicPartition tp1 = new TopicPartition(topicName, 0); + Set tps = Collections.singleton(tp1); + Long offset = null; // response should use error + Cluster cluster = createCluster(1, topicName, 1); + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(new MockTime(), cluster)) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + env.kafkaClient().prepareResponse(prepareMetadataResponse(cluster, Errors.NONE)); + env.kafkaClient().prepareResponse(listOffsetsResultWithUnknownError(tp1, offset)); + TopicAdmin admin = new TopicAdmin(null, env.adminClient()); + ConnectException e = assertThrows(ConnectException.class, () -> { + admin.endOffsets(tps); + }); + assertTrue(e.getMessage().contains("Error while getting end offsets for topic")); + } + } + + @Test + public void endOffsetsShouldReturnEmptyMapWhenPartitionsSetIsNull() { + String topicName = "myTopic"; + Cluster cluster = createCluster(1, topicName, 1); + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(new MockTime(), cluster)) { + TopicAdmin admin = new TopicAdmin(null, env.adminClient()); + Map offsets = admin.endOffsets(Collections.emptySet()); + assertTrue(offsets.isEmpty()); + } + } + + @Test + public void endOffsetsShouldReturnOffsetsForOnePartition() { + String topicName = "myTopic"; + TopicPartition tp1 = new TopicPartition(topicName, 0); + Set tps = Collections.singleton(tp1); + long offset = 1000L; + Cluster cluster = createCluster(1, topicName, 1); + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(new MockTime(), cluster)) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + env.kafkaClient().prepareResponse(prepareMetadataResponse(cluster, Errors.NONE)); + env.kafkaClient().prepareResponse(listOffsetsResult(tp1, offset)); + TopicAdmin admin = new TopicAdmin(null, env.adminClient()); + Map offsets = admin.endOffsets(tps); + assertEquals(1, offsets.size()); + assertEquals(Long.valueOf(offset), offsets.get(tp1)); + } + } + + @Test + public void endOffsetsShouldReturnOffsetsForMultiplePartitions() { + String topicName = "myTopic"; + TopicPartition tp1 = new TopicPartition(topicName, 0); + TopicPartition tp2 = new TopicPartition(topicName, 1); + Set tps = new HashSet<>(Arrays.asList(tp1, tp2)); + long offset1 = 1001; + long offset2 = 1002; + Cluster cluster = createCluster(1, topicName, 2); + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(new MockTime(), cluster)) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + env.kafkaClient().prepareResponse(prepareMetadataResponse(cluster, Errors.NONE)); + env.kafkaClient().prepareResponse(listOffsetsResult(tp1, offset1, tp2, offset2)); + TopicAdmin admin = new TopicAdmin(null, env.adminClient()); + Map offsets = admin.endOffsets(tps); + assertEquals(2, offsets.size()); + assertEquals(Long.valueOf(offset1), offsets.get(tp1)); + assertEquals(Long.valueOf(offset2), offsets.get(tp2)); + } + } + + @Test + public void endOffsetsShouldFailWhenAnyTopicPartitionHasError() { + String topicName = "myTopic"; + TopicPartition tp1 = new TopicPartition(topicName, 0); + Set tps = Collections.singleton(tp1); + long offset = 1000; + Cluster cluster = createCluster(1, topicName, 1); + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(new MockTime(), cluster)) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + env.kafkaClient().prepareResponse(prepareMetadataResponse(cluster, Errors.NONE)); + env.kafkaClient().prepareResponse(listOffsetsResultWithClusterAuthorizationException(tp1, null)); + TopicAdmin admin = new TopicAdmin(null, env.adminClient()); + ConnectException e = assertThrows(ConnectException.class, () -> { + admin.endOffsets(tps); + }); + assertTrue(e.getMessage().contains("Not authorized to get the end offsets")); + } + } + private Cluster createCluster(int numNodes) { + return createCluster(numNodes, "unused", 0); + } + + private Cluster createCluster(int numNodes, String topicName, int partitions) { + Node[] nodeArray = new Node[numNodes]; HashMap nodes = new HashMap<>(); for (int i = 0; i < numNodes; ++i) { - nodes.put(i, new Node(i, "localhost", 8121 + i)); - } - Cluster cluster = new Cluster("mockClusterId", nodes.values(), - Collections.emptySet(), Collections.emptySet(), - Collections.emptySet(), nodes.get(0)); + nodeArray[i] = new Node(i, "localhost", 8121 + i); + nodes.put(i, nodeArray[i]); + } + Node leader = nodeArray[0]; + List pInfos = new ArrayList<>(); + for (int i = 0; i < partitions; ++i) { + pInfos.add(new PartitionInfo(topicName, i, leader, nodeArray, nodeArray)); + } + Cluster cluster = new Cluster( + "mockClusterId", + nodes.values(), + pInfos, + Collections.emptySet(), + Collections.emptySet(), + leader); return cluster; } + private MetadataResponse prepareMetadataResponse(Cluster cluster, Errors error) { + List metadata = new ArrayList<>(); + for (String topic : cluster.topics()) { + List pms = new ArrayList<>(); + for (PartitionInfo pInfo : cluster.availablePartitionsForTopic(topic)) { + MetadataResponseData.MetadataResponsePartition pm = new MetadataResponseData.MetadataResponsePartition() + .setErrorCode(error.code()) + .setPartitionIndex(pInfo.partition()) + .setLeaderId(pInfo.leader().id()) + .setLeaderEpoch(234) + .setReplicaNodes(Arrays.stream(pInfo.replicas()).map(Node::id).collect(Collectors.toList())) + .setIsrNodes(Arrays.stream(pInfo.inSyncReplicas()).map(Node::id).collect(Collectors.toList())) + .setOfflineReplicas(Arrays.stream(pInfo.offlineReplicas()).map(Node::id).collect(Collectors.toList())); + pms.add(pm); + } + MetadataResponseTopic tm = new MetadataResponseTopic() + .setErrorCode(error.code()) + .setName(topic) + .setIsInternal(false) + .setPartitions(pms); + metadata.add(tm); + } + return MetadataResponse.prepareResponse(true, + 0, + cluster.nodes(), + cluster.clusterResource().clusterId(), + cluster.controller().id(), + metadata, + MetadataResponse.AUTHORIZED_OPERATIONS_OMITTED); + } + + private ListOffsetsResponse listOffsetsResultWithUnknownError(TopicPartition tp1, Long offset1) { + return listOffsetsResult( + new ApiError(Errors.UNKNOWN_SERVER_ERROR, "Unknown error"), + Collections.singletonMap(tp1, offset1) + ); + } + + private ListOffsetsResponse listOffsetsResultWithTimeout(TopicPartition tp1, Long offset1) { + return listOffsetsResult( + new ApiError(Errors.REQUEST_TIMED_OUT, "Request timed out"), + Collections.singletonMap(tp1, offset1) + ); + } + + private ListOffsetsResponse listOffsetsResultWithUnsupportedVersion(TopicPartition tp1, Long offset1) { + return listOffsetsResult( + new ApiError(Errors.UNSUPPORTED_VERSION, "This version of the API is not supported"), + Collections.singletonMap(tp1, offset1) + ); + } + + private ListOffsetsResponse listOffsetsResultWithClusterAuthorizationException(TopicPartition tp1, Long offset1) { + return listOffsetsResult( + new ApiError(Errors.CLUSTER_AUTHORIZATION_FAILED, "Not authorized to create topic(s)"), + Collections.singletonMap(tp1, offset1) + ); + } + + private ListOffsetsResponse listOffsetsResult(TopicPartition tp1, Long offset1) { + return listOffsetsResult(null, Collections.singletonMap(tp1, offset1)); + } + + private ListOffsetsResponse listOffsetsResult(TopicPartition tp1, Long offset1, TopicPartition tp2, Long offset2) { + Map offsetsByPartitions = new HashMap<>(); + offsetsByPartitions.put(tp1, offset1); + offsetsByPartitions.put(tp2, offset2); + return listOffsetsResult(null, offsetsByPartitions); + } + + /** + * Create a ListOffsetResponse that exposes the supplied error and includes offsets for the supplied partitions. + * @param error the error; may be null if an unknown error should be used + * @param offsetsByPartitions offset for each partition, where offset is null signals the error should be used + * @return the response + */ + private ListOffsetsResponse listOffsetsResult(ApiError error, Map offsetsByPartitions) { + if (error == null) error = new ApiError(Errors.UNKNOWN_TOPIC_OR_PARTITION, "unknown topic"); + List tpResponses = new ArrayList<>(); + for (TopicPartition partition : offsetsByPartitions.keySet()) { + Long offset = offsetsByPartitions.get(partition); + ListOffsetsTopicResponse topicResponse; + if (offset == null) { + topicResponse = ListOffsetsResponse.singletonListOffsetsTopicResponse(partition, error.error(), -1L, 0, 321); + } else { + topicResponse = ListOffsetsResponse.singletonListOffsetsTopicResponse(partition, Errors.NONE, -1L, offset, 321); + } + tpResponses.add(topicResponse); + } + ListOffsetsResponseData responseData = new ListOffsetsResponseData() + .setThrottleTimeMs(0) + .setTopics(tpResponses); + + return new ListOffsetsResponse(responseData); + } + private CreateTopicsResponse createTopicResponseWithUnsupportedVersion(NewTopic... topics) { return createTopicResponse(new ApiError(Errors.UNSUPPORTED_VERSION, "This version of the API is not supported"), topics); } @@ -512,7 +770,7 @@ protected void assertTopicCreation( clientBuilder.defaultPartitions(defaultPartitions.shortValue()); } if (defaultReplicationFactor != null) { - clientBuilder.defaultReplicationFactor(defaultReplicationFactor.intValue()); + clientBuilder.defaultReplicationFactor(defaultReplicationFactor); } clientBuilder.brokers(cluster.nodes()); clientBuilder.controller(0); diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampRouter.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampRouter.java index 388a7b941f..f7b1e58248 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampRouter.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampRouter.java @@ -61,14 +61,11 @@ public void configure(Map props) { topicFormat = config.getString(ConfigName.TOPIC_FORMAT); final String timestampFormatStr = config.getString(ConfigName.TIMESTAMP_FORMAT); - timestampFormat = new ThreadLocal() { - @Override - protected SimpleDateFormat initialValue() { - final SimpleDateFormat fmt = new SimpleDateFormat(timestampFormatStr); - fmt.setTimeZone(TimeZone.getTimeZone("UTC")); - return fmt; - } - }; + timestampFormat = ThreadLocal.withInitial(() -> { + final SimpleDateFormat fmt = new SimpleDateFormat(timestampFormatStr); + fmt.setTimeZone(TimeZone.getTimeZone("UTC")); + return fmt; + }); } @Override diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/FlattenTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/FlattenTest.java index 0867283eb7..541ca142ab 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/FlattenTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/FlattenTest.java @@ -135,8 +135,8 @@ public void testNestedMapWithDelimiter() { supportedTypes.put("string", "stringy"); supportedTypes.put("bytes", "bytes".getBytes()); - Map oneLevelNestedMap = Collections.singletonMap("B", (Object) supportedTypes); - Map twoLevelNestedMap = Collections.singletonMap("A", (Object) oneLevelNestedMap); + Map oneLevelNestedMap = Collections.singletonMap("B", supportedTypes); + Map twoLevelNestedMap = Collections.singletonMap("A", oneLevelNestedMap); SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, "topic", 0, @@ -232,7 +232,7 @@ public void testOptionalFieldMap() { Map supportedTypes = new HashMap<>(); supportedTypes.put("opt_int32", null); - Map oneLevelNestedMap = Collections.singletonMap("B", (Object) supportedTypes); + Map oneLevelNestedMap = Collections.singletonMap("B", supportedTypes); SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, "topic", 0, diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/SetSchemaMetadataTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/SetSchemaMetadataTest.java index a53996dbd7..04a35ca4c6 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/SetSchemaMetadataTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/SetSchemaMetadataTest.java @@ -126,7 +126,7 @@ public void updateSchemaOfStruct() { @Test public void updateSchemaOfNonStruct() { - Object value = Integer.valueOf(1); + Object value = 1; Object updatedValue = SetSchemaMetadata.updateSchemaIn(value, Schema.INT32_SCHEMA); assertSame(value, updatedValue); } diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/TimestampRouterTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/TimestampRouterTest.java index f953c3e10a..5fa87ba314 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/TimestampRouterTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/TimestampRouterTest.java @@ -34,7 +34,7 @@ public void teardown() { @Test public void defaultConfiguration() { - xform.configure(Collections.emptyMap()); // defaults + xform.configure(Collections.emptyMap()); // defaults final SourceRecord record = new SourceRecord( null, null, "test", 0, diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/predicates/HasHeaderKeyTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/predicates/HasHeaderKeyTest.java index 5e4da414cb..39635f31b7 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/predicates/HasHeaderKeyTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/predicates/HasHeaderKeyTest.java @@ -84,7 +84,7 @@ private SimpleConfig config(Map props) { private SourceRecord recordWithHeaders(String... headers) { return new SourceRecord(null, null, null, null, null, null, null, null, null, - Arrays.stream(headers).map(header -> new TestHeader(header)).collect(Collectors.toList())); + Arrays.stream(headers).map(TestHeader::new).collect(Collectors.toList())); } private static class TestHeader implements Header { diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index cf61b1e42c..6f6cb88e92 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -17,7 +17,7 @@ package kafka.cluster import java.util.concurrent.locks.ReentrantReadWriteLock -import java.util.{Optional, Properties} +import java.util.Optional import kafka.api.{ApiVersion, LeaderAndIsr} import kafka.common.UnexpectedAppendOffsetException @@ -26,9 +26,9 @@ import kafka.log._ import kafka.metrics.KafkaMetricsGroup import kafka.server._ import kafka.server.checkpoints.OffsetCheckpoints +import kafka.server.metadata.ConfigRepository import kafka.utils.CoreUtils.{inReadLock, inWriteLock} import kafka.utils._ -import kafka.zk.AdminZkClient import kafka.zookeeper.ZooKeeperClientException import org.apache.kafka.common.errors._ import org.apache.kafka.common.message.{DescribeProducersResponseData, FetchResponseData} @@ -51,10 +51,6 @@ trait IsrChangeListener { def markFailed(): Unit } -trait TopicConfigFetcher { - def fetch(): Properties -} - class DelayedOperations(topicPartition: TopicPartition, produce: DelayedOperationPurgatory[DelayedProduce], fetch: DelayedOperationPurgatory[DelayedFetch], @@ -73,6 +69,7 @@ class DelayedOperations(topicPartition: TopicPartition, object Partition extends KafkaMetricsGroup { def apply(topicPartition: TopicPartition, time: Time, + configRepository: ConfigRepository, replicaManager: ReplicaManager): Partition = { val isrChangeListener = new IsrChangeListener { @@ -87,13 +84,6 @@ object Partition extends KafkaMetricsGroup { override def markFailed(): Unit = replicaManager.failedIsrUpdatesRate.mark() } - val configProvider = new TopicConfigFetcher { - override def fetch(): Properties = { - val adminZkClient = new AdminZkClient(replicaManager.zkClient) - adminZkClient.fetchEntityConfig(ConfigType.Topic, topicPartition.topic) - } - } - val delayedOperations = new DelayedOperations( topicPartition, replicaManager.delayedProducePurgatory, @@ -105,7 +95,6 @@ object Partition extends KafkaMetricsGroup { interBrokerProtocolVersion = replicaManager.config.interBrokerProtocolVersion, localBrokerId = replicaManager.config.brokerId, time = time, - topicConfigProvider = configProvider, isrChangeListener = isrChangeListener, delayedOperations = delayedOperations, metadataCache = replicaManager.metadataCache, @@ -228,7 +217,6 @@ class Partition(val topicPartition: TopicPartition, interBrokerProtocolVersion: ApiVersion, localBrokerId: Int, time: Time, - topicConfigProvider: TopicConfigFetcher, isrChangeListener: IsrChangeListener, delayedOperations: DelayedOperations, metadataCache: MetadataCache, @@ -341,11 +329,6 @@ class Partition(val topicPartition: TopicPartition, // Visible for testing private[cluster] def createLog(isNew: Boolean, isFutureReplica: Boolean, offsetCheckpoints: OffsetCheckpoints): Log = { - def fetchLogConfig: LogConfig = { - val props = topicConfigProvider.fetch() - LogConfig.fromProps(logManager.currentDefaultConfig.originals, props) - } - def updateHighWatermark(log: Log) = { val checkpointHighWatermark = offsetCheckpoints.fetch(log.parentDir, topicPartition).getOrElse { info(s"No checkpointed highwatermark is found for partition $topicPartition") @@ -358,12 +341,12 @@ class Partition(val topicPartition: TopicPartition, logManager.initializingLog(topicPartition) var maybeLog: Option[Log] = None try { - val log = logManager.getOrCreateLog(topicPartition, () => fetchLogConfig, isNew, isFutureReplica) + val log = logManager.getOrCreateLog(topicPartition, isNew, isFutureReplica) maybeLog = Some(log) updateHighWatermark(log) log } finally { - logManager.finishedInitializingLog(topicPartition, maybeLog, () => fetchLogConfig) + logManager.finishedInitializingLog(topicPartition, maybeLog) } } diff --git a/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala b/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala index 71ed3cb1a2..2c5bf70fe7 100644 --- a/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala @@ -24,7 +24,6 @@ import kafka.log.LogConfig import kafka.message.ProducerCompressionCodec import kafka.server._ import kafka.utils.Logging -import kafka.zk.KafkaZkClient import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember @@ -104,9 +103,9 @@ class GroupCoordinator(val brokerId: Int, /** * Startup logic executed at the same time when the server starts up. */ - def startup(enableMetadataExpiration: Boolean = true): Unit = { + def startup(retrieveGroupMetadataTopicPartitionCount: () => Int, enableMetadataExpiration: Boolean = true): Unit = { info("Starting up.") - groupManager.startup(enableMetadataExpiration) + groupManager.startup(retrieveGroupMetadataTopicPartitionCount, enableMetadataExpiration) isActive.set(true) info("Startup complete.") } @@ -1311,13 +1310,12 @@ object GroupCoordinator { val NewMemberJoinTimeoutMs: Int = 5 * 60 * 1000 def apply(config: KafkaConfig, - zkClient: KafkaZkClient, replicaManager: ReplicaManager, time: Time, metrics: Metrics): GroupCoordinator = { val heartbeatPurgatory = DelayedOperationPurgatory[DelayedHeartbeat]("Heartbeat", config.brokerId) val joinPurgatory = DelayedOperationPurgatory[DelayedJoin]("Rebalance", config.brokerId) - apply(config, zkClient, replicaManager, heartbeatPurgatory, joinPurgatory, time, metrics) + GroupCoordinator(config, replicaManager, heartbeatPurgatory, joinPurgatory, time, metrics) } private[group] def offsetConfig(config: KafkaConfig) = OffsetConfig( @@ -1334,7 +1332,6 @@ object GroupCoordinator { ) def apply(config: KafkaConfig, - zkClient: KafkaZkClient, replicaManager: ReplicaManager, heartbeatPurgatory: DelayedOperationPurgatory[DelayedHeartbeat], joinPurgatory: DelayedOperationPurgatory[DelayedJoin], @@ -1347,7 +1344,7 @@ object GroupCoordinator { groupInitialRebalanceDelayMs = config.groupInitialRebalanceDelay) val groupMetadataManager = new GroupMetadataManager(config.brokerId, config.interBrokerProtocolVersion, - offsetConfig, replicaManager, zkClient, time, metrics) + offsetConfig, replicaManager, time, metrics) new GroupCoordinator(config.brokerId, groupConfig, offsetConfig, groupMetadataManager, heartbeatPurgatory, joinPurgatory, time, metrics) } diff --git a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala index 0188a8df32..fb6d07e52f 100644 --- a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala @@ -35,7 +35,6 @@ import kafka.server.{FetchLogEnd, ReplicaManager} import kafka.utils.CoreUtils.inLock import kafka.utils.Implicits._ import kafka.utils._ -import kafka.zk.KafkaZkClient import org.apache.kafka.clients.consumer.ConsumerRecord import org.apache.kafka.clients.consumer.internals.ConsumerProtocol import org.apache.kafka.common.internals.Topic @@ -57,7 +56,6 @@ class GroupMetadataManager(brokerId: Int, interBrokerProtocolVersion: ApiVersion, config: OffsetConfig, val replicaManager: ReplicaManager, - zkClient: KafkaZkClient, time: Time, metrics: Metrics) extends Logging with KafkaMetricsGroup { @@ -78,7 +76,7 @@ class GroupMetadataManager(brokerId: Int, private val shuttingDown = new AtomicBoolean(false) /* number of partitions for the consumer metadata topic */ - private val groupMetadataTopicPartitionCount = getGroupMetadataTopicPartitionCount + @volatile private var groupMetadataTopicPartitionCount: Int = _ /* single-thread scheduler to handle offset/group metadata cache loading and unloading */ private val scheduler = new KafkaScheduler(threads = 1, threadNamePrefix = "group-metadata-manager-") @@ -170,7 +168,8 @@ class GroupMetadataManager(brokerId: Int, } }) - def startup(enableMetadataExpiration: Boolean): Unit = { + def startup(retrieveGroupMetadataTopicPartitionCount: () => Int, enableMetadataExpiration: Boolean): Unit = { + groupMetadataTopicPartitionCount = retrieveGroupMetadataTopicPartitionCount() scheduler.startup() if (enableMetadataExpiration) { scheduler.schedule(name = "delete-expired-group-metadata", @@ -934,14 +933,6 @@ class GroupMetadataManager(brokerId: Int, // TODO: clear the caches } - /** - * Gets the partition count of the group metadata topic from ZooKeeper. - * If the topic does not exist, the configured partition count is returned. - */ - private def getGroupMetadataTopicPartitionCount: Int = { - zkClient.getTopicPartitionCount(Topic.GROUP_METADATA_TOPIC_NAME).getOrElse(config.offsetsTopicNumPartitions) - } - /** * Check if the replica is local and return the message format version and timestamp * diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala index 3540c6a6f3..0a2c45b100 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala @@ -21,7 +21,6 @@ import java.util.concurrent.atomic.AtomicBoolean import kafka.server.{KafkaConfig, MetadataCache, ReplicaManager} import kafka.utils.{Logging, Scheduler} -import kafka.zk.KafkaZkClient import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.metrics.Metrics @@ -36,7 +35,6 @@ object TransactionCoordinator { replicaManager: ReplicaManager, scheduler: Scheduler, createProducerIdGenerator: () => ProducerIdGenerator, - zkClient: KafkaZkClient, metrics: Metrics, metadataCache: MetadataCache, time: Time): TransactionCoordinator = { @@ -52,7 +50,7 @@ object TransactionCoordinator { config.transactionRemoveExpiredTransactionalIdCleanupIntervalMs, config.requestTimeoutMs) - val txnStateManager = new TransactionStateManager(config.brokerId, zkClient, scheduler, replicaManager, txnConfig, + val txnStateManager = new TransactionStateManager(config.brokerId, scheduler, replicaManager, txnConfig, time, metrics) val logContext = new LogContext(s"[TransactionCoordinator id=${config.brokerId}] ") @@ -593,7 +591,7 @@ class TransactionCoordinator(brokerId: Int, /** * Startup logic executed at the same time when the server starts up. */ - def startup(enableTransactionalIdExpiration: Boolean = true): Unit = { + def startup(retrieveTransactionTopicPartitionCount: () => Int, enableTransactionalIdExpiration: Boolean = true): Unit = { info("Starting up.") scheduler.startup() scheduler.schedule("transaction-abort", @@ -601,8 +599,7 @@ class TransactionCoordinator(brokerId: Int, txnConfig.abortTimedOutTransactionsIntervalMs, txnConfig.abortTimedOutTransactionsIntervalMs ) - if (enableTransactionalIdExpiration) - txnManager.enableTransactionalIdExpiration() + txnManager.startup(retrieveTransactionTopicPartitionCount, enableTransactionalIdExpiration) txnMarkerChannelManager.start() isActive.set(true) diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala index 2882d86302..b547896706 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala @@ -28,7 +28,6 @@ import kafka.server.{Defaults, FetchLogEnd, ReplicaManager} import kafka.utils.CoreUtils.{inReadLock, inWriteLock} import kafka.utils.{Logging, Pool, Scheduler} import kafka.utils.Implicits._ -import kafka.zk.KafkaZkClient import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.metrics.stats.{Avg, Max} @@ -72,7 +71,6 @@ object TransactionStateManager { * */ class TransactionStateManager(brokerId: Int, - zkClient: KafkaZkClient, scheduler: Scheduler, replicaManager: ReplicaManager, config: TransactionConfig, @@ -96,7 +94,8 @@ class TransactionStateManager(brokerId: Int, private[transaction] val transactionMetadataCache: mutable.Map[Int, TxnMetadataCacheEntry] = mutable.Map() /** number of partitions for the transaction log topic */ - private val transactionTopicPartitionCount = getTransactionTopicPartitionCount + private var retrieveTransactionTopicPartitionCount: () => Int = _ + @volatile private var transactionTopicPartitionCount: Int = _ /** setup metrics*/ private val partitionLoadSensor = metrics.sensor(TransactionStateManager.LoadTimeSensor) @@ -276,14 +275,6 @@ class TransactionStateManager(brokerId: Int, def partitionFor(transactionalId: String): Int = Utils.abs(transactionalId.hashCode) % transactionTopicPartitionCount - /** - * Gets the partition count of the transaction log topic from ZooKeeper. - * If the topic does not exist, the default partition count is returned. - */ - private def getTransactionTopicPartitionCount: Int = { - zkClient.getTopicPartitionCount(Topic.TRANSACTION_STATE_TOPIC_NAME).getOrElse(config.transactionLogNumPartitions) - } - private def loadTransactionMetadata(topicPartition: TopicPartition, coordinatorEpoch: Int): Pool[String, TransactionMetadata] = { def logEndOffset = replicaManager.getLogEndOffset(topicPartition).getOrElse(-1L) @@ -472,9 +463,10 @@ class TransactionStateManager(brokerId: Int, } private def validateTransactionTopicPartitionCountIsStable(): Unit = { - val curTransactionTopicPartitionCount = getTransactionTopicPartitionCount - if (transactionTopicPartitionCount != curTransactionTopicPartitionCount) - throw new KafkaException(s"Transaction topic number of partitions has changed from $transactionTopicPartitionCount to $curTransactionTopicPartitionCount") + val previouslyDeterminedPartitionCount = transactionTopicPartitionCount + val curTransactionTopicPartitionCount = retrieveTransactionTopicPartitionCount() + if (previouslyDeterminedPartitionCount != curTransactionTopicPartitionCount) + throw new KafkaException(s"Transaction topic number of partitions has changed from $previouslyDeterminedPartitionCount to $curTransactionTopicPartitionCount") } def appendTransactionToLog(transactionalId: String, @@ -642,6 +634,13 @@ class TransactionStateManager(brokerId: Int, } } + def startup(retrieveTransactionTopicPartitionCount: () => Int, enableTransactionalIdExpiration: Boolean = true): Unit = { + this.retrieveTransactionTopicPartitionCount = retrieveTransactionTopicPartitionCount + transactionTopicPartitionCount = retrieveTransactionTopicPartitionCount() + if (enableTransactionalIdExpiration) + this.enableTransactionalIdExpiration() + } + def shutdown(): Unit = { shuttingDown.set(true) loadingPartitions.clear() diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index 761ba69bfa..b788bf0525 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -20,17 +20,16 @@ package kafka.log import java.io._ import java.nio.file.Files import java.util.concurrent._ -import java.util.concurrent.atomic.{AtomicInteger, AtomicReference} +import java.util.concurrent.atomic.AtomicInteger import kafka.metrics.KafkaMetricsGroup import kafka.server.checkpoints.OffsetCheckpointFile +import kafka.server.metadata.ConfigRepository import kafka.server._ import kafka.utils._ -import kafka.zk.KafkaZkClient import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.kafka.common.utils.Time import org.apache.kafka.common.errors.{KafkaStorageException, LogDirNotFoundException} -import org.apache.kafka.metadata.BrokerState import scala.jdk.CollectionConverters._ import scala.collection._ @@ -51,7 +50,7 @@ import kafka.utils.Implicits._ @threadsafe class LogManager(logDirs: Seq[File], initialOfflineDirs: Seq[File], - val topicConfigs: Map[String, LogConfig], // note that this doesn't get updated after creation + configRepository: ConfigRepository, val initialDefaultConfig: LogConfig, val cleanerConfig: CleanerConfig, recoveryThreadsPerDataDir: Int, @@ -61,7 +60,6 @@ class LogManager(logDirs: Seq[File], val retentionCheckMs: Long, val maxPidExpirationMs: Int, scheduler: Scheduler, - val brokerState: AtomicReference[BrokerState], brokerTopicStats: BrokerTopicStats, logDirFailureChannel: LogDirFailureChannel, time: Time) extends Logging with KafkaMetricsGroup { @@ -118,13 +116,8 @@ class LogManager(logDirs: Seq[File], logDirsSet } - loadLogs() - - private[kafka] val cleaner: LogCleaner = - if (cleanerConfig.enableCleaner) - new LogCleaner(cleanerConfig, liveLogDirs, currentLogs, logDirFailureChannel, time = time) - else - null + @volatile private var _cleaner: LogCleaner = _ + private[kafka] def cleaner: LogCleaner = _cleaner newGauge("OfflineLogDirectoryCount", () => offlineLogDirs.size) @@ -255,11 +248,12 @@ class LogManager(logDirs: Seq[File], private[log] def hasLogsToBeDeleted: Boolean = !logsToBeDeleted.isEmpty private[log] def loadLog(logDir: File, - hadCleanShutdown: Boolean, - recoveryPoints: Map[TopicPartition, Long], - logStartOffsets: Map[TopicPartition, Long]): Log = { + hadCleanShutdown: Boolean, + recoveryPoints: Map[TopicPartition, Long], + logStartOffsets: Map[TopicPartition, Long], + topicConfigOverrides: Map[String, LogConfig]): Log = { val topicPartition = Log.parseTopicPartitionName(logDir) - val config = topicConfigs.getOrElse(topicPartition.topic, currentDefaultConfig) + val config = topicConfigOverrides.getOrElse(topicPartition.topic, currentDefaultConfig) val logRecoveryPoint = recoveryPoints.getOrElse(topicPartition, 0L) val logStartOffset = logStartOffsets.getOrElse(topicPartition, 0L) @@ -302,7 +296,7 @@ class LogManager(logDirs: Seq[File], /** * Recover and load all logs in the given data directories */ - private[log] def loadLogs(): Unit = { + private[log] def loadLogs(topicConfigOverrides: Map[String, LogConfig]): Unit = { info(s"Loading logs from log dirs $liveLogDirs") val startMs = time.hiResClockMs() val threadPools = ArrayBuffer.empty[ExecutorService] @@ -327,7 +321,6 @@ class LogManager(logDirs: Seq[File], } else { // log recovery itself is being performed by `Log` class during initialization info(s"Attempting recovery for all logs in $logDirAbsolutePath since no clean shutdown file was found") - brokerState.set(BrokerState.RECOVERY) } var recoveryPoints = Map[TopicPartition, Long]() @@ -348,7 +341,8 @@ class LogManager(logDirs: Seq[File], s"$logDirAbsolutePath, resetting to the base offset of the first segment", e) } - val logsToLoad = Option(dir.listFiles).getOrElse(Array.empty).filter(_.isDirectory) + val logsToLoad = Option(dir.listFiles).getOrElse(Array.empty).filter(logDir => + logDir.isDirectory && Log.parseTopicPartitionName(logDir).topic != KafkaRaftServer.MetadataTopic) val numLogsLoaded = new AtomicInteger(0) numTotalLogs += logsToLoad.length @@ -358,7 +352,7 @@ class LogManager(logDirs: Seq[File], debug(s"Loading log $logDir") val logLoadStartMs = time.hiResClockMs() - val log = loadLog(logDir, hadCleanShutdown, recoveryPoints, logStartOffsets) + val log = loadLog(logDir, hadCleanShutdown, recoveryPoints, logStartOffsets, topicConfigOverrides) val logLoadDurationMs = time.hiResClockMs() - logLoadStartMs val currentNumLoaded = numLogsLoaded.incrementAndGet() @@ -403,7 +397,29 @@ class LogManager(logDirs: Seq[File], /** * Start the background threads to flush logs and do log cleanup */ - def startup(): Unit = { + def startup(topicNames: Set[String]): Unit = { + startupWithConfigOverrides(fetchTopicConfigOverrides(topicNames)) + } + + // visible for testing + private[log] def fetchTopicConfigOverrides(topicNames: Set[String]): Map[String, LogConfig] = { + val topicConfigOverrides = mutable.Map[String, LogConfig]() + val defaultProps = currentDefaultConfig.originals() + topicNames.foreach { topicName => + val overrides = configRepository.topicConfig(topicName) + // save memory by only including configs for topics with overrides + if (!overrides.isEmpty) { + val logConfig = LogConfig.fromProps(defaultProps, overrides) + topicConfigOverrides(topicName) = logConfig + } + } + topicConfigOverrides + } + + // visible for testing + private[log] def startupWithConfigOverrides(topicConfigOverrides: Map[String, LogConfig]): Unit = { + loadLogs(topicConfigOverrides) // this could take a while if shutdown was not clean + /* Schedule the cleanup task to delete old logs */ if (scheduler != null) { info("Starting log cleanup with a period of %d ms.".format(retentionCheckMs)) @@ -433,8 +449,10 @@ class LogManager(logDirs: Seq[File], delay = InitialTaskDelayMs, unit = TimeUnit.MILLISECONDS) } - if (cleanerConfig.enableCleaner) - cleaner.startup() + if (cleanerConfig.enableCleaner) { + _cleaner = new LogCleaner(cleanerConfig, liveLogDirs, currentLogs, logDirFailureChannel, time = time) + _cleaner.startup() + } } /** @@ -736,11 +754,15 @@ class LogManager(logDirs: Seq[File], * relevant log was being loaded. */ def finishedInitializingLog(topicPartition: TopicPartition, - maybeLog: Option[Log], - fetchLogConfig: () => LogConfig): Unit = { + maybeLog: Option[Log]): Unit = { val removedValue = partitionsInitializing.remove(topicPartition) if (removedValue.contains(true)) - maybeLog.foreach(_.updateConfig(fetchLogConfig())) + maybeLog.foreach(_.updateConfig(fetchLogConfig(topicPartition.topic))) + } + + private def fetchLogConfig(topicName: String): LogConfig = { + val props = configRepository.topicConfig(topicName) + LogConfig.fromProps(currentDefaultConfig.originals, props) } /** @@ -754,7 +776,7 @@ class LogManager(logDirs: Seq[File], * @param isFuture True if the future log of the specified partition should be returned or created * @throws KafkaStorageException if isNew=false, log is not found in the cache and there is offline log directory on the broker */ - def getOrCreateLog(topicPartition: TopicPartition, loadConfig: () => LogConfig, isNew: Boolean = false, isFuture: Boolean = false): Log = { + def getOrCreateLog(topicPartition: TopicPartition, isNew: Boolean = false, isFuture: Boolean = false): Log = { logCreationOrDeletionLock synchronized { getLog(topicPartition, isFuture).getOrElse { // create the log if it has not already been created in another thread @@ -791,7 +813,7 @@ class LogManager(logDirs: Seq[File], .getOrElse(Failure(new KafkaStorageException("No log directories available. Tried " + logDirs.map(_.getAbsolutePath).mkString(", ")))) .get // If Failure, will throw - val config = loadConfig() + val config = fetchLogConfig(topicPartition.topic) val log = Log( dir = logDir, config = config, @@ -1182,8 +1204,7 @@ object LogManager { def apply(config: KafkaConfig, initialOfflineDirs: Seq[String], - zkClient: KafkaZkClient, - brokerState: AtomicReference[BrokerState], + configRepository: ConfigRepository, kafkaScheduler: KafkaScheduler, time: Time, brokerTopicStats: BrokerTopicStats, @@ -1193,18 +1214,11 @@ object LogManager { LogConfig.validateValues(defaultProps) val defaultLogConfig = LogConfig(defaultProps) - // read the log configurations from zookeeper - val (topicConfigs, failed) = zkClient.getLogConfigs( - zkClient.getAllTopicsInCluster(), - defaultProps - ) - if (!failed.isEmpty) throw failed.head._2 - val cleanerConfig = LogCleaner.cleanerConfig(config) new LogManager(logDirs = config.logDirs.map(new File(_).getAbsoluteFile), initialOfflineDirs = initialOfflineDirs.map(new File(_).getAbsoluteFile), - topicConfigs = topicConfigs, + configRepository = configRepository, initialDefaultConfig = defaultLogConfig, cleanerConfig = cleanerConfig, recoveryThreadsPerDataDir = config.numRecoveryThreadsPerDataDir, @@ -1214,9 +1228,9 @@ object LogManager { retentionCheckMs = config.logCleanupIntervalMs, maxPidExpirationMs = config.transactionalIdExpirationMs, scheduler = kafkaScheduler, - brokerState = brokerState, brokerTopicStats = brokerTopicStats, logDirFailureChannel = logDirFailureChannel, time = time) } + } diff --git a/core/src/main/scala/kafka/serializer/Decoder.scala b/core/src/main/scala/kafka/serializer/Decoder.scala index 9c1190b6f1..4b8c545f52 100644 --- a/core/src/main/scala/kafka/serializer/Decoder.scala +++ b/core/src/main/scala/kafka/serializer/Decoder.scala @@ -18,6 +18,7 @@ package kafka.serializer import java.nio.ByteBuffer +import java.nio.charset.StandardCharsets import kafka.utils.VerifiableProperties @@ -44,9 +45,9 @@ class DefaultDecoder(props: VerifiableProperties = null) extends Decoder[Array[B class StringDecoder(props: VerifiableProperties = null) extends Decoder[String] { val encoding = if(props == null) - "UTF8" + StandardCharsets.UTF_8.name() else - props.getString("serializer.encoding", "UTF8") + props.getString("serializer.encoding", StandardCharsets.UTF_8.name()) def fromBytes(bytes: Array[Byte]): String = { new String(bytes, encoding) diff --git a/core/src/main/scala/kafka/server/AutoTopicCreationManager.scala b/core/src/main/scala/kafka/server/AutoTopicCreationManager.scala new file mode 100644 index 0000000000..ec7f2df3e6 --- /dev/null +++ b/core/src/main/scala/kafka/server/AutoTopicCreationManager.scala @@ -0,0 +1,321 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server + +import java.util.{Collections, Properties} +import java.util.concurrent.ConcurrentHashMap +import java.util.concurrent.atomic.AtomicReference + +import kafka.controller.KafkaController +import kafka.coordinator.group.GroupCoordinator +import kafka.coordinator.transaction.TransactionCoordinator +import kafka.server.metadata.MetadataBroker +import kafka.utils.Logging +import org.apache.kafka.clients.ClientResponse +import org.apache.kafka.common.errors.InvalidTopicException +import org.apache.kafka.common.internals.Topic +import org.apache.kafka.common.internals.Topic.{GROUP_METADATA_TOPIC_NAME, TRANSACTION_STATE_TOPIC_NAME} +import org.apache.kafka.common.message.CreateTopicsRequestData +import org.apache.kafka.common.message.CreateTopicsRequestData.{CreatableTopic, CreateableTopicConfig, CreateableTopicConfigCollection} +import org.apache.kafka.common.message.MetadataResponseData.MetadataResponseTopic +import org.apache.kafka.common.metrics.Metrics +import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.requests.{ApiError, CreateTopicsRequest} +import org.apache.kafka.common.utils.Time + +import scala.collection.{Map, Seq, Set, mutable} +import scala.jdk.CollectionConverters._ + +trait AutoTopicCreationManager { + + def createTopics( + topicNames: Set[String], + controllerMutationQuota: ControllerMutationQuota + ): Seq[MetadataResponseTopic] + + def start(): Unit + + def shutdown(): Unit +} + +object AutoTopicCreationManager { + + def apply( + config: KafkaConfig, + metadataCache: MetadataCache, + time: Time, + metrics: Metrics, + threadNamePrefix: Option[String], + adminManager: ZkAdminManager, + controller: KafkaController, + groupCoordinator: GroupCoordinator, + txnCoordinator: TransactionCoordinator, + enableForwarding: Boolean + ): AutoTopicCreationManager = { + + val channelManager = + if (enableForwarding) + Some(new BrokerToControllerChannelManagerImpl( + controllerNodeProvider = MetadataCacheControllerNodeProvider( + config, metadataCache), + time = time, + metrics = metrics, + config = config, + channelName = "autoTopicCreationChannel", + threadNamePrefix = threadNamePrefix, + retryTimeoutMs = config.requestTimeoutMs.longValue + )) + else + None + new DefaultAutoTopicCreationManager(config, metadataCache, channelManager, adminManager, + controller, groupCoordinator, txnCoordinator) + } +} + +class DefaultAutoTopicCreationManager( + config: KafkaConfig, + metadataCache: MetadataCache, + channelManager: Option[BrokerToControllerChannelManager], + adminManager: ZkAdminManager, + controller: KafkaController, + groupCoordinator: GroupCoordinator, + txnCoordinator: TransactionCoordinator +) extends AutoTopicCreationManager with Logging { + + private val inflightTopics = Collections.newSetFromMap(new ConcurrentHashMap[String, java.lang.Boolean]()) + + override def start(): Unit = { + channelManager.foreach(_.start()) + } + + override def shutdown(): Unit = { + channelManager.foreach(_.shutdown()) + inflightTopics.clear() + } + + override def createTopics( + topics: Set[String], + controllerMutationQuota: ControllerMutationQuota + ): Seq[MetadataResponseTopic] = { + val (creatableTopics, uncreatableTopicResponses) = filterCreatableTopics(topics) + + val creatableTopicResponses = if (creatableTopics.isEmpty) { + Seq.empty + } else if (!controller.isActive && channelManager.isDefined) { + sendCreateTopicRequest(creatableTopics) + } else { + createTopicsInZk(creatableTopics, controllerMutationQuota) + } + + uncreatableTopicResponses ++ creatableTopicResponses + } + + private def createTopicsInZk( + creatableTopics: Map[String, CreatableTopic], + controllerMutationQuota: ControllerMutationQuota + ): Seq[MetadataResponseTopic] = { + val topicErrors = new AtomicReference[Map[String, ApiError]]() + try { + // Note that we use timeout = 0 since we do not need to wait for metadata propagation + // and we want to get the response error immediately. + adminManager.createTopics( + timeout = 0, + validateOnly = false, + creatableTopics, + Map.empty, + controllerMutationQuota, + topicErrors.set + ) + + val creatableTopicResponses = Option(topicErrors.get) match { + case Some(errors) => + errors.toSeq.map { case (topic, apiError) => + val error = apiError.error match { + case Errors.TOPIC_ALREADY_EXISTS | Errors.REQUEST_TIMED_OUT => + // The timeout error is expected because we set timeout=0. This + // nevertheless indicates that the topic metadata was created + // successfully, so we return LEADER_NOT_AVAILABLE. + Errors.LEADER_NOT_AVAILABLE + case error => error + } + + new MetadataResponseTopic() + .setErrorCode(error.code) + .setName(topic) + .setIsInternal(Topic.isInternal(topic)) + } + + case None => + creatableTopics.keySet.toSeq.map { topic => + new MetadataResponseTopic() + .setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code) + .setName(topic) + .setIsInternal(Topic.isInternal(topic)) + } + } + + creatableTopicResponses + } finally { + clearInflightRequests(creatableTopics) + } + } + + private def sendCreateTopicRequest( + creatableTopics: Map[String, CreatableTopic] + ): Seq[MetadataResponseTopic] = { + val topicsToCreate = new CreateTopicsRequestData.CreatableTopicCollection(creatableTopics.size) + topicsToCreate.addAll(creatableTopics.values.asJavaCollection) + + val createTopicsRequest = new CreateTopicsRequest.Builder( + new CreateTopicsRequestData() + .setTimeoutMs(config.requestTimeoutMs) + .setTopics(topicsToCreate) + ) + + channelManager.get.sendRequest(createTopicsRequest, new ControllerRequestCompletionHandler { + override def onTimeout(): Unit = { + debug(s"Auto topic creation timed out for ${creatableTopics.keys}.") + clearInflightRequests(creatableTopics) + } + + override def onComplete(response: ClientResponse): Unit = { + debug(s"Auto topic creation completed for ${creatableTopics.keys}.") + clearInflightRequests(creatableTopics) + } + }) + + val creatableTopicResponses = creatableTopics.keySet.toSeq.map { topic => + new MetadataResponseTopic() + .setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code) + .setName(topic) + .setIsInternal(Topic.isInternal(topic)) + } + + info(s"Sent auto-creation request for ${creatableTopics.keys} to the active controller.") + creatableTopicResponses + } + + private def clearInflightRequests(creatableTopics: Map[String, CreatableTopic]): Unit = { + creatableTopics.keySet.foreach(inflightTopics.remove) + debug(s"Cleared inflight topic creation state for $creatableTopics") + } + + private def creatableTopic(topic: String): CreatableTopic = { + topic match { + case GROUP_METADATA_TOPIC_NAME => + new CreatableTopic() + .setName(topic) + .setNumPartitions(config.offsetsTopicPartitions) + .setReplicationFactor(config.offsetsTopicReplicationFactor) + .setConfigs(convertToTopicConfigCollections(groupCoordinator.offsetsTopicConfigs)) + case TRANSACTION_STATE_TOPIC_NAME => + new CreatableTopic() + .setName(topic) + .setNumPartitions(config.transactionTopicPartitions) + .setReplicationFactor(config.transactionTopicReplicationFactor) + .setConfigs(convertToTopicConfigCollections( + txnCoordinator.transactionTopicConfigs)) + case topicName => + new CreatableTopic() + .setName(topicName) + .setNumPartitions(config.numPartitions) + .setReplicationFactor(config.defaultReplicationFactor.shortValue) + } + } + + private def convertToTopicConfigCollections(config: Properties): CreateableTopicConfigCollection = { + val topicConfigs = new CreateableTopicConfigCollection() + config.forEach { + case (name, value) => + topicConfigs.add(new CreateableTopicConfig() + .setName(name.toString) + .setValue(value.toString)) + } + topicConfigs + } + + private def isValidTopicName(topic: String): Boolean = { + try { + Topic.validate(topic) + true + } catch { + case _: InvalidTopicException => + false + } + } + + private def filterCreatableTopics( + topics: Set[String] + ): (Map[String, CreatableTopic], Seq[MetadataResponseTopic]) = { + + val aliveBrokers = metadataCache.getAliveBrokers + val creatableTopics = mutable.Map.empty[String, CreatableTopic] + val uncreatableTopics = mutable.Buffer.empty[MetadataResponseTopic] + + topics.foreach { topic => + // Attempt basic topic validation before sending any requests to the controller. + val validationError: Option[Errors] = if (!isValidTopicName(topic)) { + Some(Errors.INVALID_TOPIC_EXCEPTION) + } else if (!hasEnoughLiveBrokers(topic, aliveBrokers)) { + Some(Errors.INVALID_REPLICATION_FACTOR) + } else if (!inflightTopics.add(topic)) { + Some(Errors.UNKNOWN_TOPIC_OR_PARTITION) + } else { + None + } + + validationError match { + case Some(error) => + uncreatableTopics += new MetadataResponseTopic() + .setErrorCode(error.code) + .setName(topic) + .setIsInternal(Topic.isInternal(topic)) + case None => + creatableTopics.put(topic, creatableTopic(topic)) + } + } + + (creatableTopics, uncreatableTopics) + } + + private def hasEnoughLiveBrokers( + topicName: String, + aliveBrokers: Seq[MetadataBroker] + ): Boolean = { + val (replicationFactor, replicationFactorConfig) = topicName match { + case GROUP_METADATA_TOPIC_NAME => + (config.offsetsTopicReplicationFactor.intValue, KafkaConfig.OffsetsTopicReplicationFactorProp) + + case TRANSACTION_STATE_TOPIC_NAME => + (config.transactionTopicReplicationFactor.intValue, KafkaConfig.TransactionsTopicReplicationFactorProp) + + case _ => + (config.defaultReplicationFactor, KafkaConfig.DefaultReplicationFactorProp) + } + + if (aliveBrokers.size < replicationFactor) { + error(s"Number of alive brokers '${aliveBrokers.size}' does not meet the required replication factor " + + s"'$replicationFactor' for auto creation of topic '$topicName' which is configured by $replicationFactorConfig. " + + "This error can be ignored if the cluster is starting up and not all brokers are up yet.") + false + } else { + true + } + } + +} diff --git a/core/src/main/scala/kafka/server/BrokerMetadataCheckpoint.scala b/core/src/main/scala/kafka/server/BrokerMetadataCheckpoint.scala index 93a0abe179..e3d5c4b312 100755 --- a/core/src/main/scala/kafka/server/BrokerMetadataCheckpoint.scala +++ b/core/src/main/scala/kafka/server/BrokerMetadataCheckpoint.scala @@ -88,9 +88,9 @@ class RawMetaProperties(val props: Properties = new Properties()) { } override def toString: String = { - "RawMetaProperties(" + props.keySet().asScala.toList.asInstanceOf[List[String]].sorted.map { + "{" + props.keySet().asScala.toList.asInstanceOf[List[String]].sorted.map { key => key + "=" + props.get(key) - }.mkString(", ") + ")" + }.mkString(", ") + "}" } } @@ -130,7 +130,7 @@ case class ZkMetaProperties( } override def toString: String = { - s"LegacyMetaProperties(brokerId=$brokerId, clusterId=$clusterId)" + s"ZkMetaProperties(brokerId=$brokerId, clusterId=$clusterId)" } } diff --git a/core/src/main/scala/kafka/server/BrokerToControllerChannelManager.scala b/core/src/main/scala/kafka/server/BrokerToControllerChannelManager.scala index a4b6a044c9..1e7af76aeb 100644 --- a/core/src/main/scala/kafka/server/BrokerToControllerChannelManager.scala +++ b/core/src/main/scala/kafka/server/BrokerToControllerChannelManager.scala @@ -67,7 +67,7 @@ class MetadataCacheControllerNodeProvider( override def get(): Option[Node] = { metadataCache.getControllerId .flatMap(metadataCache.getAliveBroker) - .map(_.node(listenerName)) + .map(_.endpoints(listenerName.value)) } } @@ -133,7 +133,6 @@ class BrokerToControllerChannelManagerImpl( def shutdown(): Unit = { requestThread.shutdown() - requestThread.awaitShutdown() info(s"Broker to controller channel manager for $channelName shutdown") } diff --git a/core/src/main/scala/kafka/server/ConfigHelper.scala b/core/src/main/scala/kafka/server/ConfigHelper.scala new file mode 100644 index 0000000000..2bc8dd9232 --- /dev/null +++ b/core/src/main/scala/kafka/server/ConfigHelper.scala @@ -0,0 +1,216 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server + +import java.util.{Collections, Properties} + +import kafka.log.LogConfig +import kafka.server.metadata.ConfigRepository +import kafka.utils.{Log4jController, Logging} +import org.apache.kafka.common.config.{AbstractConfig, ConfigDef, ConfigResource} +import org.apache.kafka.common.errors.{ApiException, InvalidRequestException} +import org.apache.kafka.common.internals.Topic +import org.apache.kafka.common.message.DescribeConfigsRequestData.DescribeConfigsResource +import org.apache.kafka.common.message.DescribeConfigsResponseData +import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.requests.{ApiError, DescribeConfigsResponse} +import org.apache.kafka.common.requests.DescribeConfigsResponse.ConfigSource + +import scala.collection.{Map, mutable} +import scala.jdk.CollectionConverters._ + +class ConfigHelper(metadataCache: MetadataCache, config: KafkaConfig, configRepository: ConfigRepository) extends Logging { + + def describeConfigs(resourceToConfigNames: List[DescribeConfigsResource], + includeSynonyms: Boolean, + includeDocumentation: Boolean): List[DescribeConfigsResponseData.DescribeConfigsResult] = { + resourceToConfigNames.map { case resource => + + def allConfigs(config: AbstractConfig) = { + config.originals.asScala.filter(_._2 != null) ++ config.nonInternalValues.asScala + } + + def createResponseConfig(configs: Map[String, Any], + createConfigEntry: (String, Any) => DescribeConfigsResponseData.DescribeConfigsResourceResult): DescribeConfigsResponseData.DescribeConfigsResult = { + val filteredConfigPairs = if (resource.configurationKeys == null) + configs.toBuffer + else + configs.filter { case (configName, _) => + resource.configurationKeys.asScala.forall(_.contains(configName)) + }.toBuffer + + val configEntries = filteredConfigPairs.map { case (name, value) => createConfigEntry(name, value) } + new DescribeConfigsResponseData.DescribeConfigsResult().setErrorCode(Errors.NONE.code) + .setConfigs(configEntries.asJava) + } + + try { + val configResult = ConfigResource.Type.forId(resource.resourceType) match { + case ConfigResource.Type.TOPIC => + val topic = resource.resourceName + Topic.validate(topic) + if (metadataCache.contains(topic)) { + val topicProps = configRepository.topicConfig(topic) + val logConfig = LogConfig.fromProps(LogConfig.extractLogConfigMap(config), topicProps) + createResponseConfig(allConfigs(logConfig), createTopicConfigEntry(logConfig, topicProps, includeSynonyms, includeDocumentation)) + } else { + new DescribeConfigsResponseData.DescribeConfigsResult().setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code) + .setConfigs(Collections.emptyList[DescribeConfigsResponseData.DescribeConfigsResourceResult]) + } + + case ConfigResource.Type.BROKER => + if (resource.resourceName == null || resource.resourceName.isEmpty) + createResponseConfig(config.dynamicConfig.currentDynamicDefaultConfigs, + createBrokerConfigEntry(perBrokerConfig = false, includeSynonyms, includeDocumentation)) + else if (resourceNameToBrokerId(resource.resourceName) == config.brokerId) + createResponseConfig(allConfigs(config), + createBrokerConfigEntry(perBrokerConfig = true, includeSynonyms, includeDocumentation)) + else + throw new InvalidRequestException(s"Unexpected broker id, expected ${config.brokerId} or empty string, but received ${resource.resourceName}") + + case ConfigResource.Type.BROKER_LOGGER => + if (resource.resourceName == null || resource.resourceName.isEmpty) + throw new InvalidRequestException("Broker id must not be empty") + else if (resourceNameToBrokerId(resource.resourceName) != config.brokerId) + throw new InvalidRequestException(s"Unexpected broker id, expected ${config.brokerId} but received ${resource.resourceName}") + else + createResponseConfig(Log4jController.loggers, + (name, value) => new DescribeConfigsResponseData.DescribeConfigsResourceResult().setName(name) + .setValue(value.toString).setConfigSource(ConfigSource.DYNAMIC_BROKER_LOGGER_CONFIG.id) + .setIsSensitive(false).setReadOnly(false).setSynonyms(List.empty.asJava)) + case resourceType => throw new InvalidRequestException(s"Unsupported resource type: $resourceType") + } + configResult.setResourceName(resource.resourceName).setResourceType(resource.resourceType) + } catch { + case e: Throwable => + // Log client errors at a lower level than unexpected exceptions + val message = s"Error processing describe configs request for resource $resource" + if (e.isInstanceOf[ApiException]) + info(message, e) + else + error(message, e) + val err = ApiError.fromThrowable(e) + new DescribeConfigsResponseData.DescribeConfigsResult() + .setResourceName(resource.resourceName) + .setResourceType(resource.resourceType) + .setErrorMessage(err.message) + .setErrorCode(err.error.code) + .setConfigs(Collections.emptyList[DescribeConfigsResponseData.DescribeConfigsResourceResult]) + } + } + } + + def createTopicConfigEntry(logConfig: LogConfig, topicProps: Properties, includeSynonyms: Boolean, includeDocumentation: Boolean) + (name: String, value: Any): DescribeConfigsResponseData.DescribeConfigsResourceResult = { + val configEntryType = LogConfig.configType(name) + val isSensitive = KafkaConfig.maybeSensitive(configEntryType) + val valueAsString = if (isSensitive) null else ConfigDef.convertToString(value, configEntryType.orNull) + val allSynonyms = { + val list = LogConfig.TopicConfigSynonyms.get(name) + .map(s => configSynonyms(s, brokerSynonyms(s), isSensitive)) + .getOrElse(List.empty) + if (!topicProps.containsKey(name)) + list + else + new DescribeConfigsResponseData.DescribeConfigsSynonym().setName(name).setValue(valueAsString) + .setSource(ConfigSource.TOPIC_CONFIG.id) +: list + } + val source = if (allSynonyms.isEmpty) ConfigSource.DEFAULT_CONFIG.id else allSynonyms.head.source + val synonyms = if (!includeSynonyms) List.empty else allSynonyms + val dataType = configResponseType(configEntryType) + val configDocumentation = if (includeDocumentation) logConfig.documentationOf(name) else null + new DescribeConfigsResponseData.DescribeConfigsResourceResult() + .setName(name).setValue(valueAsString).setConfigSource(source) + .setIsSensitive(isSensitive).setReadOnly(false).setSynonyms(synonyms.asJava) + .setDocumentation(configDocumentation).setConfigType(dataType.id) + } + + private def createBrokerConfigEntry(perBrokerConfig: Boolean, includeSynonyms: Boolean, includeDocumentation: Boolean) + (name: String, value: Any): DescribeConfigsResponseData.DescribeConfigsResourceResult = { + val allNames = brokerSynonyms(name) + val configEntryType = KafkaConfig.configType(name) + val isSensitive = KafkaConfig.maybeSensitive(configEntryType) + val valueAsString = if (isSensitive) + null + else value match { + case v: String => v + case _ => ConfigDef.convertToString(value, configEntryType.orNull) + } + val allSynonyms = configSynonyms(name, allNames, isSensitive) + .filter(perBrokerConfig || _.source == ConfigSource.DYNAMIC_DEFAULT_BROKER_CONFIG.id) + val synonyms = if (!includeSynonyms) List.empty else allSynonyms + val source = if (allSynonyms.isEmpty) ConfigSource.DEFAULT_CONFIG.id else allSynonyms.head.source + val readOnly = !DynamicBrokerConfig.AllDynamicConfigs.contains(name) + + val dataType = configResponseType(configEntryType) + val configDocumentation = if (includeDocumentation) brokerDocumentation(name) else null + new DescribeConfigsResponseData.DescribeConfigsResourceResult().setName(name).setValue(valueAsString).setConfigSource(source) + .setIsSensitive(isSensitive).setReadOnly(readOnly).setSynonyms(synonyms.asJava) + .setDocumentation(configDocumentation).setConfigType(dataType.id) + } + + private def configSynonyms(name: String, synonyms: List[String], isSensitive: Boolean): List[DescribeConfigsResponseData.DescribeConfigsSynonym] = { + val dynamicConfig = config.dynamicConfig + val allSynonyms = mutable.Buffer[DescribeConfigsResponseData.DescribeConfigsSynonym]() + + def maybeAddSynonym(map: Map[String, String], source: ConfigSource)(name: String): Unit = { + map.get(name).map { value => + val configValue = if (isSensitive) null else value + allSynonyms += new DescribeConfigsResponseData.DescribeConfigsSynonym().setName(name).setValue(configValue).setSource(source.id) + } + } + + synonyms.foreach(maybeAddSynonym(dynamicConfig.currentDynamicBrokerConfigs, ConfigSource.DYNAMIC_BROKER_CONFIG)) + synonyms.foreach(maybeAddSynonym(dynamicConfig.currentDynamicDefaultConfigs, ConfigSource.DYNAMIC_DEFAULT_BROKER_CONFIG)) + synonyms.foreach(maybeAddSynonym(dynamicConfig.staticBrokerConfigs, ConfigSource.STATIC_BROKER_CONFIG)) + synonyms.foreach(maybeAddSynonym(dynamicConfig.staticDefaultConfigs, ConfigSource.DEFAULT_CONFIG)) + allSynonyms.dropWhile(s => s.name != name).toList // e.g. drop listener overrides when describing base config + } + + private def brokerSynonyms(name: String): List[String] = { + DynamicBrokerConfig.brokerConfigSynonyms(name, matchListenerOverride = true) + } + + private def brokerDocumentation(name: String): String = { + config.documentationOf(name) + } + + private def configResponseType(configType: Option[ConfigDef.Type]): DescribeConfigsResponse.ConfigType = { + if (configType.isEmpty) + DescribeConfigsResponse.ConfigType.UNKNOWN + else configType.get match { + case ConfigDef.Type.BOOLEAN => DescribeConfigsResponse.ConfigType.BOOLEAN + case ConfigDef.Type.STRING => DescribeConfigsResponse.ConfigType.STRING + case ConfigDef.Type.INT => DescribeConfigsResponse.ConfigType.INT + case ConfigDef.Type.SHORT => DescribeConfigsResponse.ConfigType.SHORT + case ConfigDef.Type.LONG => DescribeConfigsResponse.ConfigType.LONG + case ConfigDef.Type.DOUBLE => DescribeConfigsResponse.ConfigType.DOUBLE + case ConfigDef.Type.LIST => DescribeConfigsResponse.ConfigType.LIST + case ConfigDef.Type.CLASS => DescribeConfigsResponse.ConfigType.CLASS + case ConfigDef.Type.PASSWORD => DescribeConfigsResponse.ConfigType.PASSWORD + case _ => DescribeConfigsResponse.ConfigType.UNKNOWN + } + } + + private def resourceNameToBrokerId(resourceName: String): Int = { + try resourceName.toInt catch { + case _: NumberFormatException => + throw new InvalidRequestException(s"Broker id must be an integer, but it is: $resourceName") + } + } +} diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index 2370cabd3f..91d214fd8e 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -29,7 +29,7 @@ import kafka.utils.{CoreUtils, Logging, PasswordEncoder} import kafka.utils.Implicits._ import kafka.zk.{AdminZkClient, KafkaZkClient} import org.apache.kafka.common.Reconfigurable -import org.apache.kafka.common.config.{ConfigDef, ConfigException, SslConfigs, AbstractConfig} +import org.apache.kafka.common.config.{AbstractConfig, ConfigDef, ConfigException, SslConfigs} import org.apache.kafka.common.metrics.MetricsReporter import org.apache.kafka.common.config.types.Password import org.apache.kafka.common.network.{ListenerName, ListenerReconfigurable} @@ -239,7 +239,7 @@ class DynamicBrokerConfig(private val kafkaConfig: KafkaConfig) extends Logging * reconfigured after `KafkaConfig` is updated so that they can access `KafkaConfig` * directly. They are provided both old and new configs. */ - def addReconfigurables(kafkaServer: KafkaServer): Unit = { + def addReconfigurables(kafkaServer: KafkaBroker): Unit = { kafkaServer.authorizer match { case Some(authz: Reconfigurable) => addReconfigurable(authz) case _ => @@ -625,7 +625,7 @@ object DynamicLogConfig { val KafkaConfigToLogConfigName = LogConfig.TopicConfigSynonyms.map { case (k, v) => (v, k) } } -class DynamicLogConfig(logManager: LogManager, server: KafkaServer) extends BrokerReconfigurable with Logging { +class DynamicLogConfig(logManager: LogManager, server: KafkaBroker) extends BrokerReconfigurable with Logging { override def reconfigurableConfigs: Set[String] = { DynamicLogConfig.ReconfigurableConfigs @@ -669,7 +669,10 @@ class DynamicLogConfig(logManager: LogManager, server: KafkaServer) extends Brok updateLogsConfig(newBrokerDefaults.asScala) if (logManager.currentDefaultConfig.uncleanLeaderElectionEnable && !origUncleanLeaderElectionEnable) { - server.kafkaController.enableDefaultUncleanLeaderElection() + server match { + case kafkaServer: KafkaServer => kafkaServer.kafkaController.enableDefaultUncleanLeaderElection() + case _ => + } } } } @@ -683,7 +686,7 @@ object DynamicThreadPool { KafkaConfig.BackgroundThreadsProp) } -class DynamicThreadPool(server: KafkaServer) extends BrokerReconfigurable { +class DynamicThreadPool(server: KafkaBroker) extends BrokerReconfigurable { override def reconfigurableConfigs: Set[String] = { DynamicThreadPool.ReconfigurableConfigs @@ -715,7 +718,7 @@ class DynamicThreadPool(server: KafkaServer) extends BrokerReconfigurable { if (newConfig.numReplicaFetchers != oldConfig.numReplicaFetchers) server.replicaManager.replicaFetcherManager.resizeThreadPool(newConfig.numReplicaFetchers) if (newConfig.numRecoveryThreadsPerDataDir != oldConfig.numRecoveryThreadsPerDataDir) - server.getLogManager.resizeRecoveryThreadPool(newConfig.numRecoveryThreadsPerDataDir) + server.logManager.resizeRecoveryThreadPool(newConfig.numRecoveryThreadsPerDataDir) if (newConfig.backgroundThreads != oldConfig.backgroundThreads) server.kafkaScheduler.resizeThreadPool(newConfig.backgroundThreads) } @@ -732,7 +735,7 @@ class DynamicThreadPool(server: KafkaServer) extends BrokerReconfigurable { } } -class DynamicMetricsReporters(brokerId: Int, server: KafkaServer) extends Reconfigurable { +class DynamicMetricsReporters(brokerId: Int, server: KafkaBroker) extends Reconfigurable { private val dynamicConfig = server.config.dynamicConfig private val metrics = server.metrics @@ -796,8 +799,8 @@ class DynamicMetricsReporters(brokerId: Int, server: KafkaServer) extends Reconf metrics.addReporter(reporter) currentReporters += reporter.getClass.getName -> reporter } - server.notifyClusterListeners(reporters.asScala) - server.notifyMetricsReporters(reporters.asScala) + KafkaBroker.notifyClusterListeners(server.clusterId, reporters.asScala) + KafkaBroker.notifyMetricsReporters(server.clusterId, server.config, reporters.asScala) } private def removeReporter(className: String): Unit = { @@ -857,7 +860,7 @@ object DynamicListenerConfig { ) } -class DynamicClientQuotaCallback(brokerId: Int, server: KafkaServer) extends Reconfigurable { +class DynamicClientQuotaCallback(brokerId: Int, server: KafkaBroker) extends Reconfigurable { override def configure(configs: util.Map[String, _]): Unit = {} @@ -888,7 +891,7 @@ class DynamicClientQuotaCallback(brokerId: Int, server: KafkaServer) extends Rec } } -class DynamicListenerConfig(server: KafkaServer) extends BrokerReconfigurable with Logging { +class DynamicListenerConfig(server: KafkaBroker) extends BrokerReconfigurable with Logging { override def reconfigurableConfigs: Set[String] = { DynamicListenerConfig.ReconfigurableConfigs @@ -896,6 +899,9 @@ class DynamicListenerConfig(server: KafkaServer) extends BrokerReconfigurable wi def validateReconfiguration(newConfig: KafkaConfig): Unit = { val oldConfig = server.config + if (!oldConfig.requiresZookeeper) { + throw new ConfigException("Dynamic reconfiguration of listeners is not yet supported when using a Raft-based metadata quorum") + } val newListeners = listenersToMap(newConfig.listeners) val newAdvertisedListeners = listenersToMap(newConfig.advertisedListeners) val oldListeners = listenersToMap(oldConfig.listeners) @@ -936,7 +942,10 @@ class DynamicListenerConfig(server: KafkaServer) extends BrokerReconfigurable wi if (listenersAdded.nonEmpty) server.socketServer.addListeners(listenersAdded) - server.kafkaController.updateBrokerInfo(server.createBrokerInfo) + server match { + case kafkaServer: KafkaServer => kafkaServer.kafkaController.updateBrokerInfo(kafkaServer.createBrokerInfo) + case _ => + } } private def listenersToMap(listeners: Seq[EndPoint]): Map[ListenerName, EndPoint] = diff --git a/core/src/main/scala/kafka/server/ForwardingManager.scala b/core/src/main/scala/kafka/server/ForwardingManager.scala index c90387895c..b77bd34563 100644 --- a/core/src/main/scala/kafka/server/ForwardingManager.scala +++ b/core/src/main/scala/kafka/server/ForwardingManager.scala @@ -158,5 +158,4 @@ class ForwardingManagerImpl( request.getErrorResponse(Errors.UNKNOWN_SERVER_ERROR.exception) } } - } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index df695fc5ea..938c401f94 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -21,14 +21,14 @@ import java.lang.{Long => JLong} import java.net.{InetAddress, UnknownHostException} import java.nio.ByteBuffer import java.util -import java.util.{Collections, Optional} import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.atomic.AtomicInteger +import java.util.{Collections, Optional} -import kafka.admin.{AdminUtils, RackAwareMode} +import kafka.admin.AdminUtils import kafka.api.{ApiVersion, ElectLeadersRequestOps, KAFKA_0_11_0_IV0, KAFKA_2_3_IV0} import kafka.common.OffsetAndMetadata -import kafka.controller.{KafkaController, ReplicaAssignment} +import kafka.controller.ReplicaAssignment import kafka.coordinator.group.{GroupCoordinator, JoinGroupResult, LeaveGroupResult, SyncGroupResult} import kafka.coordinator.transaction.{InitProducerIdResult, TransactionCoordinator} import kafka.log.AppendOrigin @@ -36,33 +36,33 @@ import kafka.message.ZStdCompressionCodec import kafka.network.RequestChannel import kafka.security.authorizer.AuthorizerUtils import kafka.server.QuotaFactory.{QuotaManagers, UnboundedQuota} -import kafka.utils.{CoreUtils, Logging} import kafka.utils.Implicits._ -import kafka.zk.{AdminZkClient, KafkaZkClient} -import org.apache.kafka.clients.admin.{AlterConfigOp, ConfigEntry} +import kafka.utils.{CoreUtils, Logging} import org.apache.kafka.clients.admin.AlterConfigOp.OpType -import org.apache.kafka.common.acl.{AclBinding, AclOperation} +import org.apache.kafka.clients.admin.{AlterConfigOp, ConfigEntry} import org.apache.kafka.common.acl.AclOperation._ +import org.apache.kafka.common.acl.{AclBinding, AclOperation} import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.errors._ -import org.apache.kafka.common.internals.{FatalExitError, Topic} import org.apache.kafka.common.internals.Topic.{GROUP_METADATA_TOPIC_NAME, TRANSACTION_STATE_TOPIC_NAME, isInternal} -import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic +import org.apache.kafka.common.internals.{FatalExitError, Topic} +import org.apache.kafka.common.message.AlterConfigsResponseData.AlterConfigsResourceResponse +import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData.{ReassignablePartitionResponse, ReassignableTopicResponse} +import org.apache.kafka.common.message.CreateAclsResponseData.AclCreationResult import org.apache.kafka.common.message.CreatePartitionsResponseData.CreatePartitionsTopicResult -import org.apache.kafka.common.message.{AddOffsetsToTxnResponseData, AlterClientQuotasResponseData, AlterConfigsResponseData, AlterPartitionReassignmentsResponseData, AlterReplicaLogDirsResponseData, ApiVersionsResponseData, CreateAclsResponseData, CreatePartitionsResponseData, CreateTopicsResponseData, DeleteAclsResponseData, DeleteGroupsResponseData, DeleteRecordsResponseData, DeleteTopicsResponseData, DescribeAclsResponseData, DescribeClientQuotasResponseData, DescribeClusterResponseData, DescribeConfigsResponseData, DescribeGroupsResponseData, DescribeLogDirsResponseData, DescribeProducersResponseData, EndTxnResponseData, ExpireDelegationTokenResponseData, FindCoordinatorResponseData, HeartbeatResponseData, InitProducerIdResponseData, JoinGroupResponseData, LeaveGroupResponseData, ListGroupsResponseData, ListOffsetsResponseData, ListPartitionReassignmentsResponseData, MetadataResponseData, OffsetCommitRequestData, OffsetCommitResponseData, OffsetDeleteResponseData, OffsetForLeaderEpochResponseData, RenewDelegationTokenResponseData, SaslAuthenticateResponseData, SaslHandshakeResponseData, StopReplicaResponseData, SyncGroupResponseData, UpdateMetadataResponseData} +import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic import org.apache.kafka.common.message.CreateTopicsResponseData.{CreatableTopicResult, CreatableTopicResultCollection} import org.apache.kafka.common.message.DeleteGroupsResponseData.{DeletableGroupResult, DeletableGroupResultCollection} -import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData.{ReassignablePartitionResponse, ReassignableTopicResponse} -import org.apache.kafka.common.message.CreateAclsResponseData.AclCreationResult -import org.apache.kafka.common.message.DeleteTopicsResponseData.{DeletableTopicResult, DeletableTopicResultCollection} import org.apache.kafka.common.message.DeleteRecordsResponseData.{DeleteRecordsPartitionResult, DeleteRecordsTopicResult} -import org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult -import org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult +import org.apache.kafka.common.message.DeleteTopicsResponseData.{DeletableTopicResult, DeletableTopicResultCollection} +import org.apache.kafka.common.message.ElectLeadersResponseData.{PartitionResult, ReplicaElectionResult} import org.apache.kafka.common.message.LeaveGroupResponseData.MemberResponse import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition import org.apache.kafka.common.message.ListOffsetsResponseData.{ListOffsetsPartitionResponse, ListOffsetsTopicResponse} +import org.apache.kafka.common.message.MetadataResponseData.{MetadataResponsePartition, MetadataResponseTopic} import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderTopic import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.{EpochEndOffset, OffsetForLeaderTopicResult, OffsetForLeaderTopicResultCollection} +import org.apache.kafka.common.message.{AddOffsetsToTxnResponseData, AlterClientQuotasResponseData, AlterConfigsResponseData, AlterPartitionReassignmentsResponseData, AlterReplicaLogDirsResponseData, ApiVersionsResponseData, CreateAclsResponseData, CreatePartitionsResponseData, CreateTopicsResponseData, DeleteAclsResponseData, DeleteGroupsResponseData, DeleteRecordsResponseData, DeleteTopicsResponseData, DescribeAclsResponseData, DescribeClientQuotasResponseData, DescribeClusterResponseData, DescribeConfigsResponseData, DescribeGroupsResponseData, DescribeLogDirsResponseData, DescribeProducersResponseData, EndTxnResponseData, ExpireDelegationTokenResponseData, FindCoordinatorResponseData, HeartbeatResponseData, InitProducerIdResponseData, JoinGroupResponseData, LeaveGroupResponseData, ListGroupsResponseData, ListOffsetsResponseData, ListPartitionReassignmentsResponseData, OffsetCommitRequestData, OffsetCommitResponseData, OffsetDeleteResponseData, OffsetForLeaderEpochResponseData, RenewDelegationTokenResponseData, SaslAuthenticateResponseData, SaslHandshakeResponseData, StopReplicaResponseData, SyncGroupResponseData, UpdateMetadataResponseData} import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.network.{ClientInformation, ListenerName, Send} import org.apache.kafka.common.protocol.{ApiKeys, Errors} @@ -79,32 +79,29 @@ import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.apache.kafka.common.security.token.delegation.{DelegationToken, TokenInformation} import org.apache.kafka.common.utils.{ProducerIdAndEpoch, Time} import org.apache.kafka.common.{Node, TopicPartition, Uuid} -import org.apache.kafka.common.message.AlterConfigsResponseData.AlterConfigsResourceResponse -import org.apache.kafka.common.message.MetadataResponseData.{MetadataResponsePartition, MetadataResponseTopic} import org.apache.kafka.server.authorizer._ -import scala.compat.java8.OptionConverters._ -import scala.jdk.CollectionConverters._ +import scala.annotation.nowarn import scala.collection.mutable.ArrayBuffer import scala.collection.{Map, Seq, Set, immutable, mutable} +import scala.compat.java8.OptionConverters._ +import scala.jdk.CollectionConverters._ import scala.util.{Failure, Success, Try} import kafka.coordinator.group.GroupOverview - -import scala.annotation.nowarn +import kafka.server.metadata.ConfigRepository /** * Logic to handle the various Kafka requests */ class KafkaApis(val requestChannel: RequestChannel, + val metadataSupport: MetadataSupport, val replicaManager: ReplicaManager, - val adminManager: ZkAdminManager, val groupCoordinator: GroupCoordinator, val txnCoordinator: TransactionCoordinator, - val controller: KafkaController, - val forwardingManager: Option[ForwardingManager], - val zkClient: KafkaZkClient, + val autoTopicCreationManager: AutoTopicCreationManager, val brokerId: Int, val config: KafkaConfig, + val configRepository: ConfigRepository, val metadataCache: MetadataCache, val metrics: Metrics, val authorizer: Option[Authorizer], @@ -117,9 +114,11 @@ class KafkaApis(val requestChannel: RequestChannel, val brokerFeatures: BrokerFeatures, val finalizedFeatureCache: FinalizedFeatureCache) extends ApiRequestHandler with Logging { + metadataSupport.ensureConsistentWith(config) + type FetchResponseStats = Map[TopicPartition, RecordConversionStats] this.logIdent = "[KafkaApi-%d] ".format(brokerId) - val adminZkClient = new AdminZkClient(zkClient) + val configHelper = new ConfigHelper(metadataCache, config, configRepository) private val alterAclsPurgatory = new DelayedFuturePurgatory(purgatoryName = "AlterAcls", brokerId = config.brokerId) val authHelper = new AuthHelper(authorizer) @@ -131,7 +130,7 @@ class KafkaApis(val requestChannel: RequestChannel, } private def isForwardingEnabled(request: RequestChannel.Request): Boolean = { - forwardingManager.isDefined && request.context.principalSerde.isPresent + metadataSupport.forwardingManager.isDefined && request.context.principalSerde.isPresent } private def maybeForwardToController( @@ -149,13 +148,7 @@ class KafkaApis(val requestChannel: RequestChannel, } } - forwardingManager match { - case Some(mgr) if !request.isForwarded && !controller.isActive => - mgr.forwardRequest(request, responseCallback) - - case _ => - handler(request) - } + metadataSupport.maybeForward(request, handler, responseCallback) } /** @@ -252,6 +245,7 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleLeaderAndIsrRequest(request: RequestChannel.Request): Unit = { + val zkSupport = metadataSupport.requireZkOrThrow(KafkaApis.shouldNeverReceive(request)) // ensureTopicExists is only for client facing requests // We can't have the ensureTopicExists check here since the controller sends it as an advisory to all brokers so they // stop serving data to clients for the topic being deleted @@ -259,11 +253,11 @@ class KafkaApis(val requestChannel: RequestChannel, val leaderAndIsrRequest = request.body[LeaderAndIsrRequest] authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) - if (isBrokerEpochStale(leaderAndIsrRequest.brokerEpoch)) { + if (isBrokerEpochStale(zkSupport, leaderAndIsrRequest.brokerEpoch)) { // When the broker restarts very quickly, it is possible for this broker to receive request intended // for its previous generation so the broker should skip the stale request. info("Received LeaderAndIsr request with broker epoch " + - s"${leaderAndIsrRequest.brokerEpoch} smaller than the current broker epoch ${controller.brokerEpoch}") + s"${leaderAndIsrRequest.brokerEpoch} smaller than the current broker epoch ${zkSupport.controller.brokerEpoch}") requestHelper.sendResponseExemptThrottle(request, leaderAndIsrRequest.getErrorResponse(0, Errors.STALE_BROKER_EPOCH.exception)) } else { val response = replicaManager.becomeLeaderOrFollower(correlationId, leaderAndIsrRequest, @@ -273,16 +267,17 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleStopReplicaRequest(request: RequestChannel.Request): Unit = { + val zkSupport = metadataSupport.requireZkOrThrow(KafkaApis.shouldNeverReceive(request)) // ensureTopicExists is only for client facing requests // We can't have the ensureTopicExists check here since the controller sends it as an advisory to all brokers so they // stop serving data to clients for the topic being deleted val stopReplicaRequest = request.body[StopReplicaRequest] authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) - if (isBrokerEpochStale(stopReplicaRequest.brokerEpoch)) { + if (isBrokerEpochStale(zkSupport, stopReplicaRequest.brokerEpoch)) { // When the broker restarts very quickly, it is possible for this broker to receive request intended // for its previous generation so the broker should skip the stale request. info("Received StopReplica request with broker epoch " + - s"${stopReplicaRequest.brokerEpoch} smaller than the current broker epoch ${controller.brokerEpoch}") + s"${stopReplicaRequest.brokerEpoch} smaller than the current broker epoch ${zkSupport.controller.brokerEpoch}") requestHelper.sendResponseExemptThrottle(request, new StopReplicaResponse( new StopReplicaResponseData().setErrorCode(Errors.STALE_BROKER_EPOCH.code))) } else { @@ -329,15 +324,16 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleUpdateMetadataRequest(request: RequestChannel.Request): Unit = { + val zkSupport = metadataSupport.requireZkOrThrow(KafkaApis.shouldNeverReceive(request)) val correlationId = request.header.correlationId val updateMetadataRequest = request.body[UpdateMetadataRequest] authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) - if (isBrokerEpochStale(updateMetadataRequest.brokerEpoch)) { + if (isBrokerEpochStale(zkSupport, updateMetadataRequest.brokerEpoch)) { // When the broker restarts very quickly, it is possible for this broker to receive request intended // for its previous generation so the broker should skip the stale request. info("Received update metadata request with broker epoch " + - s"${updateMetadataRequest.brokerEpoch} smaller than the current broker epoch ${controller.brokerEpoch}") + s"${updateMetadataRequest.brokerEpoch} smaller than the current broker epoch ${zkSupport.controller.brokerEpoch}") requestHelper.sendResponseExemptThrottle(request, new UpdateMetadataResponse(new UpdateMetadataResponseData().setErrorCode(Errors.STALE_BROKER_EPOCH.code))) } else { @@ -345,9 +341,9 @@ class KafkaApis(val requestChannel: RequestChannel, if (deletedPartitions.nonEmpty) groupCoordinator.handleDeletedPartitions(deletedPartitions) - if (adminManager.hasDelayedTopicOperations) { + if (zkSupport.adminManager.hasDelayedTopicOperations) { updateMetadataRequest.partitionStates.forEach { partitionState => - adminManager.tryCompleteDelayedTopicOperations(partitionState.topicName) + zkSupport.adminManager.tryCompleteDelayedTopicOperations(partitionState.topicName) } } quotas.clientQuotaCallback.foreach { callback => @@ -370,6 +366,7 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleControlledShutdownRequest(request: RequestChannel.Request): Unit = { + val zkSupport = metadataSupport.requireZkOrThrow(KafkaApis.shouldNeverReceive(request)) // ensureTopicExists is only for client facing requests // We can't have the ensureTopicExists check here since the controller sends it as an advisory to all brokers so they // stop serving data to clients for the topic being deleted @@ -386,7 +383,7 @@ class KafkaApis(val requestChannel: RequestChannel, } requestHelper.sendResponseExemptThrottle(request, response) } - controller.controlledShutdown(controlledShutdownRequest.data.brokerId, controlledShutdownRequest.data.brokerEpoch, controlledShutdownCallback) + zkSupport.controller.controlledShutdown(controlledShutdownRequest.data.brokerId, controlledShutdownRequest.data.brokerEpoch, controlledShutdownCallback) } /** @@ -459,6 +456,7 @@ class KafkaApis(val requestChannel: RequestChannel, sendResponseCallback(Map.empty) else if (header.apiVersion == 0) { // for version 0 always store offsets to ZK + val zkSupport = metadataSupport.requireZkOrThrow(KafkaApis.unsupported("Version 0 offset commit requests")) val responseInfo = authorizedTopicRequestInfo.map { case (topicPartition, partitionData) => try { @@ -466,7 +464,7 @@ class KafkaApis(val requestChannel: RequestChannel, && partitionData.committedMetadata().length > config.offsetMetadataMaxSize) (topicPartition, Errors.OFFSET_METADATA_TOO_LARGE) else { - zkClient.setOrCreateConsumerOffset( + zkSupport.zkClient.setOrCreateConsumerOffset( offsetCommitRequest.data.groupId, topicPartition, partitionData.committedOffset) @@ -1088,24 +1086,9 @@ class KafkaApis(val requestChannel: RequestChannel, (responseTopics ++ unauthorizedResponseStatus).toList } - private def createTopic(topic: String, - numPartitions: Int, - replicationFactor: Int, - properties: util.Properties = new util.Properties()): MetadataResponseTopic = { - try { - adminZkClient.createTopic(topic, numPartitions, replicationFactor, properties, RackAwareMode.Safe, config.usesTopicId) - info("Auto creation of topic %s with %d partitions and replication factor %d is successful" - .format(topic, numPartitions, replicationFactor)) - metadataResponseTopic(Errors.LEADER_NOT_AVAILABLE, topic, isInternal(topic), util.Collections.emptyList()) - } catch { - case _: TopicExistsException => // let it go, possibly another broker created this topic - metadataResponseTopic(Errors.LEADER_NOT_AVAILABLE, topic, isInternal(topic), util.Collections.emptyList()) - case ex: Throwable => // Catch all to prevent unhandled errors - metadataResponseTopic(Errors.forException(ex), topic, isInternal(topic), util.Collections.emptyList()) - } - } - - private def metadataResponseTopic(error: Errors, topic: String, isInternal: Boolean, + private def metadataResponseTopic(error: Errors, + topic: String, + isInternal: Boolean, partitionData: util.List[MetadataResponsePartition]): MetadataResponseTopic = { new MetadataResponseTopic() .setErrorCode(error.code) @@ -1114,82 +1097,48 @@ class KafkaApis(val requestChannel: RequestChannel, .setPartitions(partitionData) } - private def createInternalTopic(topic: String): MetadataResponseTopic = { - if (topic == null) - throw new IllegalArgumentException("topic must not be null") - - val aliveBrokers = metadataCache.getAliveBrokers - - topic match { - case GROUP_METADATA_TOPIC_NAME => - if (aliveBrokers.size < config.offsetsTopicReplicationFactor) { - error(s"Number of alive brokers '${aliveBrokers.size}' does not meet the required replication factor " + - s"'${config.offsetsTopicReplicationFactor}' for the offsets topic (configured via " + - s"'${KafkaConfig.OffsetsTopicReplicationFactorProp}'). This error can be ignored if the cluster is starting up " + - s"and not all brokers are up yet.") - metadataResponseTopic(Errors.COORDINATOR_NOT_AVAILABLE, topic, true, util.Collections.emptyList()) - } else { - createTopic(topic, config.offsetsTopicPartitions, config.offsetsTopicReplicationFactor.toInt, - groupCoordinator.offsetsTopicConfigs) - } - case TRANSACTION_STATE_TOPIC_NAME => - if (aliveBrokers.size < config.transactionTopicReplicationFactor) { - error(s"Number of alive brokers '${aliveBrokers.size}' does not meet the required replication factor " + - s"'${config.transactionTopicReplicationFactor}' for the transactions state topic (configured via " + - s"'${KafkaConfig.TransactionsTopicReplicationFactorProp}'). This error can be ignored if the cluster is starting up " + - s"and not all brokers are up yet.") - metadataResponseTopic(Errors.COORDINATOR_NOT_AVAILABLE, topic, true, util.Collections.emptyList()) - } else { - createTopic(topic, config.transactionTopicPartitions, config.transactionTopicReplicationFactor.toInt, - txnCoordinator.transactionTopicConfigs) - } - case _ => throw new IllegalArgumentException(s"Unexpected internal topic name: $topic") - } - } - - private def getOrCreateInternalTopic(topic: String, listenerName: ListenerName): MetadataResponseData.MetadataResponseTopic = { - val topicMetadata = metadataCache.getTopicMetadata(Set(topic), listenerName) - topicMetadata.headOption.getOrElse(createInternalTopic(topic)) - } - - private def getTopicMetadata(allowAutoTopicCreation: Boolean, isFetchAllMetadata: Boolean, - topics: Set[String], listenerName: ListenerName, - errorUnavailableEndpoints: Boolean, - errorUnavailableListeners: Boolean): Seq[MetadataResponseTopic] = { + private def getTopicMetadata( + request: RequestChannel.Request, + fetchAllTopics: Boolean, + allowAutoTopicCreation: Boolean, + topics: Set[String], + listenerName: ListenerName, + errorUnavailableEndpoints: Boolean, + errorUnavailableListeners: Boolean + ): Seq[MetadataResponseTopic] = { val topicResponses = metadataCache.getTopicMetadata(topics, listenerName, - errorUnavailableEndpoints, errorUnavailableListeners) + errorUnavailableEndpoints, errorUnavailableListeners) - if (topics.isEmpty || topicResponses.size == topics.size) { + if (topics.isEmpty || topicResponses.size == topics.size || fetchAllTopics) { topicResponses } else { - val nonExistentTopics = topics.diff(topicResponses.map(_.name).toSet) - val responsesForNonExistentTopics = nonExistentTopics.flatMap { topic => - if (isInternal(topic)) { - val topicMetadata = createInternalTopic(topic) - Some( - if (topicMetadata.errorCode == Errors.COORDINATOR_NOT_AVAILABLE.code) - metadataResponseTopic(Errors.INVALID_REPLICATION_FACTOR, topic, true, util.Collections.emptyList()) - else - topicMetadata + val nonExistingTopics = topics.diff(topicResponses.map(_.name).toSet) + val nonExistingTopicResponses = if (allowAutoTopicCreation) { + val controllerMutationQuota = quotas.controllerMutation.newPermissiveQuotaFor(request) + autoTopicCreationManager.createTopics(nonExistingTopics, controllerMutationQuota) + } else { + nonExistingTopics.map { topic => + val error = try { + Topic.validate(topic) + Errors.UNKNOWN_TOPIC_OR_PARTITION + } catch { + case _: InvalidTopicException => + Errors.INVALID_TOPIC_EXCEPTION + } + + metadataResponseTopic( + error, + topic, + Topic.isInternal(topic), + util.Collections.emptyList() ) - } else if (isFetchAllMetadata) { - // A metadata request for all topics should never result in topic auto creation, but a topic may be deleted - // in between the creation of the topics parameter and topicResponses, so make sure to return None for this case. - None - } else if (allowAutoTopicCreation && config.autoCreateTopicsEnable) { - Some(createTopic(topic, config.numPartitions, config.defaultReplicationFactor)) - } else { - Some(metadataResponseTopic(Errors.UNKNOWN_TOPIC_OR_PARTITION, topic, false, util.Collections.emptyList())) } } - topicResponses ++ responsesForNonExistentTopics + topicResponses ++ nonExistingTopicResponses } } - /** - * Handle a topic metadata request - */ def handleTopicMetadataRequest(request: RequestChannel.Request): Unit = { val metadataRequest = request.body[MetadataRequest] val requestVersion = request.header.apiVersion @@ -1219,7 +1168,6 @@ class KafkaApis(val requestChannel: RequestChannel, val unauthorizedForCreateTopicMetadata = unauthorizedForCreateTopics.map(topic => metadataResponseTopic(Errors.TOPIC_AUTHORIZATION_FAILED, topic, isInternal(topic), util.Collections.emptyList())) - // do not disclose the existence of topics unauthorized for Describe, so we've not even checked if they exist or not val unauthorizedForDescribeTopicMetadata = // In case of all topics, don't include topics unauthorized for Describe @@ -1235,19 +1183,10 @@ class KafkaApis(val requestChannel: RequestChannel, // In versions 5 and below, we returned LEADER_NOT_AVAILABLE if a matching listener was not found on the leader. // From version 6 onwards, we return LISTENER_NOT_FOUND to enable diagnosis of configuration errors. val errorUnavailableListeners = requestVersion >= 6 - val topicMetadata = - if (authorizedTopics.isEmpty) - Seq.empty[MetadataResponseTopic] - else { - getTopicMetadata( - metadataRequest.allowAutoTopicCreation, - metadataRequest.isAllTopics, - authorizedTopics, - request.context.listenerName, - errorUnavailableEndpoints, - errorUnavailableListeners - ) - } + + val allowAutoCreation = config.autoCreateTopicsEnable && metadataRequest.allowAutoTopicCreation && !metadataRequest.isAllTopics + val topicMetadata = getTopicMetadata(request, metadataRequest.isAllTopics, allowAutoCreation, authorizedTopics, + request.context.listenerName, errorUnavailableEndpoints, errorUnavailableListeners) var clusterAuthorizedOperations = Int.MinValue // Default value in the schema if (requestVersion >= 8) { @@ -1263,9 +1202,12 @@ class KafkaApis(val requestChannel: RequestChannel, // get topic authorized operations if (metadataRequest.data.includeTopicAuthorizedOperations) { - topicMetadata.foreach { topicData => - topicData.setTopicAuthorizedOperations(authHelper.authorizedOperations(request, new Resource(ResourceType.TOPIC, topicData.name))) + def setTopicAuthorizedOperations(topicMetadata: Seq[MetadataResponseTopic]): Unit = { + topicMetadata.foreach { topicData => + topicData.setTopicAuthorizedOperations(authHelper.authorizedOperations(request, new Resource(ResourceType.TOPIC, topicData.name))) + } } + setTopicAuthorizedOperations(topicMetadata) } } @@ -1280,7 +1222,7 @@ class KafkaApis(val requestChannel: RequestChannel, MetadataResponse.prepareResponse( requestVersion, requestThrottleMs, - brokers.flatMap(_.getNode(request.context.listenerName)).asJava, + brokers.flatMap(_.endpoints.get(request.context.listenerName.value())).toList.asJava, clusterId, metadataCache.getControllerId.getOrElse(MetadataResponse.NO_CONTROLLER_ID), completeTopicMetadata.asJava, @@ -1305,6 +1247,7 @@ class KafkaApis(val requestChannel: RequestChannel, offsetFetchRequest.getErrorResponse(requestThrottleMs, Errors.GROUP_AUTHORIZATION_FAILED) else { if (header.apiVersion == 0) { + val zkSupport = metadataSupport.requireZkOrThrow(KafkaApis.unsupported("Version 0 offset fetch requests")) val (authorizedPartitions, unauthorizedPartitions) = partitionByAuthorized( offsetFetchRequest.partitions.asScala) @@ -1314,7 +1257,7 @@ class KafkaApis(val requestChannel: RequestChannel, if (!metadataCache.contains(topicPartition)) (topicPartition, OffsetFetchResponse.UNKNOWN_PARTITION) else { - val payloadOpt = zkClient.getConsumerOffset(offsetFetchRequest.groupId, topicPartition) + val payloadOpt = zkSupport.zkClient.getConsumerOffset(offsetFetchRequest.groupId, topicPartition) payloadOpt match { case Some(payload) => (topicPartition, new OffsetFetchResponse.PartitionData(payload.toLong, @@ -1376,55 +1319,58 @@ class KafkaApis(val requestChannel: RequestChannel, !authHelper.authorize(request.context, DESCRIBE, TRANSACTIONAL_ID, findCoordinatorRequest.data.key)) requestHelper.sendErrorResponseMaybeThrottle(request, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception) else { - // get metadata (and create the topic if necessary) - val (partition, topicMetadata) = CoordinatorType.forId(findCoordinatorRequest.data.keyType) match { + val (partition, internalTopicName) = CoordinatorType.forId(findCoordinatorRequest.data.keyType) match { case CoordinatorType.GROUP => - val partition = groupCoordinator.partitionFor(findCoordinatorRequest.data.key) - val metadata = getOrCreateInternalTopic(GROUP_METADATA_TOPIC_NAME, request.context.listenerName) - (partition, metadata) + (groupCoordinator.partitionFor(findCoordinatorRequest.data.key), GROUP_METADATA_TOPIC_NAME) case CoordinatorType.TRANSACTION => - val partition = txnCoordinator.partitionFor(findCoordinatorRequest.data.key) - val metadata = getOrCreateInternalTopic(TRANSACTION_STATE_TOPIC_NAME, request.context.listenerName) - (partition, metadata) + (txnCoordinator.partitionFor(findCoordinatorRequest.data.key), TRANSACTION_STATE_TOPIC_NAME) + } - case _ => - throw new InvalidRequestException("Unknown coordinator type in FindCoordinator request") + val topicMetadata = metadataCache.getTopicMetadata(Set(internalTopicName), request.context.listenerName) + def createFindCoordinatorResponse(error: Errors, + node: Node, + requestThrottleMs: Int): FindCoordinatorResponse = { + new FindCoordinatorResponse( + new FindCoordinatorResponseData() + .setErrorCode(error.code) + .setErrorMessage(error.message()) + .setNodeId(node.id) + .setHost(node.host) + .setPort(node.port) + .setThrottleTimeMs(requestThrottleMs)) } - def createResponse(requestThrottleMs: Int): AbstractResponse = { - def createFindCoordinatorResponse(error: Errors, node: Node): FindCoordinatorResponse = { - new FindCoordinatorResponse( - new FindCoordinatorResponseData() - .setErrorCode(error.code) - .setErrorMessage(error.message) - .setNodeId(node.id) - .setHost(node.host) - .setPort(node.port) - .setThrottleTimeMs(requestThrottleMs)) - } - val responseBody = if (topicMetadata.errorCode != Errors.NONE.code) { - createFindCoordinatorResponse(Errors.COORDINATOR_NOT_AVAILABLE, Node.noNode) - } else { - val coordinatorEndpoint = topicMetadata.partitions.asScala - .find(_.partitionIndex == partition) - .filter(_.leaderId != MetadataResponse.NO_LEADER_ID) - .flatMap(metadata => metadataCache.getAliveBroker(metadata.leaderId)) - .flatMap(_.getNode(request.context.listenerName)) - .filterNot(_.isEmpty) - - coordinatorEndpoint match { - case Some(endpoint) => - createFindCoordinatorResponse(Errors.NONE, endpoint) - case _ => - createFindCoordinatorResponse(Errors.COORDINATOR_NOT_AVAILABLE, Node.noNode) + if (topicMetadata.headOption.isEmpty) { + val controllerMutationQuota = quotas.controllerMutation.newPermissiveQuotaFor(request) + autoTopicCreationManager.createTopics(Seq(internalTopicName).toSet, controllerMutationQuota) + requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => createFindCoordinatorResponse( + Errors.COORDINATOR_NOT_AVAILABLE, Node.noNode, requestThrottleMs)) + } else { + def createResponse(requestThrottleMs: Int): AbstractResponse = { + val responseBody = if (topicMetadata.head.errorCode != Errors.NONE.code) { + createFindCoordinatorResponse(Errors.COORDINATOR_NOT_AVAILABLE, Node.noNode, requestThrottleMs) + } else { + val coordinatorEndpoint = topicMetadata.head.partitions.asScala + .find(_.partitionIndex == partition) + .filter(_.leaderId != MetadataResponse.NO_LEADER_ID) + .flatMap(metadata => metadataCache.getAliveBroker(metadata.leaderId)) + .flatMap(_.endpoints.get(request.context.listenerName.value())) + .filterNot(_.isEmpty) + + coordinatorEndpoint match { + case Some(endpoint) => + createFindCoordinatorResponse(Errors.NONE, endpoint, requestThrottleMs) + case _ => + createFindCoordinatorResponse(Errors.COORDINATOR_NOT_AVAILABLE, Node.noNode, requestThrottleMs) + } } + trace("Sending FindCoordinator response %s for correlation id %d to client %s." + .format(responseBody, request.header.correlationId, request.header.clientId)) + responseBody } - trace("Sending FindCoordinator response %s for correlation id %d to client %s." - .format(responseBody, request.header.correlationId, request.header.clientId)) - responseBody + requestHelper.sendResponseMaybeThrottle(request, createResponse) } - requestHelper.sendResponseMaybeThrottle(request, createResponse) } } @@ -1748,7 +1694,7 @@ class KafkaApis(val requestChannel: RequestChannel, else { val supportedFeatures = brokerFeatures.supportedFeatures val finalizedFeaturesOpt = finalizedFeatureCache.get - val controllerApiVersions = forwardingManager.flatMap(_.controllerApiVersions) + val controllerApiVersions = metadataSupport.forwardingManager.flatMap(_.controllerApiVersions) val apiVersionsResponse = finalizedFeaturesOpt match { @@ -1780,6 +1726,7 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleCreateTopicsRequest(request: RequestChannel.Request): Unit = { + val zkSupport = metadataSupport.requireZkOrThrow(KafkaApis.shouldAlwaysForward(request)) val controllerMutationQuota = quotas.controllerMutation.newQuotaFor(request, strictSinceVersion = 6) def sendResponseCallback(results: CreatableTopicResultCollection): Unit = { @@ -1797,7 +1744,7 @@ class KafkaApis(val requestChannel: RequestChannel, val createTopicsRequest = request.body[CreateTopicsRequest] val results = new CreatableTopicResultCollection(createTopicsRequest.data.topics.size) - if (!controller.isActive) { + if (!zkSupport.controller.isActive) { createTopicsRequest.data.topics.forEach { topic => results.add(new CreatableTopicResult().setName(topic.name) .setErrorCode(Errors.NOT_CONTROLLER.code)) @@ -1849,7 +1796,7 @@ class KafkaApis(val requestChannel: RequestChannel, } sendResponseCallback(results) } - adminManager.createTopics( + zkSupport.adminManager.createTopics( createTopicsRequest.data.timeoutMs, createTopicsRequest.data.validateOnly, toCreate, @@ -1860,6 +1807,7 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleCreatePartitionsRequest(request: RequestChannel.Request): Unit = { + val zkSupport = metadataSupport.requireZkOrThrow(KafkaApis.shouldAlwaysForward(request)) val createPartitionsRequest = request.body[CreatePartitionsRequest] val controllerMutationQuota = quotas.controllerMutation.newQuotaFor(request, strictSinceVersion = 3) @@ -1881,7 +1829,7 @@ class KafkaApis(val requestChannel: RequestChannel, requestHelper.sendResponseMaybeThrottleWithControllerQuota(controllerMutationQuota, request, createResponse) } - if (!controller.isActive) { + if (!zkSupport.controller.isActive) { val result = createPartitionsRequest.data.topics.asScala.map { topic => (topic.name, new ApiError(Errors.NOT_CONTROLLER, null)) }.toMap @@ -1897,14 +1845,14 @@ class KafkaApis(val requestChannel: RequestChannel, notDuped)(_.name) val (queuedForDeletion, valid) = authorized.partition { topic => - controller.topicDeletionManager.isTopicQueuedUpForDeletion(topic.name) + zkSupport.controller.topicDeletionManager.isTopicQueuedUpForDeletion(topic.name) } val errors = dupes.map(_ -> new ApiError(Errors.INVALID_REQUEST, "Duplicate topic in request.")) ++ unauthorized.map(_.name -> new ApiError(Errors.TOPIC_AUTHORIZATION_FAILED, "The topic authorization is failed.")) ++ queuedForDeletion.map(_.name -> new ApiError(Errors.INVALID_TOPIC_EXCEPTION, "The topic is queued for deletion.")) - adminManager.createPartitions( + zkSupport.adminManager.createPartitions( createPartitionsRequest.data.timeoutMs, valid, createPartitionsRequest.data.validateOnly, @@ -1914,6 +1862,7 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleDeleteTopicsRequest(request: RequestChannel.Request): Unit = { + val zkSupport = metadataSupport.requireZkOrThrow(KafkaApis.shouldAlwaysForward(request)) val controllerMutationQuota = quotas.controllerMutation.newQuotaFor(request, strictSinceVersion = 5) def sendResponseCallback(results: DeletableTopicResultCollection): Unit = { @@ -1931,7 +1880,7 @@ class KafkaApis(val requestChannel: RequestChannel, val deleteTopicRequest = request.body[DeleteTopicsRequest] val results = new DeletableTopicResultCollection(deleteTopicRequest.data.topicNames.size) val toDelete = mutable.Set[String]() - if (!controller.isActive) { + if (!zkSupport.controller.isActive) { deleteTopicRequest.topics().forEach { topic => results.add(new DeletableTopicResult() .setName(topic.name()) @@ -1954,7 +1903,7 @@ class KafkaApis(val requestChannel: RequestChannel, if (topic.name() != null && topic.topicId() != Uuid.ZERO_UUID) throw new InvalidRequestException("Topic name and topic ID can not both be specified.") val name = if (topic.topicId() == Uuid.ZERO_UUID) topic.name() - else controller.controllerContext.topicNames.getOrElse(topic.topicId(), null) + else zkSupport.controller.controllerContext.topicNames.getOrElse(topic.topicId(), null) results.add(new DeletableTopicResult() .setName(name) .setTopicId(topic.topicId())) @@ -1992,7 +1941,7 @@ class KafkaApis(val requestChannel: RequestChannel, sendResponseCallback(results) } - adminManager.deleteTopics( + zkSupport.adminManager.deleteTopics( deleteTopicRequest.data.timeoutMs, toDelete, controllerMutationQuota, @@ -2488,6 +2437,7 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleCreateAcls(request: RequestChannel.Request): Unit = { + metadataSupport.requireZkOrThrow(KafkaApis.shouldAlwaysForward(request)) authHelper.authorizeClusterOperation(request, ALTER) val createAclsRequest = request.body[CreateAclsRequest] @@ -2539,6 +2489,7 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleDeleteAcls(request: RequestChannel.Request): Unit = { + metadataSupport.requireZkOrThrow(KafkaApis.shouldAlwaysForward(request)) authHelper.authorizeClusterOperation(request, ALTER) val deleteAclsRequest = request.body[DeleteAclsRequest] authorizer match { @@ -2600,6 +2551,7 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleAlterConfigsRequest(request: RequestChannel.Request): Unit = { + val zkSupport = metadataSupport.requireZkOrThrow(KafkaApis.shouldAlwaysForward(request)) val alterConfigsRequest = request.body[AlterConfigsRequest] val (authorizedResources, unauthorizedResources) = alterConfigsRequest.configs.asScala.toMap.partition { case (resource, _) => resource.`type` match { @@ -2612,7 +2564,7 @@ class KafkaApis(val requestChannel: RequestChannel, case rt => throw new InvalidRequestException(s"Unexpected resource type $rt") } } - val authorizedResult = adminManager.alterConfigs(authorizedResources, alterConfigsRequest.validateOnly) + val authorizedResult = zkSupport.adminManager.alterConfigs(authorizedResources, alterConfigsRequest.validateOnly) val unauthorizedResult = unauthorizedResources.keys.map { resource => resource -> configsAuthorizationApiError(resource) } @@ -2632,6 +2584,7 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleAlterPartitionReassignmentsRequest(request: RequestChannel.Request): Unit = { + val zkSupport = metadataSupport.requireZkOrThrow(KafkaApis.shouldAlwaysForward(request)) authHelper.authorizeClusterOperation(request, ALTER) val alterPartitionReassignmentsRequest = request.body[AlterPartitionReassignmentsRequest] @@ -2669,10 +2622,11 @@ class KafkaApis(val requestChannel: RequestChannel, } }.toMap - controller.alterPartitionReassignments(reassignments, sendResponseCallback) + zkSupport.controller.alterPartitionReassignments(reassignments, sendResponseCallback) } def handleListPartitionReassignmentsRequest(request: RequestChannel.Request): Unit = { + val zkSupport = metadataSupport.requireZkOrThrow(KafkaApis.notYetSupported(request)) authHelper.authorizeClusterOperation(request, DESCRIBE) val listPartitionReassignmentsRequest = request.body[ListPartitionReassignmentsRequest] @@ -2713,7 +2667,7 @@ class KafkaApis(val requestChannel: RequestChannel, case _ => None } - controller.listPartitionReassignments(partitionsOpt, sendResponseCallback) + zkSupport.controller.listPartitionReassignments(partitionsOpt, sendResponseCallback) } private def configsAuthorizationApiError(resource: ConfigResource): ApiError = { @@ -2726,6 +2680,7 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleIncrementalAlterConfigsRequest(request: RequestChannel.Request): Unit = { + val zkSupport = metadataSupport.requireZkOrThrow(KafkaApis.shouldAlwaysForward(request)) val alterConfigsRequest = request.body[IncrementalAlterConfigsRequest] val configs = alterConfigsRequest.data.resources.iterator.asScala.map { alterConfigResource => @@ -2747,7 +2702,7 @@ class KafkaApis(val requestChannel: RequestChannel, } } - val authorizedResult = adminManager.incrementalAlterConfigs(authorizedResources, alterConfigsRequest.data.validateOnly) + val authorizedResult = zkSupport.adminManager.incrementalAlterConfigs(authorizedResources, alterConfigsRequest.data.validateOnly) val unauthorizedResult = unauthorizedResources.keys.map { resource => resource -> configsAuthorizationApiError(resource) } @@ -2767,7 +2722,7 @@ class KafkaApis(val requestChannel: RequestChannel, case rt => throw new InvalidRequestException(s"Unexpected resource type $rt for resource ${resource.resourceName}") } } - val authorizedConfigs = adminManager.describeConfigs(authorizedResources.toList, describeConfigsRequest.data.includeSynonyms, describeConfigsRequest.data.includeDocumentation) + val authorizedConfigs = configHelper.describeConfigs(authorizedResources.toList, describeConfigsRequest.data.includeSynonyms, describeConfigsRequest.data.includeDocumentation) val unauthorizedConfigs = unauthorizedResources.map { resource => val error = ConfigResource.Type.forId(resource.resourceType) match { case ConfigResource.Type.BROKER | ConfigResource.Type.BROKER_LOGGER => Errors.CLUSTER_AUTHORIZATION_FAILED @@ -2831,6 +2786,7 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleCreateTokenRequest(request: RequestChannel.Request): Unit = { + metadataSupport.requireZkOrThrow(KafkaApis.shouldAlwaysForward(request)) val createTokenRequest = request.body[CreateDelegationTokenRequest] // the callback for sending a create token response @@ -2865,6 +2821,7 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleRenewTokenRequest(request: RequestChannel.Request): Unit = { + metadataSupport.requireZkOrThrow(KafkaApis.shouldAlwaysForward(request)) val renewTokenRequest = request.body[RenewDelegationTokenRequest] // the callback for sending a renew token response @@ -2892,6 +2849,7 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleExpireTokenRequest(request: RequestChannel.Request): Unit = { + metadataSupport.requireZkOrThrow(KafkaApis.shouldAlwaysForward(request)) val expireTokenRequest = request.body[ExpireDelegationTokenRequest] // the callback for sending a expire token response @@ -2964,6 +2922,7 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleElectReplicaLeader(request: RequestChannel.Request): Unit = { + val zkSupport = metadataSupport.requireZkOrThrow(KafkaApis.notYetSupported(request)) val electionRequest = request.body[ElectLeadersRequest] @@ -3025,7 +2984,7 @@ class KafkaApis(val requestChannel: RequestChannel, } replicaManager.electLeaders( - controller, + zkSupport.controller, partitions, electionRequest.electionType, sendResponseCallback(ApiError.NONE), @@ -3093,10 +3052,11 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleDescribeClientQuotasRequest(request: RequestChannel.Request): Unit = { + val zkSupport = metadataSupport.requireZkOrThrow(KafkaApis.notYetSupported(request)) val describeClientQuotasRequest = request.body[DescribeClientQuotasRequest] if (authHelper.authorize(request.context, DESCRIBE_CONFIGS, CLUSTER, CLUSTER_NAME)) { - val result = adminManager.describeClientQuotas(describeClientQuotasRequest.filter) + val result = zkSupport.adminManager.describeClientQuotas(describeClientQuotasRequest.filter) val entriesData = result.iterator.map { case (quotaEntity, quotaValues) => val entityData = quotaEntity.entries.asScala.iterator.map { case (entityType, entityName) => @@ -3127,10 +3087,11 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleAlterClientQuotasRequest(request: RequestChannel.Request): Unit = { + val zkSupport = metadataSupport.requireZkOrThrow(KafkaApis.shouldAlwaysForward(request)) val alterClientQuotasRequest = request.body[AlterClientQuotasRequest] if (authHelper.authorize(request.context, ALTER_CONFIGS, CLUSTER, CLUSTER_NAME)) { - val result = adminManager.alterClientQuotas(alterClientQuotasRequest.entries.asScala, + val result = zkSupport.adminManager.alterClientQuotas(alterClientQuotasRequest.entries.asScala, alterClientQuotasRequest.validateOnly) val entriesData = result.iterator.map { case (quotaEntity, apiError) => @@ -3157,10 +3118,11 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleDescribeUserScramCredentialsRequest(request: RequestChannel.Request): Unit = { + val zkSupport = metadataSupport.requireZkOrThrow(KafkaApis.notYetSupported(request)) val describeUserScramCredentialsRequest = request.body[DescribeUserScramCredentialsRequest] if (authHelper.authorize(request.context, DESCRIBE, CLUSTER, CLUSTER_NAME)) { - val result = adminManager.describeUserScramCredentials( + val result = zkSupport.adminManager.describeUserScramCredentials( Option(describeUserScramCredentialsRequest.data.users).map(_.asScala.map(_.name).toList)) requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => new DescribeUserScramCredentialsResponse(result.setThrottleTimeMs(requestThrottleMs))) @@ -3171,13 +3133,14 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleAlterUserScramCredentialsRequest(request: RequestChannel.Request): Unit = { + val zkSupport = metadataSupport.requireZkOrThrow(KafkaApis.shouldAlwaysForward(request)) val alterUserScramCredentialsRequest = request.body[AlterUserScramCredentialsRequest] - if (!controller.isActive) { + if (!zkSupport.controller.isActive) { requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => alterUserScramCredentialsRequest.getErrorResponse(requestThrottleMs, Errors.NOT_CONTROLLER.exception)) } else if (authHelper.authorize(request.context, ALTER, CLUSTER, CLUSTER_NAME)) { - val result = adminManager.alterUserScramCredentials( + val result = zkSupport.adminManager.alterUserScramCredentials( alterUserScramCredentialsRequest.data.upsertions().asScala, alterUserScramCredentialsRequest.data.deletions().asScala) requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => new AlterUserScramCredentialsResponse(result.setThrottleTimeMs(requestThrottleMs))) @@ -3188,19 +3151,21 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleAlterIsrRequest(request: RequestChannel.Request): Unit = { + val zkSupport = metadataSupport.requireZkOrThrow(KafkaApis.shouldNeverReceive(request)) val alterIsrRequest = request.body[AlterIsrRequest] authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) - if (!controller.isActive) + if (!zkSupport.controller.isActive) requestHelper.sendResponseExemptThrottle(request, alterIsrRequest.getErrorResponse( AbstractResponse.DEFAULT_THROTTLE_TIME, Errors.NOT_CONTROLLER.exception)) else - controller.alterIsrs(alterIsrRequest.data, alterIsrResp => + zkSupport.controller.alterIsrs(alterIsrRequest.data, alterIsrResp => requestHelper.sendResponseExemptThrottle(request, new AlterIsrResponse(alterIsrResp)) ) } def handleUpdateFeatures(request: RequestChannel.Request): Unit = { + val zkSupport = metadataSupport.requireZkOrThrow(KafkaApis.shouldAlwaysForward(request)) val updateFeaturesRequest = request.body[UpdateFeaturesRequest] def sendResponseCallback(errors: Either[ApiError, Map[String, ApiError]]): Unit = { @@ -3223,12 +3188,12 @@ class KafkaApis(val requestChannel: RequestChannel, if (!authHelper.authorize(request.context, ALTER, CLUSTER, CLUSTER_NAME)) { sendResponseCallback(Left(new ApiError(Errors.CLUSTER_AUTHORIZATION_FAILED))) - } else if (!controller.isActive) { + } else if (!zkSupport.controller.isActive) { sendResponseCallback(Left(new ApiError(Errors.NOT_CONTROLLER))) } else if (!config.isFeatureVersioningSupported) { sendResponseCallback(Left(new ApiError(Errors.INVALID_REQUEST, "Feature versioning system is disabled."))) } else { - controller.updateFeatures(updateFeaturesRequest, sendResponseCallback) + zkSupport.controller.updateFeatures(updateFeaturesRequest, sendResponseCallback) } } @@ -3254,7 +3219,8 @@ class KafkaApis(val requestChannel: RequestChannel, .setControllerId(controllerId) .setClusterAuthorizedOperations(clusterAuthorizedOperations); - brokers.flatMap(_.getNode(request.context.listenerName)).foreach { broker => + + brokers.flatMap(_.endpoints.get(request.context.listenerName.value())).foreach { broker => data.brokers.add(new DescribeClusterResponseData.DescribeClusterBroker() .setBrokerId(broker.id) .setHost(broker.host) @@ -3267,6 +3233,7 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleEnvelope(request: RequestChannel.Request): Unit = { + val zkSupport = metadataSupport.requireZkOrThrow(KafkaApis.shouldNeverReceive(request)) val envelope = request.body[EnvelopeRequest] // If forwarding is not yet enabled or this request has been received on an invalid endpoint, @@ -3286,7 +3253,7 @@ class KafkaApis(val requestChannel: RequestChannel, requestHelper.sendErrorResponseMaybeThrottle(request, new ClusterAuthorizationException( s"Principal ${request.context.principal} does not have required CLUSTER_ACTION for envelope")) return - } else if (!controller.isActive) { + } else if (!zkSupport.controller.isActive) { requestHelper.sendErrorResponseMaybeThrottle(request, new NotControllerException( s"Broker $brokerId is not the active controller")) return @@ -3443,17 +3410,16 @@ class KafkaApis(val requestChannel: RequestChannel, request.temporaryMemoryBytes = conversionStats.temporaryMemoryBytes } - private def isBrokerEpochStale(brokerEpochInRequest: Long): Boolean = { + private def isBrokerEpochStale(zkSupport: ZkSupport, brokerEpochInRequest: Long): Boolean = { // Broker epoch in LeaderAndIsr/UpdateMetadata/StopReplica request is unknown // if the controller hasn't been upgraded to use KIP-380 if (brokerEpochInRequest == AbstractControlRequest.UNKNOWN_BROKER_EPOCH) false else { // brokerEpochInRequest > controller.brokerEpoch is possible in rare scenarios where the controller gets notified // about the new broker epoch and sends a control request with this epoch before the broker learns about it - brokerEpochInRequest < controller.brokerEpoch + brokerEpochInRequest < zkSupport.controller.brokerEpoch } } - } object KafkaApis { @@ -3465,4 +3431,26 @@ object KafkaApis { FetchResponse.sizeOf(versionId, unconvertedResponse.responseData.entrySet .iterator.asScala.filter(element => quota.isThrottled(element.getKey)).asJava) } + + // visible for testing + private[server] def shouldNeverReceive(request: RequestChannel.Request): Exception = { + new UnsupportedVersionException(s"Should never receive when using a Raft-based metadata quorum: ${request.header.apiKey()}") + } + + // visible for testing + private[server] def shouldAlwaysForward(request: RequestChannel.Request): Exception = { + new UnsupportedVersionException(s"Should always be forwarded to the Active Controller when using a Raft-based metadata quorum: ${request.header.apiKey}") + } + + private def unsupported(text: String): Exception = { + new UnsupportedVersionException(s"Unsupported when using a Raft-based metadata quorum: $text") + } + + private def notYetSupported(request: RequestChannel.Request): Exception = { + notYetSupported(request.header.apiKey().toString) + } + + private def notYetSupported(text: String): Exception = { + new UnsupportedVersionException(s"Not yet supported when using a Raft-based metadata quorum: $text") + } } diff --git a/core/src/main/scala/kafka/server/KafkaBroker.scala b/core/src/main/scala/kafka/server/KafkaBroker.scala new file mode 100644 index 0000000000..d47283e806 --- /dev/null +++ b/core/src/main/scala/kafka/server/KafkaBroker.scala @@ -0,0 +1,103 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server + +import java.util +import java.util.concurrent.atomic.AtomicReference + +import com.yammer.metrics.core.MetricName +import kafka.log.LogManager +import kafka.metrics.{KafkaMetricsGroup, KafkaYammerMetrics, LinuxIoMetricsCollector} +import kafka.network.SocketServer +import kafka.utils.KafkaScheduler +import org.apache.kafka.clients.CommonClientConfigs +import org.apache.kafka.common.ClusterResource +import org.apache.kafka.common.internals.ClusterResourceListeners +import org.apache.kafka.common.metrics.{KafkaMetricsContext, Metrics, MetricsReporter} +import org.apache.kafka.common.utils.Time +import org.apache.kafka.metadata.BrokerState +import org.apache.kafka.server.authorizer.Authorizer + +import scala.collection.Seq +import scala.jdk.CollectionConverters._ + +object KafkaBroker { + //properties for MetricsContext + val metricsPrefix: String = "kafka.server" + val metricsTypeName: String = "KafkaServer" + private val KAFKA_CLUSTER_ID: String = "kafka.cluster.id" + private val KAFKA_BROKER_ID: String = "kafka.broker.id" + + private[server] def createKafkaMetricsContext(clusterId: String, config: KafkaConfig): KafkaMetricsContext = { + val contextLabels = new util.HashMap[String, Object] + contextLabels.put(KAFKA_CLUSTER_ID, clusterId) + contextLabels.put(KAFKA_BROKER_ID, config.brokerId.toString) + contextLabels.putAll(config.originalsWithPrefix(CommonClientConfigs.METRICS_CONTEXT_PREFIX)) + val metricsContext = new KafkaMetricsContext(metricsPrefix, contextLabels) + metricsContext + } + + private[server] def notifyClusterListeners(clusterId: String, + clusterListeners: Seq[AnyRef]): Unit = { + val clusterResourceListeners = new ClusterResourceListeners + clusterResourceListeners.maybeAddAll(clusterListeners.asJava) + clusterResourceListeners.onUpdate(new ClusterResource(clusterId)) + } + + private[server] def notifyMetricsReporters(clusterId: String, + config: KafkaConfig, + metricsReporters: Seq[AnyRef]): Unit = { + val metricsContext = createKafkaMetricsContext(clusterId, config) + metricsReporters.foreach { + case x: MetricsReporter => x.contextChange(metricsContext) + case _ => //do nothing + } + } +} + +trait KafkaBroker extends KafkaMetricsGroup { + def authorizer: Option[Authorizer] + val brokerState = new AtomicReference[BrokerState](BrokerState.NOT_RUNNING) + def clusterId: String + def config: KafkaConfig + def dataPlaneRequestHandlerPool: KafkaRequestHandlerPool + def kafkaScheduler: KafkaScheduler + def kafkaYammerMetrics: KafkaYammerMetrics + def logManager: LogManager + def metrics: Metrics + def quotaManagers: QuotaFactory.QuotaManagers + def replicaManager: ReplicaManager + def socketServer: SocketServer + + // For backwards compatibility, we need to keep older metrics tied + // to their original name when this class was named `KafkaServer` + override def metricName(name: String, metricTags: scala.collection.Map[String, String]): MetricName = { + explicitMetricName(KafkaBroker.metricsPrefix, KafkaBroker.metricsTypeName, name, metricTags) + } + + newGauge("BrokerState", () => brokerState.get.value()) + newGauge("ClusterId", () => clusterId) + newGauge("yammer-metrics-count", () => KafkaYammerMetrics.defaultRegistry.allMetrics.size) + + private val linuxIoMetricsCollector = new LinuxIoMetricsCollector("/proc", Time.SYSTEM, logger.underlying) + + if (linuxIoMetricsCollector.usable()) { + newGauge("linux-disk-read-bytes", () => linuxIoMetricsCollector.readBytes()) + newGauge("linux-disk-write-bytes", () => linuxIoMetricsCollector.writeBytes()) + } +} diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 9eca82ee48..7aed40d178 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -20,7 +20,7 @@ package kafka.server import java.io.{File, IOException} import java.net.{InetAddress, SocketTimeoutException} import java.util.concurrent._ -import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger, AtomicReference} +import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} import kafka.api.{KAFKA_0_9_0, KAFKA_2_2_IV0, KAFKA_2_4_IV1} import kafka.cluster.Broker @@ -29,15 +29,16 @@ import kafka.controller.KafkaController import kafka.coordinator.group.GroupCoordinator import kafka.coordinator.transaction.{ProducerIdManager, TransactionCoordinator} import kafka.log.LogManager -import kafka.metrics.{KafkaMetricsGroup, KafkaMetricsReporter, KafkaYammerMetrics, LinuxIoMetricsCollector} +import kafka.metrics.{KafkaMetricsReporter, KafkaYammerMetrics} import kafka.network.SocketServer import kafka.security.CredentialProvider +import kafka.server.metadata.ZkConfigRepository import kafka.utils._ -import kafka.zk.{BrokerInfo, KafkaZkClient} +import kafka.zk.{AdminZkClient, BrokerInfo, KafkaZkClient} import org.apache.kafka.clients.{ApiVersions, ClientDnsLookup, ManualMetadataUpdater, NetworkClient, NetworkClientUtils} -import org.apache.kafka.common.internals.ClusterResourceListeners +import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.message.ControlledShutdownRequestData -import org.apache.kafka.common.metrics.{Metrics, MetricsReporter} +import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.network._ import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{ControlledShutdownRequest, ControlledShutdownResponse} @@ -45,7 +46,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.security.{JaasContext, JaasUtils} import org.apache.kafka.common.utils.{AppInfoParser, LogContext, Time} -import org.apache.kafka.common.{ClusterResource, Endpoint, Node} +import org.apache.kafka.common.{Endpoint, Node} import org.apache.kafka.metadata.BrokerState import org.apache.kafka.server.authorizer.Authorizer import org.apache.zookeeper.client.ZKClientConfig @@ -89,7 +90,7 @@ class KafkaServer( time: Time = Time.SYSTEM, threadNamePrefix: Option[String] = None, enableForwarding: Boolean = false -) extends Server with Logging with KafkaMetricsGroup { +) extends KafkaBroker with Server { private val startupComplete = new AtomicBoolean(false) private val isShuttingDown = new AtomicBoolean(false) @@ -103,8 +104,6 @@ class KafkaServer( var kafkaYammerMetrics: KafkaYammerMetrics = null var metrics: Metrics = null - val brokerState = new AtomicReference[BrokerState](BrokerState.NOT_RUNNING) - var dataPlaneRequestProcessor: KafkaApis = null var controlPlaneRequestProcessor: KafkaApis = null @@ -133,6 +132,8 @@ class KafkaServer( var forwardingManager: Option[ForwardingManager] = None + var autoTopicCreationManager: AutoTopicCreationManager = null + var alterIsrManager: AlterIsrManager = null var kafkaScheduler: KafkaScheduler = null @@ -142,6 +143,7 @@ class KafkaServer( val zkClientConfig: ZKClientConfig = KafkaServer.zkClientConfigFromKafkaConfig(config).getOrElse(new ZKClientConfig()) private var _zkClient: KafkaZkClient = null + private var configRepository: ZkConfigRepository = null val correlationId: AtomicInteger = new AtomicInteger(0) val brokerMetaPropsFile = "meta.properties" @@ -167,17 +169,6 @@ class KafkaServer( private[kafka] def featureChangeListener = _featureChangeListener - newGauge("BrokerState", () => brokerState.get().value()) - newGauge("ClusterId", () => clusterId) - newGauge("yammer-metrics-count", () => KafkaYammerMetrics.defaultRegistry.allMetrics.size) - - val linuxIoMetricsCollector = new LinuxIoMetricsCollector("/proc", time, logger.underlying) - - if (linuxIoMetricsCollector.usable()) { - newGauge("linux-disk-read-bytes", () => linuxIoMetricsCollector.readBytes()) - newGauge("linux-disk-write-bytes", () => linuxIoMetricsCollector.writeBytes()) - } - /** * Start up API for bringing up a single instance of the Kafka server. * Instantiates the LogManager, the SocketServer and the request handlers - KafkaRequestHandlers @@ -198,6 +189,7 @@ class KafkaServer( /* setup zookeeper */ initZkClient(time) + configRepository = new ZkConfigRepository(new AdminZkClient(zkClient)) /* initialize features */ _featureChangeListener = new FinalizedFeatureChangeListener(featureCache, _zkClient) @@ -207,7 +199,7 @@ class KafkaServer( /* Get or create cluster_id */ _clusterId = getOrGenerateClusterId(zkClient) - info(s"Cluster ID = $clusterId") + info(s"Cluster ID = ${clusterId}") /* load metadata */ val (preloadedBrokerMetadataCheckpoint, initialOfflineDirs) = @@ -247,15 +239,18 @@ class KafkaServer( _brokerTopicStats = new BrokerTopicStats quotaManagers = QuotaFactory.instantiate(config, metrics, time, threadNamePrefix.getOrElse("")) - notifyClusterListeners(kafkaMetricsReporters ++ metrics.reporters.asScala) + KafkaBroker.notifyClusterListeners(clusterId, kafkaMetricsReporters ++ metrics.reporters.asScala) logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size) /* start log manager */ - logManager = LogManager(config, initialOfflineDirs, zkClient, brokerState, kafkaScheduler, time, brokerTopicStats, logDirFailureChannel) - logManager.startup() + logManager = LogManager(config, initialOfflineDirs, + new ZkConfigRepository(new AdminZkClient(zkClient)), + kafkaScheduler, time, brokerTopicStats, logDirFailureChannel) + brokerState.set(BrokerState.RECOVERY) + logManager.startup(zkClient.getAllTopicsInCluster()) - metadataCache = new MetadataCache(config.brokerId) + metadataCache = MetadataCache.zkMetadataCache(config.brokerId) // Enable delegation token cache for all SCRAM mechanisms to simplify dynamic update. // This keeps the cache up-to-date if new SCRAM mechanisms are enabled dynamically. tokenCache = new DelegationTokenCache(ScramMechanism.mechanismNames) @@ -304,6 +299,7 @@ class KafkaServer( kafkaController = new KafkaController(config, zkClient, time, metrics, brokerInfo, brokerEpoch, tokenManager, brokerFeatures, featureCache, threadNamePrefix) kafkaController.startup() + /* start forwarding manager */ if (enableForwarding) { this.forwardingManager = Some(ForwardingManager( config, @@ -319,14 +315,30 @@ class KafkaServer( /* start group coordinator */ // Hardcode Time.SYSTEM for now as some Streams tests fail otherwise, it would be good to fix the underlying issue - groupCoordinator = GroupCoordinator(config, zkClient, replicaManager, Time.SYSTEM, metrics) - groupCoordinator.startup() + groupCoordinator = GroupCoordinator(config, replicaManager, Time.SYSTEM, metrics) + groupCoordinator.startup(() => zkClient.getTopicPartitionCount(Topic.GROUP_METADATA_TOPIC_NAME).getOrElse(config.offsetsTopicPartitions)) /* start transaction coordinator, with a separate background thread scheduler for transaction expiration and log loading */ // Hardcode Time.SYSTEM for now as some Streams tests fail otherwise, it would be good to fix the underlying issue transactionCoordinator = TransactionCoordinator(config, replicaManager, new KafkaScheduler(threads = 1, threadNamePrefix = "transaction-log-manager-"), - () => new ProducerIdManager(config.brokerId, zkClient), zkClient, metrics, metadataCache, Time.SYSTEM) - transactionCoordinator.startup() + () => new ProducerIdManager(config.brokerId, zkClient), metrics, metadataCache, Time.SYSTEM) + transactionCoordinator.startup( + () => zkClient.getTopicPartitionCount(Topic.TRANSACTION_STATE_TOPIC_NAME).getOrElse(config.transactionTopicPartitions)) + + /* start auto topic creation manager */ + this.autoTopicCreationManager = AutoTopicCreationManager( + config, + metadataCache, + time, + metrics, + threadNamePrefix, + adminManager, + kafkaController, + groupCoordinator, + transactionCoordinator, + enableForwarding + ) + autoTopicCreationManager.start() /* Get the authorizer and initialize it if one is specified.*/ authorizer = config.authorizer @@ -347,16 +359,17 @@ class KafkaServer( KafkaServer.MIN_INCREMENTAL_FETCH_SESSION_EVICTION_MS)) /* start processing requests */ - dataPlaneRequestProcessor = new KafkaApis(socketServer.dataPlaneRequestChannel, replicaManager, adminManager, groupCoordinator, transactionCoordinator, - kafkaController, forwardingManager, zkClient, config.brokerId, config, metadataCache, metrics, authorizer, quotaManagers, + val zkSupport = ZkSupport(adminManager, kafkaController, zkClient, forwardingManager) + dataPlaneRequestProcessor = new KafkaApis(socketServer.dataPlaneRequestChannel, zkSupport, replicaManager, groupCoordinator, transactionCoordinator, + autoTopicCreationManager, config.brokerId, config, configRepository, metadataCache, metrics, authorizer, quotaManagers, fetchManager, brokerTopicStats, clusterId, time, tokenManager, brokerFeatures, featureCache) dataPlaneRequestHandlerPool = new KafkaRequestHandlerPool(config.brokerId, socketServer.dataPlaneRequestChannel, dataPlaneRequestProcessor, time, config.numIoThreads, s"${SocketServer.DataPlaneMetricPrefix}RequestHandlerAvgIdlePercent", SocketServer.DataPlaneThreadPrefix) socketServer.controlPlaneRequestChannelOpt.foreach { controlPlaneRequestChannel => - controlPlaneRequestProcessor = new KafkaApis(controlPlaneRequestChannel, replicaManager, adminManager, groupCoordinator, transactionCoordinator, - kafkaController, forwardingManager, zkClient, config.brokerId, config, metadataCache, metrics, authorizer, quotaManagers, + controlPlaneRequestProcessor = new KafkaApis(controlPlaneRequestChannel, zkSupport, replicaManager, groupCoordinator, transactionCoordinator, + autoTopicCreationManager, config.brokerId, config, configRepository, metadataCache, metrics, authorizer, quotaManagers, fetchManager, brokerTopicStats, clusterId, time, tokenManager, brokerFeatures, featureCache) controlPlaneRequestHandlerPool = new KafkaRequestHandlerPool(config.brokerId, socketServer.controlPlaneRequestChannelOpt.get, controlPlaneRequestProcessor, time, @@ -398,23 +411,9 @@ class KafkaServer( } } - private[server] def notifyClusterListeners(clusterListeners: Seq[AnyRef]): Unit = { - val clusterResourceListeners = new ClusterResourceListeners - clusterResourceListeners.maybeAddAll(clusterListeners.asJava) - clusterResourceListeners.onUpdate(new ClusterResource(clusterId)) - } - - private[server] def notifyMetricsReporters(metricsReporters: Seq[AnyRef]): Unit = { - val metricsContext = Server.createKafkaMetricsContext(config, clusterId) - metricsReporters.foreach { - case x: MetricsReporter => x.contextChange(metricsContext) - case _ => //do nothing - } - } - protected def createReplicaManager(isShuttingDown: AtomicBoolean): ReplicaManager = { - new ReplicaManager(config, metrics, time, zkClient, kafkaScheduler, logManager, isShuttingDown, quotaManagers, - brokerTopicStats, metadataCache, logDirFailureChannel, alterIsrManager) + new ReplicaManager(config, metrics, time, Some(zkClient), kafkaScheduler, logManager, isShuttingDown, quotaManagers, + brokerTopicStats, metadataCache, logDirFailureChannel, alterIsrManager, configRepository) } private def initZkClient(time: Time): Unit = { @@ -696,8 +695,10 @@ class KafkaServer( if (alterIsrManager != null) CoreUtils.swallow(alterIsrManager.shutdown(), this) - if (forwardingManager != null) - CoreUtils.swallow(forwardingManager.foreach(_.shutdown()), this) + CoreUtils.swallow(forwardingManager.foreach(_.shutdown()), this) + + if (autoTopicCreationManager != null) + CoreUtils.swallow(autoTopicCreationManager.shutdown(), this) if (logManager != null) CoreUtils.swallow(logManager.shutdown(), this) diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala index 9abef3d668..6b7422ffbe 100755 --- a/core/src/main/scala/kafka/server/MetadataCache.scala +++ b/core/src/main/scala/kafka/server/MetadataCache.scala @@ -20,12 +20,12 @@ package kafka.server import java.util import java.util.Collections import java.util.concurrent.locks.ReentrantReadWriteLock - import scala.collection.{Seq, Set, mutable} import scala.jdk.CollectionConverters._ import kafka.cluster.{Broker, EndPoint} import kafka.api._ import kafka.controller.StateChangeLogger +import kafka.server.metadata.{MetadataBroker, RaftMetadataCache} import kafka.utils.CoreUtils._ import kafka.utils.Logging import kafka.utils.Implicits._ @@ -34,16 +34,88 @@ import org.apache.kafka.common.message.UpdateMetadataRequestData.UpdateMetadataP import org.apache.kafka.common.{Cluster, Node, PartitionInfo, TopicPartition, Uuid} import org.apache.kafka.common.message.MetadataResponseData.MetadataResponseTopic import org.apache.kafka.common.message.MetadataResponseData.MetadataResponsePartition +import org.apache.kafka.common.message.{MetadataResponseData, UpdateMetadataRequestData} import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{MetadataResponse, UpdateMetadataRequest} import org.apache.kafka.common.security.auth.SecurityProtocol +trait MetadataCache { + + /** + * Return topic metadata for a given set of topics and listener. See KafkaApis#handleTopicMetadataRequest for details + * on the use of the two boolean flags. + * + * @param topics The set of topics. + * @param listenerName The listener name. + * @param errorUnavailableEndpoints If true, we return an error on unavailable brokers. This is used to support + * MetadataResponse version 0. + * @param errorUnavailableListeners If true, return LEADER_NOT_AVAILABLE if the listener is not found on the leader. + * This is used for MetadataResponse versions 0-5. + * @return A collection of topic metadata. + */ + def getTopicMetadata( + topics: collection.Set[String], + listenerName: ListenerName, + errorUnavailableEndpoints: Boolean = false, + errorUnavailableListeners: Boolean = false): collection.Seq[MetadataResponseData.MetadataResponseTopic] + + def getAllTopics(): collection.Set[String] + + def getAllPartitions(): collection.Set[TopicPartition] + + def getNonExistingTopics(topics: collection.Set[String]): collection.Set[String] + + def getAliveBroker(brokerId: Int): Option[MetadataBroker] + + def getAliveBrokers: collection.Seq[MetadataBroker] + + def getPartitionInfo(topic: String, partitionId: Int): Option[UpdateMetadataRequestData.UpdateMetadataPartitionState] + + def numPartitions(topic: String): Option[Int] + + /** + * Get a partition leader's endpoint + * + * @return If the leader is known, and the listener name is available, return Some(node). If the the leader is known, + * but the listener is unavailable, return Some(Node.NO_NODE). Otherwise, if the leader is not known, + * return None + */ + def getPartitionLeaderEndpoint(topic: String, partitionId: Int, listenerName: ListenerName): Option[Node] + + def getPartitionReplicaEndpoints(tp: TopicPartition, listenerName: ListenerName): Map[Int, Node] + + def getControllerId: Option[Int] + + def getClusterMetadata(clusterId: String, listenerName: ListenerName): Cluster + + /** + * Update the metadata cache with a given UpdateMetadataRequest. + * + * @return The deleted topics from the given UpdateMetadataRequest. + */ + def updateMetadata(correlationId: Int, request: UpdateMetadataRequest): collection.Seq[TopicPartition] + + def contains(topic: String): Boolean + + def contains(tp: TopicPartition): Boolean +} + +object MetadataCache { + def zkMetadataCache(brokerId: Int): ZkMetadataCache = { + new ZkMetadataCache(brokerId) + } + + def raftMetadataCache(brokerId: Int): RaftMetadataCache = { + new RaftMetadataCache(brokerId) + } +} + /** * A cache for the state (e.g., current leader) of each partition. This cache is updated through * UpdateMetadataRequest from the controller. Every broker maintains the same cache, asynchronously. */ -class MetadataCache(brokerId: Int) extends Logging { +class ZkMetadataCache(brokerId: Int) extends MetadataCache with Logging { private val partitionMetadataLock = new ReentrantReadWriteLock() //this is the cache state. every MetadataSnapshot instance is immutable, and updates (performed under a lock) @@ -203,12 +275,12 @@ class MetadataCache(brokerId: Int) extends Logging { topics.diff(metadataSnapshot.partitionStates.keySet) } - def getAliveBroker(brokerId: Int): Option[Broker] = { - metadataSnapshot.aliveBrokers.get(brokerId) + def getAliveBroker(brokerId: Int): Option[MetadataBroker] = { + metadataSnapshot.aliveBrokers.get(brokerId).map(MetadataBroker.apply) } - def getAliveBrokers: Seq[Broker] = { - metadataSnapshot.aliveBrokers.values.toBuffer + def getAliveBrokers: Seq[MetadataBroker] = { + metadataSnapshot.aliveBrokers.values.map(MetadataBroker.apply).toBuffer } private def addOrUpdatePartitionInfo(partitionStates: mutable.AnyRefMap[String, mutable.LongMap[UpdateMetadataPartitionState]], @@ -299,9 +371,9 @@ class MetadataCache(brokerId: Int) extends Logging { val aliveBrokers = new mutable.LongMap[Broker](metadataSnapshot.aliveBrokers.size) val aliveNodes = new mutable.LongMap[collection.Map[ListenerName, Node]](metadataSnapshot.aliveNodes.size) val controllerIdOpt = updateMetadataRequest.controllerId match { - case id if id < 0 => None - case id => Some(id) - } + case id if id < 0 => None + case id => Some(id) + } updateMetadataRequest.liveBrokers.forEach { broker => // `aliveNodes` is a hot path for metadata requests for large clusters, so we use java.util.HashMap which diff --git a/core/src/main/scala/kafka/server/MetadataSupport.scala b/core/src/main/scala/kafka/server/MetadataSupport.scala new file mode 100644 index 0000000000..00b029f582 --- /dev/null +++ b/core/src/main/scala/kafka/server/MetadataSupport.scala @@ -0,0 +1,108 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server + +import kafka.controller.KafkaController +import kafka.network.RequestChannel +import kafka.zk.{AdminZkClient, KafkaZkClient} +import org.apache.kafka.common.requests.AbstractResponse + +sealed trait MetadataSupport { + /** + * Provide a uniform way of getting to the ForwardingManager, which is a shared concept + * despite being optional when using ZooKeeper and required when using Raft + */ + val forwardingManager: Option[ForwardingManager] + + /** + * Return this instance downcast for use with ZooKeeper + * + * @param createException function to create an exception to throw + * @return this instance downcast for use with ZooKeeper + * @throws Exception if this instance is not for ZooKeeper + */ + def requireZkOrThrow(createException: => Exception): ZkSupport + + /** + * Return this instance downcast for use with Raft + * + * @param createException function to create an exception to throw + * @return this instance downcast for use with Raft + * @throws Exception if this instance is not for Raft + */ + def requireRaftOrThrow(createException: => Exception): RaftSupport + + /** + * Confirm that this instance is consistent with the given config + * + * @param config the config to check for consistency with this instance + * @throws IllegalStateException if there is an inconsistency (Raft for a ZooKeeper config or vice-versa) + */ + def ensureConsistentWith(config: KafkaConfig): Unit + + def maybeForward(request: RequestChannel.Request, + handler: RequestChannel.Request => Unit, + responseCallback: Option[AbstractResponse] => Unit): Unit +} + +case class ZkSupport(adminManager: ZkAdminManager, + controller: KafkaController, + zkClient: KafkaZkClient, + forwardingManager: Option[ForwardingManager]) extends MetadataSupport { + val adminZkClient = new AdminZkClient(zkClient) + + override def requireZkOrThrow(createException: => Exception): ZkSupport = this + override def requireRaftOrThrow(createException: => Exception): RaftSupport = throw createException + + override def ensureConsistentWith(config: KafkaConfig): Unit = { + if (!config.requiresZookeeper) { + throw new IllegalStateException("Config specifies Raft but metadata support instance is for ZooKeeper") + } + } + + override def maybeForward(request: RequestChannel.Request, + handler: RequestChannel.Request => Unit, + responseCallback: Option[AbstractResponse] => Unit): Unit = { + forwardingManager match { + case Some(mgr) if !request.isForwarded && !controller.isActive => mgr.forwardRequest(request, responseCallback) + case _ => handler(request) + } + } +} + +case class RaftSupport(fwdMgr: ForwardingManager) extends MetadataSupport { + override val forwardingManager: Option[ForwardingManager] = Some(fwdMgr) + override def requireZkOrThrow(createException: => Exception): ZkSupport = throw createException + override def requireRaftOrThrow(createException: => Exception): RaftSupport = this + + override def ensureConsistentWith(config: KafkaConfig): Unit = { + if (config.requiresZookeeper) { + throw new IllegalStateException("Config specifies ZooKeeper but metadata support instance is for Raft") + } + } + + override def maybeForward(request: RequestChannel.Request, + handler: RequestChannel.Request => Unit, + responseCallback: Option[AbstractResponse] => Unit): Unit = { + if (!request.isForwarded) { + fwdMgr.forwardRequest(request, responseCallback) + } else { + handler(request) // will reject + } + } +} diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 8a31132b7f..e6397274e5 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -33,6 +33,7 @@ import kafka.server.{FetchMetadata => SFetchMetadata} import kafka.server.HostedPartition.Online import kafka.server.QuotaFactory.QuotaManagers import kafka.server.checkpoints.{LazyOffsetCheckpoints, OffsetCheckpointFile, OffsetCheckpoints} +import kafka.server.metadata.ConfigRepository import kafka.utils._ import kafka.utils.Implicits._ import kafka.zk.KafkaZkClient @@ -199,7 +200,7 @@ object ReplicaManager { class ReplicaManager(val config: KafkaConfig, metrics: Metrics, time: Time, - val zkClient: KafkaZkClient, + val zkClient: Option[KafkaZkClient], scheduler: Scheduler, val logManager: LogManager, val isShuttingDown: AtomicBoolean, @@ -212,12 +213,13 @@ class ReplicaManager(val config: KafkaConfig, val delayedDeleteRecordsPurgatory: DelayedOperationPurgatory[DelayedDeleteRecords], val delayedElectLeaderPurgatory: DelayedOperationPurgatory[DelayedElectLeader], threadNamePrefix: Option[String], + configRepository: ConfigRepository, val alterIsrManager: AlterIsrManager) extends Logging with KafkaMetricsGroup { def this(config: KafkaConfig, metrics: Metrics, time: Time, - zkClient: KafkaZkClient, + zkClient: Option[KafkaZkClient], scheduler: Scheduler, logManager: LogManager, isShuttingDown: AtomicBoolean, @@ -226,6 +228,7 @@ class ReplicaManager(val config: KafkaConfig, metadataCache: MetadataCache, logDirFailureChannel: LogDirFailureChannel, alterIsrManager: AlterIsrManager, + configRepository: ConfigRepository, threadNamePrefix: Option[String] = None) = { this(config, metrics, time, zkClient, scheduler, logManager, isShuttingDown, quotaManagers, brokerTopicStats, metadataCache, logDirFailureChannel, @@ -240,14 +243,14 @@ class ReplicaManager(val config: KafkaConfig, purgeInterval = config.deleteRecordsPurgatoryPurgeIntervalRequests), DelayedOperationPurgatory[DelayedElectLeader]( purgatoryName = "ElectLeader", brokerId = config.brokerId), - threadNamePrefix, alterIsrManager) + threadNamePrefix, configRepository, alterIsrManager) } /* epoch of the controller that last changed the leader */ @volatile var controllerEpoch: Int = KafkaController.InitialControllerEpoch private val localBrokerId = config.brokerId private val allPartitions = new Pool[TopicPartition, HostedPartition]( - valueFactory = Some(tp => HostedPartition.Online(Partition(tp, time, this))) + valueFactory = Some(tp => HostedPartition.Online(Partition(tp, time, configRepository, this))) ) private val replicaStateChangeLock = new Object val replicaFetcherManager = createReplicaFetcherManager(metrics, time, threadNamePrefix, quotaManagers.follower) @@ -514,7 +517,7 @@ class ReplicaManager(val config: KafkaConfig, // Visible for testing def createPartition(topicPartition: TopicPartition): Partition = { - val partition = Partition(topicPartition, time, this) + val partition = Partition(topicPartition, time, configRepository, this) allPartitions.put(topicPartition, HostedPartition.Online(partition)) partition } @@ -1369,7 +1372,7 @@ class ReplicaManager(val config: KafkaConfig, Some(partition) case HostedPartition.None => - val partition = Partition(topicPartition, time, this) + val partition = Partition(topicPartition, time, configRepository, this) allPartitions.putIfNotExists(topicPartition, HostedPartition.Online(partition)) Some(partition) } @@ -1881,7 +1884,11 @@ class ReplicaManager(val config: KafkaConfig, logManager.handleLogDirFailure(dir) if (sendZkNotification) - zkClient.propagateLogDirEvent(localBrokerId) + if (zkClient.isEmpty) { + warn("Unable to propagate log dir failure via Zookeeper in KIP-500 mode") // will be handled via KIP-589 + } else { + zkClient.get.propagateLogDirEvent(localBrokerId) + } warn(s"Stopped serving replicas in dir $dir") } diff --git a/core/src/main/scala/kafka/server/ZkAdminManager.scala b/core/src/main/scala/kafka/server/ZkAdminManager.scala index 3c96520e01..87f522fb10 100644 --- a/core/src/main/scala/kafka/server/ZkAdminManager.scala +++ b/core/src/main/scala/kafka/server/ZkAdminManager.scala @@ -17,7 +17,7 @@ package kafka.server import java.util -import java.util.{Collections, Properties} +import java.util.Properties import kafka.admin.{AdminOperationException, AdminUtils} import kafka.common.TopicAlreadyMarkedForDeletionException @@ -25,6 +25,7 @@ import kafka.log.LogConfig import kafka.utils.Log4jController import kafka.metrics.KafkaMetricsGroup import kafka.server.DynamicConfig.QuotaConfigs +import kafka.server.metadata.ZkConfigRepository import kafka.utils._ import kafka.utils.Implicits._ import kafka.zk.{AdminZkClient, KafkaZkClient} @@ -32,16 +33,14 @@ import org.apache.kafka.clients.admin.{AlterConfigOp, ScramMechanism} import org.apache.kafka.clients.admin.AlterConfigOp.OpType import org.apache.kafka.common.Uuid import org.apache.kafka.common.config.ConfigDef.ConfigKey -import org.apache.kafka.common.config.{AbstractConfig, ConfigDef, ConfigException, ConfigResource, LogLevelConfig} +import org.apache.kafka.common.config.{ConfigDef, ConfigException, ConfigResource, LogLevelConfig} import org.apache.kafka.common.errors.ThrottlingQuotaExceededException import org.apache.kafka.common.errors.{ApiException, InvalidConfigurationException, InvalidPartitionsException, InvalidReplicaAssignmentException, InvalidRequestException, ReassignmentInProgressException, TopicExistsException, UnknownTopicOrPartitionException, UnsupportedVersionException} -import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.message.AlterUserScramCredentialsResponseData.AlterUserScramCredentialsResult import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic import org.apache.kafka.common.message.CreateTopicsResponseData.{CreatableTopicConfigs, CreatableTopicResult} -import org.apache.kafka.common.message.{AlterUserScramCredentialsRequestData, AlterUserScramCredentialsResponseData, DescribeConfigsResponseData, DescribeUserScramCredentialsResponseData} -import org.apache.kafka.common.message.DescribeConfigsRequestData.DescribeConfigsResource +import org.apache.kafka.common.message.{AlterUserScramCredentialsRequestData, AlterUserScramCredentialsResponseData, DescribeUserScramCredentialsResponseData} import org.apache.kafka.common.message.DescribeUserScramCredentialsResponseData.CredentialInfo import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.security.scram.internals.{ScramMechanism => InternalScramMechanism} @@ -50,8 +49,7 @@ import org.apache.kafka.server.policy.CreateTopicPolicy.RequestMetadata import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.quota.{ClientQuotaAlteration, ClientQuotaEntity, ClientQuotaFilter, ClientQuotaFilterComponent} import org.apache.kafka.common.requests.CreateTopicsRequest._ -import org.apache.kafka.common.requests.DescribeConfigsResponse.ConfigSource -import org.apache.kafka.common.requests.{AlterConfigsRequest, ApiError, DescribeConfigsResponse} +import org.apache.kafka.common.requests.{AlterConfigsRequest, ApiError} import org.apache.kafka.common.security.scram.internals.{ScramCredentialUtils, ScramFormatter} import org.apache.kafka.common.utils.Sanitizer @@ -59,14 +57,15 @@ import scala.collection.{Map, mutable, _} import scala.jdk.CollectionConverters._ class ZkAdminManager(val config: KafkaConfig, - val metrics: Metrics, - val metadataCache: MetadataCache, - val zkClient: KafkaZkClient) extends Logging with KafkaMetricsGroup { + val metrics: Metrics, + val metadataCache: MetadataCache, + val zkClient: KafkaZkClient) extends Logging with KafkaMetricsGroup { this.logIdent = "[Admin Manager on Broker " + config.brokerId + "]: " private val topicPurgatory = DelayedOperationPurgatory[DelayedOperation]("topic", config.brokerId) private val adminZkClient = new AdminZkClient(zkClient) + private val configHelper = new ConfigHelper(metadataCache, config, new ZkConfigRepository(adminZkClient)) private val createTopicPolicy = Option(config.getConfiguredInstance(KafkaConfig.CreateTopicPolicyClassNameProp, classOf[CreateTopicPolicy])) @@ -118,7 +117,7 @@ class ZkAdminManager(val config: KafkaConfig, assignments: Map[Int, Seq[Int]]): Unit = { metadataAndConfigs.get(topicName).foreach { result => val logConfig = LogConfig.fromProps(LogConfig.extractLogConfigMap(config), configs) - val createEntry = createTopicConfigEntry(logConfig, configs, includeSynonyms = false, includeDocumentation = false)(_, _) + val createEntry = configHelper.createTopicConfigEntry(logConfig, configs, includeSynonyms = false, includeDocumentation = false)(_, _) val topicConfigs = logConfig.values.asScala.map { case (k, v) => val entry = createEntry(k, v) new CreatableTopicConfigs() @@ -153,7 +152,7 @@ class ZkAdminManager(val config: KafkaConfig, responseCallback: Map[String, ApiError] => Unit): Unit = { // 1. map over topics creating assignment and calling zookeeper - val brokers = metadataCache.getAliveBrokers.map { b => kafka.admin.BrokerMetadata(b.id, b.rack) } + val brokers = metadataCache.getAliveBrokers.map { b => kafka.admin.BrokerMetadata(b.id, Option(b.rack)) } val metadata = toCreate.values.map(topic => try { if (metadataCache.contains(topic.name)) @@ -389,86 +388,7 @@ class ZkAdminManager(val config: KafkaConfig, } } - def describeConfigs(resourceToConfigNames: List[DescribeConfigsResource], - includeSynonyms: Boolean, - includeDocumentation: Boolean): List[DescribeConfigsResponseData.DescribeConfigsResult] = { - resourceToConfigNames.map { case resource => - - def allConfigs(config: AbstractConfig) = { - config.originals.asScala.filter(_._2 != null) ++ config.nonInternalValues.asScala - } - def createResponseConfig(configs: Map[String, Any], - createConfigEntry: (String, Any) => DescribeConfigsResponseData.DescribeConfigsResourceResult): DescribeConfigsResponseData.DescribeConfigsResult = { - val filteredConfigPairs = if (resource.configurationKeys == null) - configs.toBuffer - else - configs.filter { case (configName, _) => - resource.configurationKeys.asScala.forall(_.contains(configName)) - }.toBuffer - - val configEntries = filteredConfigPairs.map { case (name, value) => createConfigEntry(name, value) } - new DescribeConfigsResponseData.DescribeConfigsResult().setErrorCode(Errors.NONE.code) - .setConfigs(configEntries.asJava) - } - - try { - val configResult = ConfigResource.Type.forId(resource.resourceType) match { - case ConfigResource.Type.TOPIC => - val topic = resource.resourceName - Topic.validate(topic) - if (metadataCache.contains(topic)) { - // Consider optimizing this by caching the configs or retrieving them from the `Log` when possible - val topicProps = adminZkClient.fetchEntityConfig(ConfigType.Topic, topic) - val logConfig = LogConfig.fromProps(LogConfig.extractLogConfigMap(config), topicProps) - createResponseConfig(allConfigs(logConfig), createTopicConfigEntry(logConfig, topicProps, includeSynonyms, includeDocumentation)) - } else { - new DescribeConfigsResponseData.DescribeConfigsResult().setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code) - .setConfigs(Collections.emptyList[DescribeConfigsResponseData.DescribeConfigsResourceResult]) - } - - case ConfigResource.Type.BROKER => - if (resource.resourceName == null || resource.resourceName.isEmpty) - createResponseConfig(config.dynamicConfig.currentDynamicDefaultConfigs, - createBrokerConfigEntry(perBrokerConfig = false, includeSynonyms, includeDocumentation)) - else if (resourceNameToBrokerId(resource.resourceName) == config.brokerId) - createResponseConfig(allConfigs(config), - createBrokerConfigEntry(perBrokerConfig = true, includeSynonyms, includeDocumentation)) - else - throw new InvalidRequestException(s"Unexpected broker id, expected ${config.brokerId} or empty string, but received ${resource.resourceName}") - - case ConfigResource.Type.BROKER_LOGGER => - if (resource.resourceName == null || resource.resourceName.isEmpty) - throw new InvalidRequestException("Broker id must not be empty") - else if (resourceNameToBrokerId(resource.resourceName) != config.brokerId) - throw new InvalidRequestException(s"Unexpected broker id, expected ${config.brokerId} but received ${resource.resourceName}") - else - createResponseConfig(Log4jController.loggers, - (name, value) => new DescribeConfigsResponseData.DescribeConfigsResourceResult().setName(name) - .setValue(value.toString).setConfigSource(ConfigSource.DYNAMIC_BROKER_LOGGER_CONFIG.id) - .setIsSensitive(false).setReadOnly(false).setSynonyms(List.empty.asJava)) - case resourceType => throw new InvalidRequestException(s"Unsupported resource type: $resourceType") - } - configResult.setResourceName(resource.resourceName).setResourceType(resource.resourceType) - } catch { - case e: Throwable => - // Log client errors at a lower level than unexpected exceptions - val message = s"Error processing describe configs request for resource $resource" - if (e.isInstanceOf[ApiException]) - info(message, e) - else - error(message, e) - val err = ApiError.fromThrowable(e) - new DescribeConfigsResponseData.DescribeConfigsResult() - .setResourceName(resource.resourceName) - .setResourceType(resource.resourceType) - .setErrorMessage(err.message) - .setErrorCode(err.error.code) - .setConfigs(Collections.emptyList[DescribeConfigsResponseData.DescribeConfigsResourceResult]) - } - }.toList - } - - def alterConfigs(configs: Map[ConfigResource, AlterConfigsRequest.Config], validateOnly: Boolean): Map[ConfigResource, ApiError] = { + def alterConfigs(configs: Map[ConfigResource, AlterConfigsRequest.Config], validateOnly: Boolean): Map[ConfigResource, ApiError] = { configs.map { case (resource, config) => try { @@ -728,98 +648,6 @@ class ZkAdminManager(val config: KafkaConfig, } } - private def brokerSynonyms(name: String): List[String] = { - DynamicBrokerConfig.brokerConfigSynonyms(name, matchListenerOverride = true) - } - - private def brokerDocumentation(name: String): String = { - config.documentationOf(name) - } - - private def configResponseType(configType: Option[ConfigDef.Type]): DescribeConfigsResponse.ConfigType = { - if (configType.isEmpty) - DescribeConfigsResponse.ConfigType.UNKNOWN - else configType.get match { - case ConfigDef.Type.BOOLEAN => DescribeConfigsResponse.ConfigType.BOOLEAN - case ConfigDef.Type.STRING => DescribeConfigsResponse.ConfigType.STRING - case ConfigDef.Type.INT => DescribeConfigsResponse.ConfigType.INT - case ConfigDef.Type.SHORT => DescribeConfigsResponse.ConfigType.SHORT - case ConfigDef.Type.LONG => DescribeConfigsResponse.ConfigType.LONG - case ConfigDef.Type.DOUBLE => DescribeConfigsResponse.ConfigType.DOUBLE - case ConfigDef.Type.LIST => DescribeConfigsResponse.ConfigType.LIST - case ConfigDef.Type.CLASS => DescribeConfigsResponse.ConfigType.CLASS - case ConfigDef.Type.PASSWORD => DescribeConfigsResponse.ConfigType.PASSWORD - case _ => DescribeConfigsResponse.ConfigType.UNKNOWN - } - } - - private def configSynonyms(name: String, synonyms: List[String], isSensitive: Boolean): List[DescribeConfigsResponseData.DescribeConfigsSynonym] = { - val dynamicConfig = config.dynamicConfig - val allSynonyms = mutable.Buffer[DescribeConfigsResponseData.DescribeConfigsSynonym]() - - def maybeAddSynonym(map: Map[String, String], source: ConfigSource)(name: String): Unit = { - map.get(name).map { value => - val configValue = if (isSensitive) null else value - allSynonyms += new DescribeConfigsResponseData.DescribeConfigsSynonym().setName(name).setValue(configValue).setSource(source.id) - } - } - - synonyms.foreach(maybeAddSynonym(dynamicConfig.currentDynamicBrokerConfigs, ConfigSource.DYNAMIC_BROKER_CONFIG)) - synonyms.foreach(maybeAddSynonym(dynamicConfig.currentDynamicDefaultConfigs, ConfigSource.DYNAMIC_DEFAULT_BROKER_CONFIG)) - synonyms.foreach(maybeAddSynonym(dynamicConfig.staticBrokerConfigs, ConfigSource.STATIC_BROKER_CONFIG)) - synonyms.foreach(maybeAddSynonym(dynamicConfig.staticDefaultConfigs, ConfigSource.DEFAULT_CONFIG)) - allSynonyms.dropWhile(s => s.name != name).toList // e.g. drop listener overrides when describing base config - } - - private def createTopicConfigEntry(logConfig: LogConfig, topicProps: Properties, includeSynonyms: Boolean, includeDocumentation: Boolean) - (name: String, value: Any): DescribeConfigsResponseData.DescribeConfigsResourceResult = { - val configEntryType = LogConfig.configType(name) - val isSensitive = KafkaConfig.maybeSensitive(configEntryType) - val valueAsString = if (isSensitive) null else ConfigDef.convertToString(value, configEntryType.orNull) - val allSynonyms = { - val list = LogConfig.TopicConfigSynonyms.get(name) - .map(s => configSynonyms(s, brokerSynonyms(s), isSensitive)) - .getOrElse(List.empty) - if (!topicProps.containsKey(name)) - list - else - new DescribeConfigsResponseData.DescribeConfigsSynonym().setName(name).setValue(valueAsString) - .setSource(ConfigSource.TOPIC_CONFIG.id) +: list - } - val source = if (allSynonyms.isEmpty) ConfigSource.DEFAULT_CONFIG.id else allSynonyms.head.source - val synonyms = if (!includeSynonyms) List.empty else allSynonyms - val dataType = configResponseType(configEntryType) - val configDocumentation = if (includeDocumentation) logConfig.documentationOf(name) else null - new DescribeConfigsResponseData.DescribeConfigsResourceResult() - .setName(name).setValue(valueAsString).setConfigSource(source) - .setIsSensitive(isSensitive).setReadOnly(false).setSynonyms(synonyms.asJava) - .setDocumentation(configDocumentation).setConfigType(dataType.id) - } - - private def createBrokerConfigEntry(perBrokerConfig: Boolean, includeSynonyms: Boolean, includeDocumentation: Boolean) - (name: String, value: Any): DescribeConfigsResponseData.DescribeConfigsResourceResult = { - val allNames = brokerSynonyms(name) - val configEntryType = KafkaConfig.configType(name) - val isSensitive = KafkaConfig.maybeSensitive(configEntryType) - val valueAsString = if (isSensitive) - null - else value match { - case v: String => v - case _ => ConfigDef.convertToString(value, configEntryType.orNull) - } - val allSynonyms = configSynonyms(name, allNames, isSensitive) - .filter(perBrokerConfig || _.source == ConfigSource.DYNAMIC_DEFAULT_BROKER_CONFIG.id) - val synonyms = if (!includeSynonyms) List.empty else allSynonyms - val source = if (allSynonyms.isEmpty) ConfigSource.DEFAULT_CONFIG.id else allSynonyms.head.source - val readOnly = !DynamicBrokerConfig.AllDynamicConfigs.contains(name) - - val dataType = configResponseType(configEntryType) - val configDocumentation = if (includeDocumentation) brokerDocumentation(name) else null - new DescribeConfigsResponseData.DescribeConfigsResourceResult().setName(name).setValue(valueAsString).setConfigSource(source) - .setIsSensitive(isSensitive).setReadOnly(readOnly).setSynonyms(synonyms.asJava) - .setDocumentation(configDocumentation).setConfigType(dataType.id) - } - private def sanitizeEntityName(entityName: String): String = Option(entityName) match { case None => ConfigEntityName.Default diff --git a/core/src/main/scala/kafka/server/metadata/CachedConfigRepository.scala b/core/src/main/scala/kafka/server/metadata/CachedConfigRepository.scala new file mode 100644 index 0000000000..2b52106219 --- /dev/null +++ b/core/src/main/scala/kafka/server/metadata/CachedConfigRepository.scala @@ -0,0 +1,106 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server.metadata + +import java.util +import java.util.Properties +import java.util.concurrent.ConcurrentHashMap +import java.util.function.BiFunction + +import org.apache.kafka.common.config.ConfigResource +import org.apache.kafka.common.config.ConfigResource.Type + +import scala.jdk.CollectionConverters._ + +/** + * A ConfigRepository that stores configurations locally. + */ +class CachedConfigRepository extends ConfigRepository { + private val configMap = new ConcurrentHashMap[ConfigResource, util.HashMap[String, String]] + + /** + * Set the topic config for the given topic name and the given key to the given value. + * + * @param topicName the name of the topic for which the config will be set + * @param key the key identifying the topic config to set + * @param value the value to set for the topic config with null implying a removal + */ + def setTopicConfig(topicName: String, key: String, value: String): Unit = { + setConfig(new ConfigResource(Type.TOPIC, topicName), key, value) + } + + /** + * Set the broker config for the given broker ID and the given key to the given value. + * + * @param brokerId the ID of the broker for which the config will be set + * @param key the key identifying the broker config to set + * @param value the value to set for the broker config with null implying a removal + */ + def setBrokerConfig(brokerId: Int, key: String, value: String): Unit = { + setConfig(new ConfigResource(Type.BROKER, brokerId.toString()), key, value) + } + + /** + * Set the config for the given resource and the given key to the given value. + * + * @param configResource the resource for which the config will be set + * @param key the key identifying the resource config to set + * @param value the value to set for the resource config with null implying a removal + */ + def setConfig(configResource: ConfigResource, key: String, value: String): Unit = { + configMap.compute(configResource, new BiFunction[ConfigResource, util.HashMap[String, String], util.HashMap[String, String]] { + override def apply(resource: ConfigResource, + curConfig: util.HashMap[String, String]): util.HashMap[String, String] = { + if (value == null) { + if (curConfig == null) { + null + } else { + val newConfig = new util.HashMap[String, String](curConfig) + newConfig.remove(key) + if (newConfig.isEmpty) { + null + } else { + newConfig + } + } + } else { + if (curConfig == null) { + val newConfig = new util.HashMap[String, String](1) + newConfig.put(key, value) + newConfig + } else { + val newConfig = new util.HashMap[String, String](curConfig.size() + 1) + newConfig.putAll(curConfig) + newConfig.put(key, value) + newConfig + } + } + } + }) + } + + override def config(configResource: ConfigResource): Properties = { + val properties = new Properties() + Option(configMap.get(configResource)).foreach { + _.entrySet().iterator().asScala.foreach { case e => + properties.put(e.getKey, e.getValue) + } + } + properties + } +} diff --git a/core/src/main/scala/kafka/server/metadata/ConfigRepository.scala b/core/src/main/scala/kafka/server/metadata/ConfigRepository.scala new file mode 100644 index 0000000000..68000d0654 --- /dev/null +++ b/core/src/main/scala/kafka/server/metadata/ConfigRepository.scala @@ -0,0 +1,52 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server.metadata + +import java.util.Properties + +import org.apache.kafka.common.config.ConfigResource +import org.apache.kafka.common.config.ConfigResource.Type + +trait ConfigRepository { + /** + * Return a copy of the topic configuration for the given topic. Future changes will not be reflected. + * + * @param topicName the name of the topic for which the configuration will be returned + * @return a copy of the topic configuration for the given topic + */ + def topicConfig(topicName: String): Properties = { + config(new ConfigResource(Type.TOPIC, topicName)) + } + + /** + * Return a copy of the broker configuration for the given broker. Future changes will not be reflected. + * + * @param brokerId the id of the broker for which configuration will be returned + * @return a copy of the broker configuration for the given broker + */ + def brokerConfig(brokerId: Int): Properties = { + config(new ConfigResource(Type.BROKER, brokerId.toString)) + } + + /** + * Return a copy of the configuration for the given resource. Future changes will not be reflected. + * @param configResource the resource for which the configuration will be returned + * @return a copy of the configuration for the given resource + */ + def config(configResource: ConfigResource): Properties +} diff --git a/core/src/main/scala/kafka/server/metadata/MetadataBrokers.scala b/core/src/main/scala/kafka/server/metadata/MetadataBrokers.scala index ad4986e173..f407f67cbd 100644 --- a/core/src/main/scala/kafka/server/metadata/MetadataBrokers.scala +++ b/core/src/main/scala/kafka/server/metadata/MetadataBrokers.scala @@ -20,8 +20,7 @@ package kafka.server.metadata import java.util import java.util.Collections import java.util.concurrent.ThreadLocalRandom - -import kafka.cluster.BrokerEndPoint +import kafka.cluster.{Broker, BrokerEndPoint} import kafka.common.BrokerEndPointNotAvailableException import org.apache.kafka.common.Node import org.apache.kafka.common.metadata.RegisterBrokerRecord @@ -37,7 +36,15 @@ object MetadataBroker { endPoint.name() -> new Node(record.brokerId, endPoint.host, endPoint.port, record.rack) }.toMap, - true) + fenced = true) + } + + def apply(broker: Broker): MetadataBroker = { + new MetadataBroker(broker.id, broker.rack.orNull, + broker.endPoints.map { endpoint => + endpoint.listenerName.value -> new Node(broker.id, endpoint.host, endpoint.port, broker.rack.orNull) + }.toMap, + fenced = false) } } diff --git a/core/src/main/scala/kafka/server/metadata/MetadataImage.scala b/core/src/main/scala/kafka/server/metadata/MetadataImage.scala index 7723fb6898..1993065279 100755 --- a/core/src/main/scala/kafka/server/metadata/MetadataImage.scala +++ b/core/src/main/scala/kafka/server/metadata/MetadataImage.scala @@ -118,5 +118,9 @@ case class MetadataImage(partitions: MetadataPartitions, def topicIdToName(uuid: Uuid): Option[String] = { partitions.topicIdToName(uuid) } + + def topicNameToId(name: String): Option[Uuid] = { + partitions.topicNameToId(name) + } } diff --git a/core/src/main/scala/kafka/server/metadata/MetadataPartitions.scala b/core/src/main/scala/kafka/server/metadata/MetadataPartitions.scala index bed5c58ce9..c3efac5cbe 100644 --- a/core/src/main/scala/kafka/server/metadata/MetadataPartitions.scala +++ b/core/src/main/scala/kafka/server/metadata/MetadataPartitions.scala @@ -199,10 +199,22 @@ class MetadataPartitionsBuilder(val brokerId: Int, def localRemoved(): collection.Set[MetadataPartition] = _localRemoved.asScala } +object MetadataPartitions { + def apply(nameMap: util.Map[String, util.Map[Int, MetadataPartition]], + idMap: util.Map[Uuid, String]): MetadataPartitions = { + val reverseMap = idMap.asScala.map(_.swap).toMap.asJava + new MetadataPartitions(nameMap, idMap, reverseMap) + } +} + case class MetadataPartitions(private val nameMap: util.Map[String, util.Map[Int, MetadataPartition]], - private val idMap: util.Map[Uuid, String]) { + private val idMap: util.Map[Uuid, String], + private val reverseIdMap: util.Map[String, Uuid]) { + def topicIdToName(uuid: Uuid): Option[String] = Option(idMap.get(uuid)) + def topicNameToId(name: String): Option[Uuid] = Option(reverseIdMap.get(name)) + def copyNameMap(): util.Map[String, util.Map[Int, MetadataPartition]] = { val copy = new util.HashMap[String, util.Map[Int, MetadataPartition]](nameMap.size()) copy.putAll(nameMap) diff --git a/core/src/main/scala/kafka/server/metadata/RaftMetadataCache.scala b/core/src/main/scala/kafka/server/metadata/RaftMetadataCache.scala new file mode 100644 index 0000000000..59a82d9cfb --- /dev/null +++ b/core/src/main/scala/kafka/server/metadata/RaftMetadataCache.scala @@ -0,0 +1,390 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server.metadata + +import kafka.api.LeaderAndIsr +import kafka.controller.StateChangeLogger +import kafka.server.MetadataCache +import kafka.utils.CoreUtils.inLock +import kafka.utils.Logging +import org.apache.kafka.common.internals.Topic +import org.apache.kafka.common.message.MetadataResponseData.{MetadataResponsePartition, MetadataResponseTopic} +import org.apache.kafka.common.{Cluster, Node, PartitionInfo, TopicPartition, Uuid} +import org.apache.kafka.common.message.UpdateMetadataRequestData.{UpdateMetadataBroker, UpdateMetadataPartitionState} +import org.apache.kafka.common.network.ListenerName +import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.requests.{MetadataResponse, UpdateMetadataRequest} + +import java.util +import java.util.Collections +import java.util.concurrent.locks.ReentrantLock +import scala.collection.{Seq, Set, mutable} +import scala.jdk.CollectionConverters._ + +object RaftMetadataCache { + def removePartitionInfo(partitionStates: mutable.AnyRefMap[String, mutable.LongMap[UpdateMetadataPartitionState]], + topic: String, partitionId: Int): Boolean = { + partitionStates.get(topic).exists { infos => + infos.remove(partitionId) + if (infos.isEmpty) partitionStates.remove(topic) + true + } + } + + def addOrUpdatePartitionInfo(partitionStates: mutable.AnyRefMap[String, mutable.LongMap[UpdateMetadataPartitionState]], + topic: String, + partitionId: Int, + stateInfo: UpdateMetadataPartitionState): Unit = { + val infos = partitionStates.getOrElseUpdate(topic, mutable.LongMap.empty) + infos(partitionId) = stateInfo + } +} + + +class RaftMetadataCache(val brokerId: Int) extends MetadataCache with Logging { + this.logIdent = s"[MetadataCache brokerId=$brokerId] " + + private val lock = new ReentrantLock() + + //this is the cache state. every MetadataImage instance is immutable, and updates (performed under a lock) + //replace the value with a completely new one. this means reads (which are not under any lock) need to grab + //the value of this var (into a val) ONCE and retain that read copy for the duration of their operation. + //multiple reads of this value risk getting different snapshots. + @volatile private var _currentImage: MetadataImage = new MetadataImage() + + private val stateChangeLogger = new StateChangeLogger(brokerId, inControllerContext = false, None) + + // This method is the main hotspot when it comes to the performance of metadata requests, + // we should be careful about adding additional logic here. Relatedly, `brokers` is + // `List[Integer]` instead of `List[Int]` to avoid a collection copy. + // filterUnavailableEndpoints exists to support v0 MetadataResponses + private def maybeFilterAliveReplicas(image: MetadataImage, + brokers: java.util.List[Integer], + listenerName: ListenerName, + filterUnavailableEndpoints: Boolean): java.util.List[Integer] = { + if (!filterUnavailableEndpoints) { + brokers + } else { + val res = new util.ArrayList[Integer](math.min(image.brokers.aliveBrokers().size, brokers.size)) + for (brokerId <- brokers.asScala) { + if (hasAliveEndpoint(image, brokerId, listenerName)) + res.add(brokerId) + } + res + } + } + + def currentImage(): MetadataImage = _currentImage + + // errorUnavailableEndpoints exists to support v0 MetadataResponses + // If errorUnavailableListeners=true, return LISTENER_NOT_FOUND if listener is missing on the broker. + // Otherwise, return LEADER_NOT_AVAILABLE for broker unavailable and missing listener (Metadata response v5 and below). + private def getPartitionMetadata(image: MetadataImage, topic: String, listenerName: ListenerName, errorUnavailableEndpoints: Boolean, + errorUnavailableListeners: Boolean): Option[Iterator[MetadataResponsePartition]] = { + val partitionsIterator = image.partitions.topicPartitions(topic) + if (!partitionsIterator.hasNext) { + None + } else { + Some(partitionsIterator.map { partition => + val filteredReplicas = maybeFilterAliveReplicas(image, partition.replicas, + listenerName, errorUnavailableEndpoints) + val filteredIsr = maybeFilterAliveReplicas(image, partition.isr, listenerName, + errorUnavailableEndpoints) + val maybeLeader = getAliveEndpoint(image, partition.leaderId, listenerName) + maybeLeader match { + case None => + val error = if (image.aliveBroker(partition.leaderId).isEmpty) { + debug(s"Error while fetching metadata for ${partition.toTopicPartition}: leader not available") + Errors.LEADER_NOT_AVAILABLE + } else { + debug(s"Error while fetching metadata for ${partition.toTopicPartition}: listener $listenerName " + + s"not found on leader ${partition.leaderId}") + if (errorUnavailableListeners) Errors.LISTENER_NOT_FOUND else Errors.LEADER_NOT_AVAILABLE + } + + new MetadataResponsePartition() + .setErrorCode(error.code) + .setPartitionIndex(partition.partitionIndex) + .setLeaderId(MetadataResponse.NO_LEADER_ID) + .setLeaderEpoch(partition.leaderEpoch) + .setReplicaNodes(filteredReplicas) + .setIsrNodes(filteredIsr) + .setOfflineReplicas(partition.offlineReplicas) + + case Some(leader) => + val error = if (filteredReplicas.size < partition.replicas.size) { + debug(s"Error while fetching metadata for ${partition.toTopicPartition}: replica information not available for " + + s"following brokers ${partition.replicas.asScala.filterNot(filteredReplicas.contains).mkString(",")}") + Errors.REPLICA_NOT_AVAILABLE + } else if (filteredIsr.size < partition.isr.size) { + debug(s"Error while fetching metadata for ${partition.toTopicPartition}: in sync replica information not available for " + + s"following brokers ${partition.isr.asScala.filterNot(filteredIsr.contains).mkString(",")}") + Errors.REPLICA_NOT_AVAILABLE + } else { + Errors.NONE + } + + new MetadataResponsePartition() + .setErrorCode(error.code) + .setPartitionIndex(partition.partitionIndex) + .setLeaderId(leader.id()) + .setLeaderEpoch(partition.leaderEpoch) + .setReplicaNodes(filteredReplicas) + .setIsrNodes(filteredIsr) + .setOfflineReplicas(partition.offlineReplicas) + } + }) + } + } + + /** + * Check whether a broker is alive and has a registered listener matching the provided name. + * This method was added to avoid unnecessary allocations in [[maybeFilterAliveReplicas]], which is + * a hotspot in metadata handling. + */ + private def hasAliveEndpoint(image: MetadataImage, id: Int, listenerName: ListenerName): Boolean = { + image.brokers.aliveBroker(id).exists(_.endpoints.contains(listenerName.value())) + } + + /** + * Get the endpoint matching the provided listener if the broker is alive. Note that listeners can + * be added dynamically, so a broker with a missing listener could be a transient error. + * + * @return None if broker is not alive or if the broker does not have a listener named `listenerName`. + */ + private def getAliveEndpoint(image: MetadataImage, id: Int, listenerName: ListenerName): Option[Node] = { + image.brokers.aliveBroker(id).flatMap(_.endpoints.get(listenerName.value())) + } + + // errorUnavailableEndpoints exists to support v0 MetadataResponses + override def getTopicMetadata(topics: Set[String], + listenerName: ListenerName, + errorUnavailableEndpoints: Boolean = false, + errorUnavailableListeners: Boolean = false): Seq[MetadataResponseTopic] = { + val image = _currentImage + topics.toSeq.flatMap { topic => + getPartitionMetadata(image, topic, listenerName, errorUnavailableEndpoints, errorUnavailableListeners).map { partitionMetadata => + new MetadataResponseTopic() + .setErrorCode(Errors.NONE.code) + .setName(topic) + .setTopicId(image.topicNameToId(topic).getOrElse(Uuid.ZERO_UUID)) + .setIsInternal(Topic.isInternal(topic)) + .setPartitions(partitionMetadata.toBuffer.asJava) + } + } + } + + override def getAllTopics(): Set[String] = _currentImage.partitions.allTopicNames() + + override def getAllPartitions(): Set[TopicPartition] = { + _currentImage.partitions.allPartitions().map { + partition => partition.toTopicPartition + }.toSet + } + + override def getNonExistingTopics(topics: Set[String]): Set[String] = { + topics.diff(_currentImage.partitions.allTopicNames()) + } + + override def getAliveBroker(brokerId: Int): Option[MetadataBroker] = { + _currentImage.brokers.aliveBroker(brokerId) + } + + override def getAliveBrokers: Seq[MetadataBroker] = { + _currentImage.brokers.aliveBrokers() + } + + override def getPartitionInfo(topic: String, partitionId: Int): Option[UpdateMetadataPartitionState] = { + _currentImage.partitions.topicPartition(topic, partitionId).map { partition => + new UpdateMetadataPartitionState(). + setTopicName(partition.topicName). + setPartitionIndex(partition.partitionIndex). + setControllerEpoch(-1). // Controller epoch is not stored in the cache. + setLeader(partition.leaderId). + setLeaderEpoch(partition.leaderEpoch). + setIsr(partition.isr). + setZkVersion(-1) // ZK version is not stored in the cache. + } + } + + override def numPartitions(topic: String): Option[Int] = { + _currentImage.partitions.numTopicPartitions(topic) + } + + // if the leader is not known, return None; + // if the leader is known and corresponding node is available, return Some(node) + // if the leader is known but corresponding node with the listener name is not available, return Some(NO_NODE) + override def getPartitionLeaderEndpoint(topic: String, partitionId: Int, listenerName: ListenerName): Option[Node] = { + val image = _currentImage + image.partitions.topicPartition(topic, partitionId).map { partition => + image.aliveBroker(partition.leaderId) match { + case Some(broker) => + broker.endpoints.getOrElse(listenerName.value(), Node.noNode) + case None => + Node.noNode + } + } + } + + override def getPartitionReplicaEndpoints(tp: TopicPartition, listenerName: ListenerName): Map[Int, Node] = { + val image = _currentImage + image.partitions.topicPartition(tp.topic(), tp.partition()).map { partition => + partition.replicas.asScala.map(replicaId => replicaId.intValue() -> { + image.aliveBroker(replicaId) match { + case Some(broker) => + broker.endpoints.getOrElse(listenerName.value(), Node.noNode()) + case None => + Node.noNode() + }}).toMap + .filter(pair => pair match { + case (_, node) => !node.isEmpty + }) + }.getOrElse(Map.empty[Int, Node]) + } + + override def getControllerId: Option[Int] = { + _currentImage.controllerId + } + + override def getClusterMetadata(clusterId: String, listenerName: ListenerName): Cluster = { + val image = _currentImage + val nodes = new util.HashMap[Integer, Node] + image.brokers.aliveBrokers().foreach { node => + if (!node.fenced) { + node.endpoints.get(listenerName.value()).foreach { nodes.put(node.id, _) } + } + } + + def node(id: Integer): Node = { + Option(nodes.get(id)).getOrElse(new Node(id, "", -1)) + } + + val partitionInfos = new util.ArrayList[PartitionInfo] + val internalTopics = new util.HashSet[String] + + image.partitions.allPartitions().foreach { partition => + partitionInfos.add(new PartitionInfo(partition.topicName, + partition.partitionIndex, node(partition.leaderId), + partition.replicas.asScala.map(node).toArray, + partition.isr.asScala.map(node).toArray, + partition.offlineReplicas.asScala.map(node).toArray)) + if (Topic.isInternal(partition.topicName)) { + internalTopics.add(partition.topicName) + } + } + + new Cluster(clusterId, nodes.values(), + partitionInfos, Collections.emptySet[String], internalTopics, + node(Integer.valueOf(image.controllerId.getOrElse(-1)))) + } + + def stateChangeTraceEnabled(): Boolean = { + stateChangeLogger.isTraceEnabled + } + + def logStateChangeTrace(str: String): Unit = { + stateChangeLogger.trace(str) + } + + // This method returns the deleted TopicPartitions received from UpdateMetadataRequest + override def updateMetadata(correlationId: Int, request: UpdateMetadataRequest): Seq[TopicPartition] = { + inLock(lock) { + val image = _currentImage + val builder = MetadataImageBuilder(brokerId, logger.underlying, image) + + builder.controllerId(if (request.controllerId() < 0) None else Some(request.controllerId())) + + // Compare the new brokers with the existing ones. + def toMetadataBroker(broker: UpdateMetadataBroker): MetadataBroker = { + val endpoints = broker.endpoints().asScala.map { endpoint => + endpoint.listener -> new Node(broker.id(), endpoint.host(), endpoint.port()) + }.toMap + MetadataBroker(broker.id(), broker.rack(), endpoints, fenced = false) + } + val found = new util.IdentityHashMap[MetadataBroker, Boolean](image.numAliveBrokers()) + request.liveBrokers().iterator().asScala.foreach { brokerInfo => + val newBroker = toMetadataBroker(brokerInfo) + image.brokers.get(brokerInfo.id) match { + case None => builder.brokersBuilder().add(newBroker) + case Some(existingBroker) => + found.put(existingBroker, true) + if (!existingBroker.equals(newBroker)) { + builder.brokersBuilder().add(newBroker) + } + } + } + image.brokers.iterator().foreach { broker => + if (!found.containsKey(broker)) { + builder.brokersBuilder().remove(broker.id) + } + } + + val topicIds = request.topicStates().iterator().asScala.map { topic => + topic.topicName() -> topic.topicId() + }.toMap + + val traceEnabled = stateChangeLogger.isTraceEnabled + var numDeleted = 0 + var numAdded = 0 + val deleted = mutable.Buffer[TopicPartition]() + request.partitionStates().iterator().asScala.foreach { partition => + if (partition.leader() == LeaderAndIsr.LeaderDuringDelete) { + if (traceEnabled) { + stateChangeLogger.trace(s"Deleted partition ${partition.topicName()}-${partition.partitionIndex()} " + + "from metadata cache in response to UpdateMetadata request sent by " + + s"controller ${request.controllerId} epoch ${request.controllerEpoch} " + + s"with correlation id $correlationId") + } + builder.partitionsBuilder().remove(partition.topicName(), partition.partitionIndex()) + deleted += new TopicPartition(partition.topicName(), partition.partitionIndex()) + numDeleted = numDeleted + 1 + } else { + val prevPartition = builder.partition(partition.topicName(), partition.partitionIndex()) + val newPartition = MetadataPartition(prevPartition, partition) + if (traceEnabled) { + stateChangeLogger.trace(s"Cached leader info $newPartition in response to " + + s"UpdateMetadata request sent by controller $request.controllerId epoch " + + s"$request.controllerEpoch with correlation id $correlationId") + } + builder.partitionsBuilder().set(newPartition) + topicIds.get(newPartition.topicName).foreach { + topicId => builder.partitionsBuilder().addUuidMapping(newPartition.topicName, topicId) + } + numAdded = numAdded + 1 + } + } + stateChangeLogger.info(s"Add ${numAdded} partitions and deleted ${numDeleted} " + + "partitions to the metadata cache in response to UpdateMetadata request sent by " + + s"controller ${request.controllerId} epoch ${request.controllerEpoch} with " + + s"correlation id ${correlationId}") + + _currentImage = builder.build() + deleted + } + } + + override def contains(topic: String): Boolean = _currentImage.partitions.contains(topic) + + override def contains(tp: TopicPartition): Boolean = { + _currentImage.partitions.topicPartition(tp.topic(), tp.partition()).isDefined + } + + def image(newImage: MetadataImage): Unit = inLock(lock) { + _currentImage = newImage + } +} diff --git a/core/src/main/scala/kafka/server/metadata/ZkConfigRepository.scala b/core/src/main/scala/kafka/server/metadata/ZkConfigRepository.scala new file mode 100644 index 0000000000..95fe75258e --- /dev/null +++ b/core/src/main/scala/kafka/server/metadata/ZkConfigRepository.scala @@ -0,0 +1,42 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server.metadata + +import java.util.Properties + +import kafka.server.ConfigType +import kafka.zk.{AdminZkClient, KafkaZkClient} +import org.apache.kafka.common.config.ConfigResource +import org.apache.kafka.common.config.ConfigResource.Type + + +object ZkConfigRepository { + def apply(zkClient: KafkaZkClient): ZkConfigRepository = + new ZkConfigRepository(new AdminZkClient(zkClient)) +} + +class ZkConfigRepository(adminZkClient: AdminZkClient) extends ConfigRepository { + override def config(configResource: ConfigResource): Properties = { + val configTypeForZk = configResource.`type` match { + case Type.TOPIC => ConfigType.Topic + case Type.BROKER => ConfigType.Broker + case tpe => throw new IllegalArgumentException(s"Unsupported config type: $tpe") + } + adminZkClient.fetchEntityConfig(configTypeForZk, configResource.name) + } +} diff --git a/core/src/main/scala/kafka/tools/ClusterTool.scala b/core/src/main/scala/kafka/tools/ClusterTool.scala new file mode 100644 index 0000000000..f0d3d90a8c --- /dev/null +++ b/core/src/main/scala/kafka/tools/ClusterTool.scala @@ -0,0 +1,125 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.tools + +import java.io.PrintStream +import java.util.Properties +import java.util.concurrent.ExecutionException + +import kafka.utils.{Exit, Logging} +import net.sourceforge.argparse4j.ArgumentParsers +import net.sourceforge.argparse4j.impl.Arguments.store +import org.apache.kafka.clients.admin.Admin +import org.apache.kafka.common.errors.UnsupportedVersionException +import org.apache.kafka.common.utils.Utils + +object ClusterTool extends Logging { + def main(args: Array[String]): Unit = { + try { + val parser = ArgumentParsers. + newArgumentParser("kafka-cluster"). + defaultHelp(true). + description("The Kafka cluster tool.") + val subparsers = parser.addSubparsers().dest("command") + + val clusterIdParser = subparsers.addParser("cluster-id"). + help("Get information about the ID of a cluster.") + val decommissionParser = subparsers.addParser("decommission"). + help("Decommission a broker..") + List(clusterIdParser, decommissionParser).foreach(parser => { + parser.addArgument("--bootstrap-server", "-b"). + action(store()). + help("A list of host/port pairs to use for establishing the connection to the kafka cluster.") + parser.addArgument("--config", "-c"). + action(store()). + help("A property file containing configs to passed to AdminClient.") + }) + decommissionParser.addArgument("--id", "-i"). + `type`(classOf[Integer]). + action(store()). + help("The ID of the broker to decommission.") + + val namespace = parser.parseArgsOrFail(args) + val command = namespace.getString("command") + val configPath = namespace.getString("config") + val properties = if (configPath == null) { + new Properties() + } else { + Utils.loadProps(configPath) + } + Option(namespace.getString("bootstrap_server")). + foreach(b => properties.setProperty("bootstrap.servers", b)) + if (properties.getProperty("bootstrap.servers") == null) { + throw new TerseFailure("Please specify --bootstrap-server.") + } + + command match { + case "cluster-id" => + val adminClient = Admin.create(properties) + try { + clusterIdCommand(System.out, adminClient) + } finally { + adminClient.close() + } + Exit.exit(0) + case "decommission" => + val adminClient = Admin.create(properties) + try { + decommissionCommand(System.out, adminClient, namespace.getInt("id")) + } finally { + adminClient.close() + } + Exit.exit(0) + case _ => + throw new RuntimeException(s"Unknown command $command") + } + } catch { + case e: TerseFailure => + System.err.println(e.getMessage) + System.exit(1) + } + } + + def clusterIdCommand(stream: PrintStream, + adminClient: Admin): Unit = { + val clusterId = Option(adminClient.describeCluster().clusterId().get()) + clusterId match { + case None => stream.println(s"No cluster ID found. The Kafka version is probably too old.") + case Some(id) => stream.println(s"Cluster ID: ${id}") + } + } + + def decommissionCommand(stream: PrintStream, + adminClient: Admin, + id: Int): Unit = { + try { + Option(adminClient.decommissionBroker(id).all().get()) + stream.println(s"Broker ${id} is no longer registered. Note that if the broker " + + "is still running, or is restarted, it will re-register.") + } catch { + case e: ExecutionException => { + val cause = e.getCause() + if (cause.isInstanceOf[UnsupportedVersionException]) { + stream.println(s"The target cluster does not support broker decommissioning.") + } else { + throw e + } + } + } + } +} diff --git a/core/src/main/scala/kafka/tools/StorageTool.scala b/core/src/main/scala/kafka/tools/StorageTool.scala new file mode 100644 index 0000000000..ff84007a4e --- /dev/null +++ b/core/src/main/scala/kafka/tools/StorageTool.scala @@ -0,0 +1,238 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.tools + +import java.io.PrintStream +import java.nio.file.{Files, Paths} + +import kafka.server.{BrokerMetadataCheckpoint, KafkaConfig, MetaProperties, RawMetaProperties} +import kafka.utils.{Exit, Logging} +import net.sourceforge.argparse4j.ArgumentParsers +import net.sourceforge.argparse4j.impl.Arguments.{store, storeTrue} +import org.apache.kafka.common.Uuid +import org.apache.kafka.common.utils.Utils + +import scala.collection.mutable + +object StorageTool extends Logging { + def main(args: Array[String]): Unit = { + try { + val parser = ArgumentParsers. + newArgumentParser("kafka-storage"). + defaultHelp(true). + description("The Kafka storage tool.") + val subparsers = parser.addSubparsers().dest("command") + + val infoParser = subparsers.addParser("info"). + help("Get information about the Kafka log directories on this node.") + val formatParser = subparsers.addParser("format"). + help("Format the Kafka log directories on this node.") + subparsers.addParser("random-uuid").help("Print a random UUID.") + List(infoParser, formatParser).foreach(parser => { + parser.addArgument("--config", "-c"). + action(store()). + required(true). + help("The Kafka configuration file to use.") + }) + formatParser.addArgument("--cluster-id", "-t"). + action(store()). + required(true). + help("The cluster ID to use.") + formatParser.addArgument("--ignore-formatted", "-g"). + action(storeTrue()) + + val namespace = parser.parseArgsOrFail(args) + val command = namespace.getString("command") + val config = Option(namespace.getString("config")).flatMap( + p => Some(new KafkaConfig(Utils.loadProps(p)))) + + command match { + case "info" => + val directories = configToLogDirectories(config.get) + val kip500Mode = configToKip500Mode(config.get) + Exit.exit(infoCommand(System.out, kip500Mode, directories)) + + case "format" => + val directories = configToLogDirectories(config.get) + val clusterId = namespace.getString("cluster_id") + val metaProperties = buildMetadataProperties(clusterId, config.get) + val ignoreFormatted = namespace.getBoolean("ignore_formatted") + if (!configToKip500Mode(config.get)) { + throw new TerseFailure("The kafka configuration file appears to be for " + + "a legacy cluster. Formatting is only supported for kip-500 clusters.") + } + Exit.exit(formatCommand(System.out, directories, metaProperties, ignoreFormatted )) + + case "random-uuid" => + System.out.println(Uuid.randomUuid) + Exit.exit(0) + + case _ => + throw new RuntimeException(s"Unknown command $command") + } + } catch { + case e: TerseFailure => + System.err.println(e.getMessage) + System.exit(1) + } + } + + def configToLogDirectories(config: KafkaConfig): Seq[String] = { + val directories = new mutable.TreeSet[String] + directories ++= config.logDirs + Option(config.metadataLogDir).foreach(directories.add) + directories.toSeq + } + + def configToKip500Mode(config: KafkaConfig): Boolean = config.processRoles.nonEmpty + + def infoCommand(stream: PrintStream, kip500Mode: Boolean, directories: Seq[String]): Int = { + val problems = new mutable.ArrayBuffer[String] + val foundDirectories = new mutable.ArrayBuffer[String] + var prevMetadata: Option[RawMetaProperties] = None + directories.sorted.foreach(directory => { + val directoryPath = Paths.get(directory) + if (!Files.isDirectory(directoryPath)) { + if (!Files.exists(directoryPath)) { + problems += s"$directoryPath does not exist" + } else { + problems += s"$directoryPath is not a directory" + } + } else { + foundDirectories += directoryPath.toString + val metaPath = directoryPath.resolve("meta.properties") + if (!Files.exists(metaPath)) { + problems += s"$directoryPath is not formatted." + } else { + val properties = Utils.loadProps(metaPath.toString) + val rawMetaProperties = new RawMetaProperties(properties) + + val curMetadata = rawMetaProperties.version match { + case 0 | 1 => Some(rawMetaProperties) + case v => + problems += s"Unsupported version for $metaPath: $v" + None + } + + if (prevMetadata.isEmpty) { + prevMetadata = curMetadata + } else { + if (!prevMetadata.get.equals(curMetadata.get)) { + problems += s"Metadata for $metaPath was ${curMetadata.get}, " + + s"but other directories featured ${prevMetadata.get}" + } + } + } + } + }) + + prevMetadata.foreach { prev => + if (kip500Mode) { + if (prev.version == 0) { + problems += "The kafka configuration file appears to be for a kip-500 cluster, but " + + "the directories are formatted for legacy mode." + } + } else if (prev.version == 1) { + problems += "The kafka configuration file appears to be for a legacy cluster, but " + + "the directories are formatted for kip-500." + } + } + + if (directories.isEmpty) { + stream.println("No directories specified.") + 0 + } else { + if (foundDirectories.nonEmpty) { + if (foundDirectories.size == 1) { + stream.println("Found log directory:") + } else { + stream.println("Found log directories:") + } + foundDirectories.foreach(d => stream.println(" %s".format(d))) + stream.println("") + } + + prevMetadata.foreach { prev => + stream.println(s"Found metadata: ${prev}") + stream.println("") + } + + if (problems.nonEmpty) { + if (problems.size == 1) { + stream.println("Found problem:") + } else { + stream.println("Found problems:") + } + problems.foreach(d => stream.println(" %s".format(d))) + stream.println("") + 1 + } else { + 0 + } + } + } + + def buildMetadataProperties( + clusterIdStr: String, + config: KafkaConfig + ): MetaProperties = { + val effectiveClusterId = try { + Uuid.fromString(clusterIdStr) + } catch { + case e: Throwable => throw new TerseFailure(s"Cluster ID string $clusterIdStr " + + s"does not appear to be a valid UUID: ${e.getMessage}") + } + require(config.nodeId >= 0, s"The node.id must be set to a non-negative integer.") + new MetaProperties(effectiveClusterId, config.nodeId) + } + + def formatCommand(stream: PrintStream, + directories: Seq[String], + metaProperties: MetaProperties, + ignoreFormatted: Boolean): Int = { + if (directories.isEmpty) { + throw new TerseFailure("No log directories found in the configuration.") + } + val unformattedDirectories = directories.filter(directory => { + if (!Files.isDirectory(Paths.get(directory)) || !Files.exists(Paths.get(directory, "meta.properties"))) { + true + } else if (!ignoreFormatted) { + throw new TerseFailure(s"Log directory ${directory} is already formatted. " + + "Use --ignore-formatted to ignore this directory and format the others.") + } else { + false + } + }) + if (unformattedDirectories.isEmpty) { + stream.println("All of the log directories are already formatted.") + } + unformattedDirectories.foreach(directory => { + try { + Files.createDirectories(Paths.get(directory)) + } catch { + case e: Throwable => throw new TerseFailure(s"Unable to create storage " + + s"directory ${directory}: ${e.getMessage}") + } + val metaPropertiesPath = Paths.get(directory, "meta.properties") + val checkpoint = new BrokerMetadataCheckpoint(metaPropertiesPath.toFile) + checkpoint.write(metaProperties.toProperties) + stream.println(s"Formatting ${directory}") + }) + 0 + } +} diff --git a/core/src/main/scala/kafka/tools/TerseFailure.scala b/core/src/main/scala/kafka/tools/TerseFailure.scala new file mode 100644 index 0000000000..c37b613d71 --- /dev/null +++ b/core/src/main/scala/kafka/tools/TerseFailure.scala @@ -0,0 +1,30 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.tools + +import org.apache.kafka.common.KafkaException + +/** + * An exception thrown to indicate that the command has failed, but we don't want to + * print a stack trace. + * + * @param message The message to print out before exiting. A stack trace will not + * be printed. + */ +class TerseFailure(message: String) extends KafkaException(message) { +} diff --git a/core/src/test/java/kafka/test/ClusterConfig.java b/core/src/test/java/kafka/test/ClusterConfig.java new file mode 100644 index 0000000000..db5f14e1a4 --- /dev/null +++ b/core/src/test/java/kafka/test/ClusterConfig.java @@ -0,0 +1,207 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.test; + +import kafka.test.annotation.Type; +import org.apache.kafka.common.security.auth.SecurityProtocol; + +import java.io.File; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Optional; +import java.util.Properties; + +/** + * Represents a requested configuration of a Kafka cluster for integration testing + */ +public class ClusterConfig { + + private final Type type; + private final int brokers; + private final int controllers; + private final String name; + private final boolean autoStart; + + private final SecurityProtocol securityProtocol; + private final String listenerName; + private final File trustStoreFile; + + private final Properties serverProperties = new Properties(); + private final Properties producerProperties = new Properties(); + private final Properties consumerProperties = new Properties(); + private final Properties adminClientProperties = new Properties(); + private final Properties saslServerProperties = new Properties(); + private final Properties saslClientProperties = new Properties(); + + ClusterConfig(Type type, int brokers, int controllers, String name, boolean autoStart, + SecurityProtocol securityProtocol, String listenerName, File trustStoreFile) { + this.type = type; + this.brokers = brokers; + this.controllers = controllers; + this.name = name; + this.autoStart = autoStart; + this.securityProtocol = securityProtocol; + this.listenerName = listenerName; + this.trustStoreFile = trustStoreFile; + } + + public Type clusterType() { + return type; + } + + public int numBrokers() { + return brokers; + } + + public int numControllers() { + return controllers; + } + + public Optional name() { + return Optional.ofNullable(name); + } + + public boolean isAutoStart() { + return autoStart; + } + + public Properties serverProperties() { + return serverProperties; + } + + public Properties producerProperties() { + return producerProperties; + } + + public Properties consumerProperties() { + return consumerProperties; + } + + public Properties adminClientProperties() { + return adminClientProperties; + } + + public Properties saslServerProperties() { + return saslServerProperties; + } + + public Properties saslClientProperties() { + return saslClientProperties; + } + + public SecurityProtocol securityProtocol() { + return securityProtocol; + } + + public Optional listenerName() { + return Optional.ofNullable(listenerName); + } + + public Optional trustStoreFile() { + return Optional.ofNullable(trustStoreFile); + } + + public Map nameTags() { + Map tags = new LinkedHashMap<>(3); + name().ifPresent(name -> tags.put("Name", name)); + tags.put("security", securityProtocol.name()); + listenerName().ifPresent(listener -> tags.put("listener", listener)); + return tags; + } + + public ClusterConfig copyOf() { + ClusterConfig copy = new ClusterConfig(type, brokers, controllers, name, autoStart, securityProtocol, listenerName, trustStoreFile); + copy.serverProperties.putAll(serverProperties); + copy.producerProperties.putAll(producerProperties); + copy.consumerProperties.putAll(consumerProperties); + copy.saslServerProperties.putAll(saslServerProperties); + copy.saslClientProperties.putAll(saslClientProperties); + return copy; + } + + public static Builder defaultClusterBuilder() { + return new Builder(Type.ZK, 1, 1, true, SecurityProtocol.PLAINTEXT); + } + + public static Builder clusterBuilder(Type type, int brokers, int controllers, boolean autoStart, SecurityProtocol securityProtocol) { + return new Builder(type, brokers, controllers, autoStart, securityProtocol); + } + + public static class Builder { + private Type type; + private int brokers; + private int controllers; + private String name; + private boolean autoStart; + private SecurityProtocol securityProtocol; + private String listenerName; + private File trustStoreFile; + + Builder(Type type, int brokers, int controllers, boolean autoStart, SecurityProtocol securityProtocol) { + this.type = type; + this.brokers = brokers; + this.controllers = controllers; + this.autoStart = autoStart; + this.securityProtocol = securityProtocol; + } + + public Builder type(Type type) { + this.type = type; + return this; + } + + public Builder brokers(int brokers) { + this.brokers = brokers; + return this; + } + + public Builder controllers(int controllers) { + this.controllers = controllers; + return this; + } + + public Builder name(String name) { + this.name = name; + return this; + } + + public Builder autoStart(boolean autoStart) { + this.autoStart = autoStart; + return this; + } + + public Builder securityProtocol(SecurityProtocol securityProtocol) { + this.securityProtocol = securityProtocol; + return this; + } + + public Builder listenerName(String listenerName) { + this.listenerName = listenerName; + return this; + } + + public Builder trustStoreFile(File trustStoreFile) { + this.trustStoreFile = trustStoreFile; + return this; + } + + public ClusterConfig build() { + return new ClusterConfig(type, brokers, controllers, name, autoStart, securityProtocol, listenerName, trustStoreFile); + } + } +} diff --git a/core/src/test/java/kafka/test/ClusterGenerator.java b/core/src/test/java/kafka/test/ClusterGenerator.java new file mode 100644 index 0000000000..97a24633b0 --- /dev/null +++ b/core/src/test/java/kafka/test/ClusterGenerator.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.test; + +import java.util.function.Consumer; + +@FunctionalInterface +public interface ClusterGenerator extends Consumer { + +} diff --git a/core/src/test/java/kafka/test/ClusterInstance.java b/core/src/test/java/kafka/test/ClusterInstance.java new file mode 100644 index 0000000000..8732aa90ec --- /dev/null +++ b/core/src/test/java/kafka/test/ClusterInstance.java @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.test; + +import kafka.network.SocketServer; +import kafka.test.annotation.ClusterTest; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.common.network.ListenerName; + +import java.util.Collection; +import java.util.Properties; + +public interface ClusterInstance { + + enum ClusterType { + ZK, + // RAFT + } + + /** + * Cluster type. For now, only ZK is supported. + */ + ClusterType clusterType(); + + /** + * The cluster configuration used to create this cluster. Changing data in this instance through this accessor will + * have no affect on the cluster since it is already provisioned. + */ + ClusterConfig config(); + + /** + * 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 + */ + ListenerName clientListener(); + + /** + * The broker connect string which can be used by clients for bootstrapping + */ + String bootstrapServers(); + + /** + * A collection of all brokers in the cluster. In ZK-based clusters this will also include the broker which is + * acting as the controller (since ZK controllers serve both broker and controller roles). + */ + Collection brokerSocketServers(); + + /** + * A collection of all controllers in the cluster. For ZK-based clusters, this will return the broker which is also + * currently the active controller. For Raft-based clusters, this will return all controller servers. + */ + Collection controllerSocketServers(); + + /** + * Return any one of the broker servers. Throw an error if none are found + */ + SocketServer anyBrokerSocketServer(); + + /** + * Return any one of the controller servers. Throw an error if none are found + */ + SocketServer anyControllerSocketServer(); + + /** + * The underlying object which is responsible for setting up and tearing down the cluster. + */ + Object getUnderlying(); + + default T getUnderlying(Class asClass) { + return asClass.cast(getUnderlying()); + } + + Admin createAdminClient(Properties configOverrides); + + default Admin createAdminClient() { + return createAdminClient(new Properties()); + } + + void start(); + + void stop(); +} diff --git a/core/src/test/java/kafka/test/ClusterTestExtensionsTest.java b/core/src/test/java/kafka/test/ClusterTestExtensionsTest.java new file mode 100644 index 0000000000..6818e4332f --- /dev/null +++ b/core/src/test/java/kafka/test/ClusterTestExtensionsTest.java @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.test; + +import kafka.test.annotation.AutoStart; +import kafka.test.annotation.ClusterConfigProperty; +import kafka.test.annotation.ClusterTemplate; +import kafka.test.annotation.ClusterTest; +import kafka.test.annotation.ClusterTestDefaults; +import kafka.test.annotation.ClusterTests; +import kafka.test.annotation.Type; +import kafka.test.junit.ClusterTestExtensions; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.extension.ExtendWith; + + +@ClusterTestDefaults(clusterType = Type.ZK) // Set defaults for a few params in @ClusterTest(s) +@ExtendWith(ClusterTestExtensions.class) +public class ClusterTestExtensionsTest { + + private final ClusterInstance clusterInstance; + private final ClusterConfig config; + + ClusterTestExtensionsTest(ClusterInstance clusterInstance, ClusterConfig config) { // Constructor injections + this.clusterInstance = clusterInstance; + this.config = config; + } + + // Static methods can generate cluster configurations + static void generate1(ClusterGenerator clusterGenerator) { + clusterGenerator.accept(ClusterConfig.defaultClusterBuilder().name("Generated Test").build()); + } + + // BeforeEach run after class construction, but before cluster initialization and test invocation + @BeforeEach + public void beforeEach(ClusterConfig config) { + Assertions.assertSame(this.config, config, "Injected objects should be the same"); + config.serverProperties().put("before", "each"); + } + + // AfterEach runs after test invocation and cluster teardown + @AfterEach + public void afterEach(ClusterConfig config) { + Assertions.assertSame(this.config, config, "Injected objects should be the same"); + } + + // With no params, configuration comes from the annotation defaults as well as @ClusterTestDefaults (if present) + @ClusterTest + public void testClusterTest(ClusterConfig config, ClusterInstance clusterInstance) { + Assertions.assertSame(this.config, config, "Injected objects should be the same"); + Assertions.assertSame(this.clusterInstance, clusterInstance, "Injected objects should be the same"); + Assertions.assertEquals(clusterInstance.clusterType(), ClusterInstance.ClusterType.ZK); // From the class level default + Assertions.assertEquals(clusterInstance.config().serverProperties().getProperty("before"), "each"); + } + + // generate1 is a template method which generates any number of cluster configs + @ClusterTemplate("generate1") + public void testClusterTemplate() { + Assertions.assertEquals(clusterInstance.clusterType(), ClusterInstance.ClusterType.ZK, + "generate1 provided a Zk cluster, so we should see that here"); + Assertions.assertEquals(clusterInstance.config().name().orElse(""), "Generated Test", + "generate 1 named this cluster config, so we should see that here"); + Assertions.assertEquals(clusterInstance.config().serverProperties().getProperty("before"), "each"); + } + + // Multiple @ClusterTest can be used with @ClusterTests + @ClusterTests({ + @ClusterTest(name = "cluster-tests-1", clusterType = Type.ZK, serverProperties = { + @ClusterConfigProperty(key = "foo", value = "bar"), + @ClusterConfigProperty(key = "spam", value = "eggs") + }), + @ClusterTest(name = "cluster-tests-2", clusterType = Type.ZK, serverProperties = { + @ClusterConfigProperty(key = "foo", value = "baz"), + @ClusterConfigProperty(key = "spam", value = "eggz") + }) + }) + public void testClusterTests() { + if (clusterInstance.config().name().filter(name -> name.equals("cluster-tests-1")).isPresent()) { + Assertions.assertEquals(clusterInstance.config().serverProperties().getProperty("foo"), "bar"); + Assertions.assertEquals(clusterInstance.config().serverProperties().getProperty("spam"), "eggs"); + } else if (clusterInstance.config().name().filter(name -> name.equals("cluster-tests-2")).isPresent()) { + Assertions.assertEquals(clusterInstance.config().serverProperties().getProperty("foo"), "baz"); + Assertions.assertEquals(clusterInstance.config().serverProperties().getProperty("spam"), "eggz"); + } else { + Assertions.fail("Unknown cluster config " + clusterInstance.config().name()); + } + } + + @ClusterTest(autoStart = AutoStart.NO) + public void testNoAutoStart() { + Assertions.assertThrows(RuntimeException.class, clusterInstance::anyBrokerSocketServer); + clusterInstance.start(); + Assertions.assertNotNull(clusterInstance.anyBrokerSocketServer()); + } +} diff --git a/core/src/test/java/kafka/test/annotation/AutoStart.java b/core/src/test/java/kafka/test/annotation/AutoStart.java new file mode 100644 index 0000000000..24fdedfb22 --- /dev/null +++ b/core/src/test/java/kafka/test/annotation/AutoStart.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.test.annotation; + +public enum AutoStart { + YES, + NO, + DEFAULT +} diff --git a/core/src/test/java/kafka/test/annotation/ClusterConfigProperty.java b/core/src/test/java/kafka/test/annotation/ClusterConfigProperty.java new file mode 100644 index 0000000000..eb1434d3b0 --- /dev/null +++ b/core/src/test/java/kafka/test/annotation/ClusterConfigProperty.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 kafka.test.annotation; + +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +@Documented +@Target({ElementType.ANNOTATION_TYPE}) +@Retention(RetentionPolicy.RUNTIME) +public @interface ClusterConfigProperty { + String key(); + String value(); +} diff --git a/core/src/test/java/kafka/test/annotation/ClusterTemplate.java b/core/src/test/java/kafka/test/annotation/ClusterTemplate.java new file mode 100644 index 0000000000..f776b4e832 --- /dev/null +++ b/core/src/test/java/kafka/test/annotation/ClusterTemplate.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.test.annotation; + +import kafka.test.ClusterConfig; +import kafka.test.ClusterGenerator; +import org.junit.jupiter.api.TestTemplate; + +import java.lang.annotation.Documented; +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.ElementType.METHOD; +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +/** + * Used to indicate that a test should call the method given by {@link #value()} to generate a number of + * cluster configurations. The method specified by the value should accept a single argument of the type + * {@link ClusterGenerator}. Any return value from the method is ignore. A test invocation + * will be generated for each {@link ClusterConfig} provided to the ClusterGenerator instance. + * + * The method given here must be static since it is invoked before any tests are actually run. Each test generated + * by this annotation will run as if it was defined as a separate test method with its own + * {@link org.junit.jupiter.api.Test}. That is to say, each generated test invocation will have a separate lifecycle. + * + * This annotation may be used in conjunction with {@link ClusterTest} and {@link ClusterTests} which also yield + * ClusterConfig instances. + * + * For Scala tests, the method should be defined in a companion object with the same name as the test class. + */ +@Documented +@Target({METHOD}) +@Retention(RUNTIME) +@TestTemplate +public @interface ClusterTemplate { + /** + * Specify the static method used for generating cluster configs + */ + String value(); +} diff --git a/core/src/test/java/kafka/test/annotation/ClusterTest.java b/core/src/test/java/kafka/test/annotation/ClusterTest.java new file mode 100644 index 0000000000..687255c3c4 --- /dev/null +++ b/core/src/test/java/kafka/test/annotation/ClusterTest.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.test.annotation; + +import org.apache.kafka.common.security.auth.SecurityProtocol; +import org.junit.jupiter.api.TestTemplate; + +import java.lang.annotation.Documented; +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.ElementType.METHOD; +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +@Documented +@Target({METHOD}) +@Retention(RUNTIME) +@TestTemplate +public @interface ClusterTest { + Type clusterType() default Type.DEFAULT; + int brokers() default 0; + int controllers() default 0; + AutoStart autoStart() default AutoStart.DEFAULT; + + String name() default ""; + SecurityProtocol securityProtocol() default SecurityProtocol.PLAINTEXT; + String listener() default ""; + ClusterConfigProperty[] serverProperties() default {}; +} diff --git a/core/src/test/java/kafka/test/annotation/ClusterTestDefaults.java b/core/src/test/java/kafka/test/annotation/ClusterTestDefaults.java new file mode 100644 index 0000000000..cd8a66dfda --- /dev/null +++ b/core/src/test/java/kafka/test/annotation/ClusterTestDefaults.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.test.annotation; + +import kafka.test.junit.ClusterTestExtensions; + +import java.lang.annotation.Documented; +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.ElementType.TYPE; +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +/** + * Used to set class level defaults for any test template methods annotated with {@link ClusterTest} or + * {@link ClusterTests}. The default values here are also used as the source for defaults in + * {@link ClusterTestExtensions}. + */ +@Documented +@Target({TYPE}) +@Retention(RUNTIME) +public @interface ClusterTestDefaults { + Type clusterType() default Type.ZK; + int brokers() default 1; + int controllers() default 1; + boolean autoStart() default true; +} diff --git a/core/src/test/java/kafka/test/annotation/ClusterTests.java b/core/src/test/java/kafka/test/annotation/ClusterTests.java new file mode 100644 index 0000000000..64905f8810 --- /dev/null +++ b/core/src/test/java/kafka/test/annotation/ClusterTests.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.test.annotation; + +import org.junit.jupiter.api.TestTemplate; + +import java.lang.annotation.Documented; +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.ElementType.METHOD; +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +@Documented +@Target({METHOD}) +@Retention(RUNTIME) +@TestTemplate +public @interface ClusterTests { + ClusterTest[] value(); +} diff --git a/core/src/test/java/kafka/test/annotation/Type.java b/core/src/test/java/kafka/test/annotation/Type.java new file mode 100644 index 0000000000..8e8f23627c --- /dev/null +++ b/core/src/test/java/kafka/test/annotation/Type.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.test.annotation; + +/** + * The type of cluster config being requested. Used by {@link kafka.test.ClusterConfig} and the test annotations. + */ +public enum Type { + // RAFT, + ZK, + BOTH, + DEFAULT +} diff --git a/core/src/test/java/kafka/test/junit/ClusterInstanceParameterResolver.java b/core/src/test/java/kafka/test/junit/ClusterInstanceParameterResolver.java new file mode 100644 index 0000000000..3329e328b5 --- /dev/null +++ b/core/src/test/java/kafka/test/junit/ClusterInstanceParameterResolver.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.test.junit; + +import kafka.test.ClusterInstance; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtensionContext; +import org.junit.jupiter.api.extension.ParameterContext; +import org.junit.jupiter.api.extension.ParameterResolver; + +import java.lang.reflect.Executable; + +import static org.junit.platform.commons.util.AnnotationUtils.isAnnotated; + +/** + * This resolver provides an instance of {@link ClusterInstance} to a test invocation. The instance represents the + * underlying cluster being run for the current test. It can be injected into test methods or into the class + * constructor. + * + * N.B., if injected into the class constructor, the instance will not be fully initialized until the actual test method + * is being invoked. This is because the cluster is not started until after class construction and after "before" + * lifecycle methods have been run. Constructor injection is meant for convenience so helper methods can be defined on + * the test which can rely on a class member rather than an argument for ClusterInstance. + */ +public class ClusterInstanceParameterResolver implements ParameterResolver { + private final ClusterInstance clusterInstance; + + ClusterInstanceParameterResolver(ClusterInstance clusterInstance) { + this.clusterInstance = clusterInstance; + } + + @Override + public boolean supportsParameter(ParameterContext parameterContext, ExtensionContext extensionContext) { + if (!parameterContext.getParameter().getType().equals(ClusterInstance.class)) { + return false; + } + + if (!extensionContext.getTestMethod().isPresent()) { + // Allow this to be injected into the class + extensionContext.getRequiredTestClass(); + return true; + } else { + // If we're injecting into a method, make sure it's a test method and not a lifecycle method + Executable parameterizedMethod = parameterContext.getParameter().getDeclaringExecutable(); + return isAnnotated(parameterizedMethod, TestTemplate.class); + } + } + + @Override + public Object resolveParameter(ParameterContext parameterContext, ExtensionContext extensionContext) { + return clusterInstance; + } +} diff --git a/core/src/test/java/kafka/test/junit/ClusterTestExtensions.java b/core/src/test/java/kafka/test/junit/ClusterTestExtensions.java new file mode 100644 index 0000000000..872b669e21 --- /dev/null +++ b/core/src/test/java/kafka/test/junit/ClusterTestExtensions.java @@ -0,0 +1,220 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.test.junit; + +import kafka.test.ClusterConfig; +import kafka.test.ClusterGenerator; +import kafka.test.annotation.ClusterTestDefaults; +import kafka.test.annotation.ClusterConfigProperty; +import kafka.test.annotation.ClusterTemplate; +import kafka.test.annotation.ClusterTest; +import kafka.test.annotation.ClusterTests; +import kafka.test.annotation.Type; +import org.junit.jupiter.api.extension.ExtensionContext; +import org.junit.jupiter.api.extension.TestTemplateInvocationContext; +import org.junit.jupiter.api.extension.TestTemplateInvocationContextProvider; +import org.junit.platform.commons.util.ReflectionUtils; + +import java.lang.reflect.Method; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.Properties; +import java.util.function.Consumer; +import java.util.stream.Stream; + +/** + * This class is a custom JUnit extension that will generate some number of test invocations depending on the processing + * of a few custom annotations. These annotations are placed on so-called test template methods. Template methods look + * like normal JUnit test methods, but instead of being invoked directly, they are used as templates for generating + * multiple test invocations. + * + * Test class that use this extension should use one of the following annotations on each template method: + * + *
    + *
  • {@link ClusterTest}, define a single cluster configuration
  • + *
  • {@link ClusterTests}, provide multiple instances of @ClusterTest
  • + *
  • {@link ClusterTemplate}, define a static method that generates cluster configurations
  • + *
+ * + * Any combination of these annotations may be used on a given test template method. If no test invocations are + * generated after processing the annotations, an error is thrown. + * + * Depending on which annotations are used, and what values are given, different {@link ClusterConfig} will be + * generated. Each ClusterConfig is used to create an underlying Kafka cluster that is used for the actual test + * invocation. + * + * For example: + * + *
+ * @ExtendWith(value = Array(classOf[ClusterTestExtensions]))
+ * class SomeIntegrationTest {
+ *   @ClusterTest(brokers = 1, controllers = 1, clusterType = ClusterType.Both)
+ *   def someTest(): Unit = {
+ *     assertTrue(condition)
+ *   }
+ * }
+ * 
+ * + * will generate two invocations of "someTest" (since ClusterType.Both was given). For each invocation, the test class + * SomeIntegrationTest will be instantiated, lifecycle methods (before/after) will be run, and "someTest" will be invoked. + * + **/ +public class ClusterTestExtensions implements TestTemplateInvocationContextProvider { + @Override + public boolean supportsTestTemplate(ExtensionContext context) { + return true; + } + + @Override + public Stream provideTestTemplateInvocationContexts(ExtensionContext context) { + ClusterTestDefaults defaults = getClusterTestDefaults(context.getRequiredTestClass()); + List generatedContexts = new ArrayList<>(); + + // Process the @ClusterTemplate annotation + ClusterTemplate clusterTemplateAnnot = context.getRequiredTestMethod().getDeclaredAnnotation(ClusterTemplate.class); + if (clusterTemplateAnnot != null) { + processClusterTemplate(context, clusterTemplateAnnot, generatedContexts::add); + if (generatedContexts.size() == 0) { + throw new IllegalStateException("ClusterConfig generator method should provide at least one config"); + } + } + + // Process single @ClusterTest annotation + ClusterTest clusterTestAnnot = context.getRequiredTestMethod().getDeclaredAnnotation(ClusterTest.class); + if (clusterTestAnnot != null) { + processClusterTest(clusterTestAnnot, defaults, generatedContexts::add); + } + + // Process multiple @ClusterTest annotation within @ClusterTests + ClusterTests clusterTestsAnnot = context.getRequiredTestMethod().getDeclaredAnnotation(ClusterTests.class); + if (clusterTestsAnnot != null) { + for (ClusterTest annot : clusterTestsAnnot.value()) { + processClusterTest(annot, defaults, generatedContexts::add); + } + } + + if (generatedContexts.size() == 0) { + throw new IllegalStateException("Please annotate test methods with @ClusterTemplate, @ClusterTest, or " + + "@ClusterTests when using the ClusterTestExtensions provider"); + } + + return generatedContexts.stream(); + } + + private void processClusterTemplate(ExtensionContext context, ClusterTemplate annot, + Consumer testInvocations) { + // If specified, call cluster config generated method (must be static) + List generatedClusterConfigs = new ArrayList<>(); + if (!annot.value().isEmpty()) { + generateClusterConfigurations(context, annot.value(), generatedClusterConfigs::add); + } else { + // Ensure we have at least one cluster config + generatedClusterConfigs.add(ClusterConfig.defaultClusterBuilder().build()); + } + + generatedClusterConfigs.forEach(config -> { + if (config.clusterType() == Type.ZK) { + testInvocations.accept(new ZkClusterInvocationContext(config.copyOf())); + } else { + throw new IllegalStateException("Unknown cluster type " + config.clusterType()); + } + }); + } + + private void generateClusterConfigurations(ExtensionContext context, String generateClustersMethods, ClusterGenerator generator) { + Object testInstance = context.getTestInstance().orElse(null); + Method method = ReflectionUtils.getRequiredMethod(context.getRequiredTestClass(), generateClustersMethods, ClusterGenerator.class); + ReflectionUtils.invokeMethod(method, testInstance, generator); + } + + private void processClusterTest(ClusterTest annot, ClusterTestDefaults defaults, + Consumer testInvocations) { + final Type type; + if (annot.clusterType() == Type.DEFAULT) { + type = defaults.clusterType(); + } else { + type = annot.clusterType(); + } + + final int brokers; + if (annot.brokers() == 0) { + brokers = defaults.brokers(); + } else { + brokers = annot.brokers(); + } + + final int controllers; + if (annot.controllers() == 0) { + controllers = defaults.controllers(); + } else { + controllers = annot.controllers(); + } + + if (brokers <= 0 || controllers <= 0) { + throw new IllegalArgumentException("Number of brokers/controllers must be greater than zero."); + } + + final boolean autoStart; + switch (annot.autoStart()) { + case YES: + autoStart = true; + break; + case NO: + autoStart = false; + break; + case DEFAULT: + autoStart = defaults.autoStart(); + break; + default: + throw new IllegalStateException(); + } + + ClusterConfig.Builder builder = ClusterConfig.clusterBuilder(type, brokers, controllers, autoStart, annot.securityProtocol()); + if (!annot.name().isEmpty()) { + builder.name(annot.name()); + } + if (!annot.listener().isEmpty()) { + builder.listenerName(annot.listener()); + } + + Properties properties = new Properties(); + for (ClusterConfigProperty property : annot.serverProperties()) { + properties.put(property.key(), property.value()); + } + + switch (type) { + case ZK: + case BOTH: + ClusterConfig config = builder.build(); + config.serverProperties().putAll(properties); + testInvocations.accept(new ZkClusterInvocationContext(config)); + break; + } + } + + private ClusterTestDefaults getClusterTestDefaults(Class testClass) { + return Optional.ofNullable(testClass.getDeclaredAnnotation(ClusterTestDefaults.class)) + .orElseGet(() -> EmptyClass.class.getDeclaredAnnotation(ClusterTestDefaults.class)); + } + + @ClusterTestDefaults + private final static class EmptyClass { + // Just used as a convenience to get default values from the annotation + } +} diff --git a/core/src/test/java/kafka/test/junit/GenericParameterResolver.java b/core/src/test/java/kafka/test/junit/GenericParameterResolver.java new file mode 100644 index 0000000000..70387e1680 --- /dev/null +++ b/core/src/test/java/kafka/test/junit/GenericParameterResolver.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.test.junit; + +import org.junit.jupiter.api.extension.ExtensionContext; +import org.junit.jupiter.api.extension.ParameterContext; +import org.junit.jupiter.api.extension.ParameterResolver; + +/** + * This resolver is used for supplying any type of object to the test invocation. It does not restrict where the given + * type can be injected, it simply checks if the requested injection type matches the type given in the constructor. If + * it matches, the given object is returned. + * + * This is useful for injecting helper objects and objects which can be fully initialized before the test lifecycle + * begins. + */ +public class GenericParameterResolver implements ParameterResolver { + + private final T instance; + private final Class clazz; + + GenericParameterResolver(T instance, Class clazz) { + this.instance = instance; + this.clazz = clazz; + } + + @Override + public boolean supportsParameter(ParameterContext parameterContext, ExtensionContext extensionContext) { + return parameterContext.getParameter().getType().equals(clazz); + } + + @Override + public Object resolveParameter(ParameterContext parameterContext, ExtensionContext extensionContext) { + return instance; + } +} diff --git a/core/src/test/java/kafka/test/junit/README.md b/core/src/test/java/kafka/test/junit/README.md new file mode 100644 index 0000000000..dbd2bf408c --- /dev/null +++ b/core/src/test/java/kafka/test/junit/README.md @@ -0,0 +1,139 @@ +This document describes a custom JUnit extension which allows for running the same JUnit tests against multiple Kafka +cluster configurations. + +# Annotations + +A new `@ClusterTest` annotation is introduced which allows for a test to declaratively configure an underlying Kafka cluster. + +```scala +@ClusterTest +def testSomething(): Unit = { ... } +``` + +This annotation has fields for cluster type and number of brokers, as well as commonly parameterized configurations. +Arbitrary server properties can also be provided in the annotation: + +```java +@ClusterTest(clusterType = Type.Zk, securityProtocol = "PLAINTEXT", properties = { + @ClusterProperty(key = "inter.broker.protocol.version", value = "2.7-IV2"), + @ClusterProperty(key = "socket.send.buffer.bytes", value = "10240"), +}) +void testSomething() { ... } +``` + +Multiple `@ClusterTest` annotations can be given to generate more than one test invocation for the annotated method. + +```scala +@ClusterTests(Array( + @ClusterTest(securityProtocol = "PLAINTEXT"), + @ClusterTest(securityProtocol = "SASL_PLAINTEXT") +)) +def testSomething(): Unit = { ... } +``` + +A class-level `@ClusterTestDefaults` annotation is added to provide default values for `@ClusterTest` defined within +the class. The intention here is to reduce repetitive annotation declarations and also make changing defaults easier +for a class with many test cases. + +# Dynamic Configuration + +In order to allow for more flexible cluster configuration, a `@ClusterTemplate` annotation is also introduced. This +annotation takes a single string value which references a static method on the test class. This method is used to +produce any number of test configurations using a fluent builder style API. + +```java +@ClusterTemplate("generateConfigs") +void testSomething() { ... } + +static void generateConfigs(ClusterGenerator clusterGenerator) { + clusterGenerator.accept(ClusterConfig.defaultClusterBuilder() + .name("Generated Test 1") + .serverProperties(props1) + .ibp("2.7-IV1") + .build()); + clusterGenerator.accept(ClusterConfig.defaultClusterBuilder() + .name("Generated Test 2") + .serverProperties(props2) + .ibp("2.7-IV2") + .build()); + clusterGenerator.accept(ClusterConfig.defaultClusterBuilder() + .name("Generated Test 3") + .serverProperties(props3) + .build()); +} +``` + +This "escape hatch" from the simple declarative style configuration makes it easy to dynamically configure clusters. + + +# JUnit Extension + +One thing to note is that our "test*" methods are no longer _tests_, but rather they are test templates. We have added +a JUnit extension called `ClusterTestExtensions` which knows how to process these annotations in order to generate test +invocations. Test classes that wish to make use of these annotations need to explicitly register this extension: + +```scala +import kafka.test.junit.ClusterTestExtensions + +@ExtendWith(value = Array(classOf[ClusterTestExtensions])) +class ApiVersionsRequestTest { + ... +} +``` + +# JUnit Lifecycle + +The lifecycle of a test class that is extended with `ClusterTestExtensions` follows: + +* JUnit discovers test template methods that are annotated with `@ClusterTest`, `@ClusterTests`, or `@ClusterTemplate` +* `ClusterTestExtensions` is called for each of these template methods in order to generate some number of test invocations + +For each generated invocation: +* Static `@BeforeAll` methods are called +* Test class is instantiated +* Non-static `@BeforeEach` methods are called +* Kafka Cluster is started +* Test method is invoked +* Kafka Cluster is stopped +* Non-static `@AfterEach` methods are called +* Static `@AfterAll` methods are called + +`@BeforeEach` methods give an opportunity to setup additional test dependencies before the cluster is started. + +# Dependency Injection + +A few classes are introduced to provide context to the underlying cluster and to provide reusable functionality that was +previously garnered from the test hierarchy. + +* ClusterConfig: a mutable cluster configuration, includes cluster type, number of brokers, properties, etc +* ClusterInstance: a shim to the underlying class that actually runs the cluster, provides access to things like SocketServers +* IntegrationTestHelper: connection related functions taken from IntegrationTestHarness and BaseRequestTest + +In order to have one of these objects injected, simply add it as a parameter to your test class, `@BeforeEach` method, or test method. + +| Injection | Class | BeforeEach | Test | Notes +| --- | --- | --- | --- | --- | +| ClusterConfig | yes | yes | yes* | Once in the test, changing config has no effect | +| ClusterInstance | yes* | no | yes | Injectable at class level for convenience, can only be accessed inside test | +| IntegrationTestHelper | yes | yes | yes | - | + +```scala +@ExtendWith(value = Array(classOf[ClusterTestExtensions])) +class SomeTestClass(helper: IntegrationTestHelper) { + + @BeforeEach + def setup(config: ClusterConfig): Unit = { + config.serverProperties().put("foo", "bar") + } + + @ClusterTest + def testSomething(cluster: ClusterInstance): Unit = { + val topics = cluster.createAdminClient().listTopics() + } +} +``` + +# Gotchas +* Test methods annotated with JUnit's `@Test` will still be run, but no cluster will be started and no dependency + injection will happen. This is generally not what you want +* Even though ClusterConfig is accessible and mutable inside the test method, changing it will have no affect on the cluster \ No newline at end of file diff --git a/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java b/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java new file mode 100644 index 0000000000..62cc80df27 --- /dev/null +++ b/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java @@ -0,0 +1,252 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.test.junit; + +import kafka.api.IntegrationTestHarness; +import kafka.network.SocketServer; +import kafka.server.KafkaServer; +import kafka.utils.TestUtils; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.common.network.ListenerName; +import org.apache.kafka.common.security.auth.SecurityProtocol; +import kafka.test.ClusterConfig; +import kafka.test.ClusterInstance; +import org.junit.jupiter.api.extension.AfterTestExecutionCallback; +import org.junit.jupiter.api.extension.BeforeTestExecutionCallback; +import org.junit.jupiter.api.extension.Extension; +import org.junit.jupiter.api.extension.TestTemplateInvocationContext; +import scala.Option; +import scala.collection.JavaConverters; +import scala.compat.java8.OptionConverters; + +import java.io.File; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Properties; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; + +/** + * Wraps a {@link IntegrationTestHarness} inside lifecycle methods for a test invocation. Each instance of this + * class is provided with a configuration for the cluster. + * + * This context also provides parameter resolvers for: + * + *
    + *
  • ClusterConfig (the same instance passed to the constructor)
  • + *
  • ClusterInstance (includes methods to expose underlying SocketServer-s)
  • + *
  • IntegrationTestHelper (helper methods)
  • + *
+ */ +public class ZkClusterInvocationContext implements TestTemplateInvocationContext { + + private final ClusterConfig clusterConfig; + private final AtomicReference clusterReference; + + public ZkClusterInvocationContext(ClusterConfig clusterConfig) { + this.clusterConfig = clusterConfig; + this.clusterReference = new AtomicReference<>(); + } + + @Override + public String getDisplayName(int invocationIndex) { + String clusterDesc = clusterConfig.nameTags().entrySet().stream() + .map(Object::toString) + .collect(Collectors.joining(", ")); + return String.format("[Zk %d] %s", invocationIndex, clusterDesc); + } + + @Override + public List getAdditionalExtensions() { + if (clusterConfig.numControllers() != 1) { + throw new IllegalArgumentException("For ZK clusters, please specify exactly 1 controller."); + } + ClusterInstance clusterShim = new ZkClusterInstance(clusterConfig, clusterReference); + return Arrays.asList( + (BeforeTestExecutionCallback) context -> { + // We have to wait to actually create the underlying cluster until after our @BeforeEach methods + // have run. This allows tests to set up external dependencies like ZK, MiniKDC, etc. + // However, since we cannot create this instance until we are inside the test invocation, we have + // to use a container class (AtomicReference) to provide this cluster object to the test itself + + // This is what tests normally extend from to start a cluster, here we create it anonymously and + // configure the cluster using values from ClusterConfig + IntegrationTestHarness cluster = new IntegrationTestHarness() { + @Override + public Properties serverConfig() { + return clusterConfig.serverProperties(); + } + + @Override + public Properties adminClientConfig() { + return clusterConfig.adminClientProperties(); + } + + @Override + public Properties consumerConfig() { + return clusterConfig.consumerProperties(); + } + + @Override + public Properties producerConfig() { + return clusterConfig.producerProperties(); + } + + @Override + public SecurityProtocol securityProtocol() { + return clusterConfig.securityProtocol(); + } + + @Override + public ListenerName listenerName() { + return clusterConfig.listenerName().map(ListenerName::normalised) + .orElseGet(() -> ListenerName.forSecurityProtocol(securityProtocol())); + } + + @Override + public Option serverSaslProperties() { + if (clusterConfig.saslServerProperties().isEmpty()) { + return Option.empty(); + } else { + return Option.apply(clusterConfig.saslServerProperties()); + } + } + + @Override + public Option clientSaslProperties() { + if (clusterConfig.saslClientProperties().isEmpty()) { + return Option.empty(); + } else { + return Option.apply(clusterConfig.saslClientProperties()); + } + } + + @Override + public int brokerCount() { + // Controllers are also brokers in zk mode, so just use broker count + return clusterConfig.numBrokers(); + } + + @Override + public Option trustStoreFile() { + return OptionConverters.toScala(clusterConfig.trustStoreFile()); + } + }; + + clusterReference.set(cluster); + if (clusterConfig.isAutoStart()) { + clusterShim.start(); + } + }, + (AfterTestExecutionCallback) context -> clusterShim.stop(), + new ClusterInstanceParameterResolver(clusterShim), + new GenericParameterResolver<>(clusterConfig, ClusterConfig.class) + ); + } + + public static class ZkClusterInstance implements ClusterInstance { + + final AtomicReference clusterReference; + final ClusterConfig config; + final AtomicBoolean started = new AtomicBoolean(false); + final AtomicBoolean stopped = new AtomicBoolean(false); + + ZkClusterInstance(ClusterConfig config, AtomicReference clusterReference) { + this.config = config; + this.clusterReference = clusterReference; + } + + @Override + public String bootstrapServers() { + return TestUtils.bootstrapServers(clusterReference.get().servers(), clusterReference.get().listenerName()); + } + + @Override + public Collection brokerSocketServers() { + return JavaConverters.asJavaCollection(clusterReference.get().servers()).stream() + .map(KafkaServer::socketServer) + .collect(Collectors.toList()); + } + + @Override + public ListenerName clientListener() { + return clusterReference.get().listenerName(); + } + + @Override + public Collection controllerSocketServers() { + return JavaConverters.asJavaCollection(clusterReference.get().servers()).stream() + .filter(broker -> broker.kafkaController().isActive()) + .map(KafkaServer::socketServer) + .collect(Collectors.toList()); + } + + @Override + public SocketServer anyBrokerSocketServer() { + return JavaConverters.asJavaCollection(clusterReference.get().servers()).stream() + .map(KafkaServer::socketServer) + .findFirst() + .orElseThrow(() -> new RuntimeException("No broker SocketServers found")); + } + + @Override + public SocketServer anyControllerSocketServer() { + return JavaConverters.asJavaCollection(clusterReference.get().servers()).stream() + .filter(broker -> broker.kafkaController().isActive()) + .map(KafkaServer::socketServer) + .findFirst() + .orElseThrow(() -> new RuntimeException("No broker SocketServers found")); + } + + @Override + public ClusterType clusterType() { + return ClusterType.ZK; + } + + @Override + public ClusterConfig config() { + return config; + } + + @Override + public IntegrationTestHarness getUnderlying() { + return clusterReference.get(); + } + + @Override + public Admin createAdminClient(Properties configOverrides) { + return clusterReference.get().createAdminClient(configOverrides); + } + + @Override + public void start() { + if (started.compareAndSet(false, true)) { + clusterReference.get().setUp(); + } + } + + @Override + public void stop() { + if (stopped.compareAndSet(false, true)) { + clusterReference.get().tearDown(); + } + } + } +} diff --git a/core/src/test/scala/integration/kafka/api/ConsumerTopicCreationTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerTopicCreationTest.scala index e68ce2ff60..9335eac42b 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerTopicCreationTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerTopicCreationTest.scala @@ -17,6 +17,12 @@ package kafka.api +import java.lang.{Boolean => JBoolean} +import java.time.Duration +import java.util +import java.util.Collections + +import kafka.api import kafka.server.KafkaConfig import kafka.utils.TestUtils import org.apache.kafka.clients.admin.NewTopic @@ -26,15 +32,11 @@ import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.{Arguments, MethodSource} -import java.lang.{Boolean => JBoolean} -import java.time.Duration -import java.util -import java.util.Collections - /** * Tests behavior of specifying auto topic creation configuration for the consumer and broker */ class ConsumerTopicCreationTest { + @ParameterizedTest @MethodSource(Array("parameters")) def testAutoTopicCreation(brokerAutoTopicCreationEnable: JBoolean, consumerAllowAutoCreateTopics: JBoolean): Unit = { @@ -42,10 +44,26 @@ class ConsumerTopicCreationTest { testCase.setUp() try testCase.test() finally testCase.tearDown() } + + @ParameterizedTest + @MethodSource(Array("parameters")) + def testAutoTopicCreationWithForwarding(brokerAutoTopicCreationEnable: JBoolean, consumerAllowAutoCreateTopics: JBoolean): Unit = { + val testCase = new api.ConsumerTopicCreationTest.TestCaseWithForwarding(brokerAutoTopicCreationEnable, consumerAllowAutoCreateTopics) + testCase.setUp() + try testCase.test() finally testCase.tearDown() + } } object ConsumerTopicCreationTest { + private class TestCaseWithForwarding(brokerAutoTopicCreationEnable: JBoolean, consumerAllowAutoCreateTopics: JBoolean) + extends TestCase(brokerAutoTopicCreationEnable, consumerAllowAutoCreateTopics) { + + override protected def brokerCount: Int = 3 + + override def enableForwarding: Boolean = true + } + private class TestCase(brokerAutoTopicCreationEnable: JBoolean, consumerAllowAutoCreateTopics: JBoolean) extends IntegrationTestHarness { private val topic_1 = "topic-1" private val topic_2 = "topic-2" diff --git a/core/src/test/scala/integration/kafka/api/MetricsTest.scala b/core/src/test/scala/integration/kafka/api/MetricsTest.scala index 64c40993f8..068ff103db 100644 --- a/core/src/test/scala/integration/kafka/api/MetricsTest.scala +++ b/core/src/test/scala/integration/kafka/api/MetricsTest.scala @@ -23,7 +23,7 @@ import org.apache.kafka.clients.consumer.KafkaConsumer import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} import org.apache.kafka.common.{Metric, MetricName, TopicPartition} import org.apache.kafka.common.config.SaslConfigs -import org.apache.kafka.common.errors.InvalidTopicException +import org.apache.kafka.common.errors.{InvalidTopicException, UnknownTopicOrPartitionException} import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.security.authenticator.TestJaasConfig @@ -42,7 +42,7 @@ class MetricsTest extends IntegrationTestHarness with SaslSetup { private val kafkaServerJaasEntryName = s"${listenerName.value.toLowerCase(Locale.ROOT)}.${JaasTestUtils.KafkaServerContextName}" this.serverConfig.setProperty(KafkaConfig.ZkEnableSecureAclsProp, "false") - this.serverConfig.setProperty(KafkaConfig.AutoCreateTopicsEnableDoc, "false") + this.serverConfig.setProperty(KafkaConfig.AutoCreateTopicsEnableProp, "false") this.producerConfig.setProperty(ProducerConfig.LINGER_MS_CONFIG, "10") // intentionally slow message down conversion via gzip compression to ensure we can measure the time it takes this.producerConfig.setProperty(ProducerConfig.COMPRESSION_TYPE_CONFIG, "gzip") @@ -226,8 +226,8 @@ class MetricsTest extends IntegrationTestHarness with SaslSetup { val errorMetricPrefix = "kafka.network:type=RequestMetrics,name=ErrorsPerSec" verifyYammerMetricRecorded(s"$errorMetricPrefix,request=Metadata,error=NONE") + val consumer = createConsumer() try { - val consumer = createConsumer() consumer.partitionsFor("12{}!") } catch { case _: InvalidTopicException => // expected @@ -239,10 +239,12 @@ class MetricsTest extends IntegrationTestHarness with SaslSetup { assertEquals(startErrorMetricCount + 1, currentErrorMetricCount) assertTrue(currentErrorMetricCount < 10, s"Too many error metrics $currentErrorMetricCount") - // Verify that error metric is updated with producer acks=0 when no response is sent - val producer = createProducer() - sendRecords(producer, numRecords = 1, recordSize = 100, new TopicPartition("non-existent", 0)) - verifyYammerMetricRecorded(s"$errorMetricPrefix,request=Metadata,error=LEADER_NOT_AVAILABLE") + try { + consumer.partitionsFor("non-existing-topic") + } catch { + case _: UnknownTopicOrPartitionException => // expected + } + verifyYammerMetricRecorded(s"$errorMetricPrefix,request=Metadata,error=UNKNOWN_TOPIC_OR_PARTITION") } private def verifyKafkaMetric[T](name: String, metrics: java.util.Map[MetricName, _ <: Metric], entity: String, diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index 5b09b288d7..d0b9084227 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -557,6 +557,8 @@ class PlaintextConsumerTest extends BaseConsumerTest { @Test def testPartitionsForAutoCreate(): Unit = { val consumer = createConsumer() + // First call would create the topic + consumer.partitionsFor("non-exist-topic") val partitions = consumer.partitionsFor("non-exist-topic") assertFalse(partitions.isEmpty) } diff --git a/core/src/test/scala/integration/kafka/api/SaslSetup.scala b/core/src/test/scala/integration/kafka/api/SaslSetup.scala index 2bd55b855a..92370113d1 100644 --- a/core/src/test/scala/integration/kafka/api/SaslSetup.scala +++ b/core/src/test/scala/integration/kafka/api/SaslSetup.scala @@ -97,7 +97,7 @@ trait SaslSetup { (serverKeytabFile.get, clientKeytabFile.get) } - protected def jaasSections(kafkaServerSaslMechanisms: Seq[String], + def jaasSections(kafkaServerSaslMechanisms: Seq[String], kafkaClientSaslMechanism: Option[String], mode: SaslSetupMode = Both, kafkaServerEntryName: String = JaasTestUtils.KafkaServerContextName): Seq[JaasSection] = { diff --git a/core/src/test/scala/integration/kafka/server/IntegrationTestUtils.scala b/core/src/test/scala/integration/kafka/server/IntegrationTestUtils.scala new file mode 100644 index 0000000000..c960e35c59 --- /dev/null +++ b/core/src/test/scala/integration/kafka/server/IntegrationTestUtils.scala @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package integration.kafka.server + +import kafka.network.SocketServer +import kafka.utils.{NotNothing, TestUtils} +import org.apache.kafka.common.network.{ListenerName, Mode} +import org.apache.kafka.common.protocol.ApiKeys +import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, RequestHeader, RequestTestUtils, ResponseHeader} +import org.apache.kafka.common.security.auth.SecurityProtocol +import org.apache.kafka.common.utils.Utils + +import java.io.{DataInputStream, DataOutputStream} +import java.net.Socket +import java.nio.ByteBuffer +import java.util.Properties +import scala.annotation.nowarn +import scala.reflect.ClassTag + +object IntegrationTestUtils { + + private def sendRequest(socket: Socket, request: Array[Byte]): Unit = { + val outgoing = new DataOutputStream(socket.getOutputStream) + outgoing.writeInt(request.length) + outgoing.write(request) + outgoing.flush() + } + + def sendWithHeader(request: AbstractRequest, header: RequestHeader, socket: Socket): Unit = { + val serializedBytes = Utils.toArray(RequestTestUtils.serializeRequestWithHeader(header, request)) + sendRequest(socket, serializedBytes) + } + + def nextRequestHeader[T <: AbstractResponse](apiKey: ApiKeys, + apiVersion: Short, + clientId: String = "client-id", + correlationIdOpt: Option[Int] = None): RequestHeader = { + val correlationId = correlationIdOpt.getOrElse { + this.correlationId += 1 + this.correlationId + } + new RequestHeader(apiKey, apiVersion, clientId, correlationId) + } + + def send(request: AbstractRequest, + socket: Socket, + clientId: String = "client-id", + correlationId: Option[Int] = None): Unit = { + val header = nextRequestHeader(request.apiKey, request.version, clientId, correlationId) + sendWithHeader(request, header, socket) + } + + def receive[T <: AbstractResponse](socket: Socket, apiKey: ApiKeys, version: Short) + (implicit classTag: ClassTag[T], @nowarn("cat=unused") nn: NotNothing[T]): T = { + val incoming = new DataInputStream(socket.getInputStream) + val len = incoming.readInt() + + val responseBytes = new Array[Byte](len) + incoming.readFully(responseBytes) + + val responseBuffer = ByteBuffer.wrap(responseBytes) + ResponseHeader.parse(responseBuffer, apiKey.responseHeaderVersion(version)) + + AbstractResponse.parseResponse(apiKey, responseBuffer, version) match { + case response: T => response + case response => + throw new ClassCastException(s"Expected response with type ${classTag.runtimeClass}, but found ${response.getClass}") + } + } + + def sendAndReceive[T <: AbstractResponse](request: AbstractRequest, + socket: Socket, + clientId: String = "client-id", + correlationId: Option[Int] = None) + (implicit classTag: ClassTag[T], nn: NotNothing[T]): T = { + send(request, socket, clientId, correlationId) + receive[T](socket, request.apiKey, request.version) + } + + def connectAndReceive[T <: AbstractResponse](request: AbstractRequest, + destination: SocketServer, + listenerName: ListenerName) + (implicit classTag: ClassTag[T], nn: NotNothing[T]): T = { + val socket = connect(destination, listenerName) + try sendAndReceive[T](request, socket) + finally socket.close() + } + + protected def securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT + private var correlationId = 0 + + def connect(socketServer: SocketServer, + listenerName: ListenerName): Socket = { + new Socket("localhost", socketServer.boundPort(listenerName)) + } + + def clientSecurityProps(certAlias: String): Properties = { + TestUtils.securityConfigs(Mode.CLIENT, securityProtocol, None, certAlias, TestUtils.SslCertificateCn, None) // TODO use real trust store and client SASL properties + } +} diff --git a/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala index 1633ffa557..a897fe8ebf 100644 --- a/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala @@ -18,11 +18,11 @@ package kafka.cluster import java.io.File import java.util.Properties - import kafka.api.ApiVersion import kafka.log.{CleanerConfig, LogConfig, LogManager} import kafka.server.{Defaults, MetadataCache} import kafka.server.checkpoints.OffsetCheckpoints +import kafka.server.metadata.CachedConfigRepository import kafka.utils.TestUtils.{MockAlterIsrManager, MockIsrChangeListener} import kafka.utils.{MockTime, TestUtils} import org.apache.kafka.common.TopicPartition @@ -31,9 +31,13 @@ import org.junit.jupiter.api.{AfterEach, BeforeEach} import org.mockito.ArgumentMatchers import org.mockito.Mockito.{mock, when} +object AbstractPartitionTest { + val brokerId = 101 +} + class AbstractPartitionTest { - val brokerId = 101 + val brokerId = AbstractPartitionTest.brokerId val topicPartition = new TopicPartition("test-topic", 0) val time = new MockTime() var tmpDir: File = _ @@ -43,7 +47,7 @@ class AbstractPartitionTest { var alterIsrManager: MockAlterIsrManager = _ var isrChangeListener: MockIsrChangeListener = _ var logConfig: LogConfig = _ - var topicConfigProvider: TopicConfigFetcher = _ + var configRepository: CachedConfigRepository = _ val delayedOperations: DelayedOperations = mock(classOf[DelayedOperations]) val metadataCache: MetadataCache = mock(classOf[MetadataCache]) val offsetCheckpoints: OffsetCheckpoints = mock(classOf[OffsetCheckpoints]) @@ -55,14 +59,14 @@ class AbstractPartitionTest { val logProps = createLogProperties(Map.empty) logConfig = LogConfig(logProps) - topicConfigProvider = TestUtils.createTopicConfigProvider(logProps) + configRepository = TestUtils.createConfigRepository(topicPartition.topic(), logProps) tmpDir = TestUtils.tempDir() logDir1 = TestUtils.randomPartitionLogDir(tmpDir) logDir2 = TestUtils.randomPartitionLogDir(tmpDir) - logManager = TestUtils.createLogManager( - logDirs = Seq(logDir1, logDir2), defaultConfig = logConfig, CleanerConfig(enableCleaner = false), time) - logManager.startup() + logManager = TestUtils.createLogManager(Seq(logDir1, logDir2), logConfig, configRepository, + CleanerConfig(enableCleaner = false), time) + logManager.startup(Set.empty) alterIsrManager = TestUtils.createAlterIsrManager() isrChangeListener = TestUtils.createIsrChangeListener() @@ -71,7 +75,6 @@ class AbstractPartitionTest { interBrokerProtocolVersion = ApiVersion.latestVersion, localBrokerId = brokerId, time, - topicConfigProvider, isrChangeListener, delayedOperations, metadataCache, diff --git a/core/src/test/scala/unit/kafka/cluster/AssignmentStateTest.scala b/core/src/test/scala/unit/kafka/cluster/AssignmentStateTest.scala index 098988c0f6..5e087c5e40 100644 --- a/core/src/test/scala/unit/kafka/cluster/AssignmentStateTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/AssignmentStateTest.scala @@ -23,7 +23,8 @@ import org.junit.jupiter.params.provider.{Arguments, MethodSource} import scala.jdk.CollectionConverters._ -object AssignmentStateTest extends AbstractPartitionTest { +object AssignmentStateTest { + import AbstractPartitionTest._ def parameters: java.util.stream.Stream[Arguments] = Seq[Arguments]( Arguments.of( diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala index de13144490..b105df3110 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala @@ -63,11 +63,15 @@ class PartitionLockTest extends Logging { var logManager: LogManager = _ var partition: Partition = _ + private val topicPartition = new TopicPartition("test-topic", 0) + @BeforeEach def setUp(): Unit = { val logConfig = new LogConfig(new Properties) - logManager = TestUtils.createLogManager(Seq(logDir), logConfig, CleanerConfig(enableCleaner = false), mockTime) - partition = setupPartitionWithMocks(logManager, logConfig) + val configRepository = TestUtils.createConfigRepository(topicPartition.topic, createLogProperties(Map.empty)) + logManager = TestUtils.createLogManager(Seq(logDir), logConfig, configRepository, + CleanerConfig(enableCleaner = false), mockTime) + partition = setupPartitionWithMocks(logManager) } @AfterEach @@ -244,24 +248,21 @@ class PartitionLockTest extends Logging { }): Runnable) } - private def setupPartitionWithMocks(logManager: LogManager, logConfig: LogConfig): Partition = { + private def setupPartitionWithMocks(logManager: LogManager): Partition = { val leaderEpoch = 1 val brokerId = 0 - val topicPartition = new TopicPartition("test-topic", 0) - val topicConfigProvider = TestUtils.createTopicConfigProvider(createLogProperties(Map.empty)) val isrChangeListener: IsrChangeListener = mock(classOf[IsrChangeListener]) val delayedOperations: DelayedOperations = mock(classOf[DelayedOperations]) val metadataCache: MetadataCache = mock(classOf[MetadataCache]) val offsetCheckpoints: OffsetCheckpoints = mock(classOf[OffsetCheckpoints]) val alterIsrManager: AlterIsrManager = mock(classOf[AlterIsrManager]) - logManager.startup() + logManager.startup(Set.empty) val partition = new Partition(topicPartition, replicaLagTimeMaxMs = kafka.server.Defaults.ReplicaLagTimeMaxMs, interBrokerProtocolVersion = ApiVersion.latestVersion, localBrokerId = brokerId, mockTime, - topicConfigProvider, isrChangeListener, delayedOperations, metadataCache, diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala index e42b0392c2..9e66969602 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala @@ -51,7 +51,7 @@ class PartitionTest extends AbstractPartitionTest { @Test def testLastFetchedOffsetValidation(): Unit = { - val log = logManager.getOrCreateLog(topicPartition, () => logConfig) + val log = logManager.getOrCreateLog(topicPartition) def append(leaderEpoch: Int, count: Int): Unit = { val recordArray = (1 to count).map { i => new SimpleRecord(s"$i".getBytes) @@ -130,7 +130,7 @@ class PartitionTest extends AbstractPartitionTest { def testMakeLeaderUpdatesEpochCache(): Unit = { val leaderEpoch = 8 - val log = logManager.getOrCreateLog(topicPartition, () => logConfig) + val log = logManager.getOrCreateLog(topicPartition) log.appendAsLeader(MemoryRecords.withRecords(0L, CompressionType.NONE, 0, new SimpleRecord("k1".getBytes, "v1".getBytes), new SimpleRecord("k2".getBytes, "v2".getBytes) @@ -154,9 +154,9 @@ class PartitionTest extends AbstractPartitionTest { def testMakeLeaderDoesNotUpdateEpochCacheForOldFormats(): Unit = { val leaderEpoch = 8 - val logConfig = LogConfig(createLogProperties(Map( - LogConfig.MessageFormatVersionProp -> kafka.api.KAFKA_0_10_2_IV0.shortVersion))) - val log = logManager.getOrCreateLog(topicPartition, () => logConfig) + configRepository.setTopicConfig(topicPartition.topic, + LogConfig.MessageFormatVersionProp, kafka.api.KAFKA_0_10_2_IV0.shortVersion) + val log = logManager.getOrCreateLog(topicPartition) log.appendAsLeader(TestUtils.records(List( new SimpleRecord("k1".getBytes, "v1".getBytes), new SimpleRecord("k2".getBytes, "v2".getBytes)), @@ -224,7 +224,6 @@ class PartitionTest extends AbstractPartitionTest { interBrokerProtocolVersion = ApiVersion.latestVersion, localBrokerId = brokerId, time, - topicConfigProvider, isrChangeListener, delayedOperations, metadataCache, @@ -714,7 +713,7 @@ class PartitionTest extends AbstractPartitionTest { private def setupPartitionWithMocks(leaderEpoch: Int, isLeader: Boolean, - log: Log = logManager.getOrCreateLog(topicPartition, () => logConfig)): Partition = { + log: Log = logManager.getOrCreateLog(topicPartition)): Partition = { partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints) val controllerEpoch = 0 @@ -1050,7 +1049,7 @@ class PartitionTest extends AbstractPartitionTest { @Test def testUpdateFollowerFetchState(): Unit = { - val log = logManager.getOrCreateLog(topicPartition, () => logConfig) + val log = logManager.getOrCreateLog(topicPartition) seedLogData(log, numRecords = 6, leaderEpoch = 4) val controllerEpoch = 0 @@ -1106,7 +1105,7 @@ class PartitionTest extends AbstractPartitionTest { @Test def testIsrExpansion(): Unit = { - val log = logManager.getOrCreateLog(topicPartition, () => logConfig) + val log = logManager.getOrCreateLog(topicPartition) seedLogData(log, numRecords = 10, leaderEpoch = 4) val controllerEpoch = 0 @@ -1167,7 +1166,7 @@ class PartitionTest extends AbstractPartitionTest { @Test def testIsrNotExpandedIfUpdateFails(): Unit = { - val log = logManager.getOrCreateLog(topicPartition, () => logConfig) + val log = logManager.getOrCreateLog(topicPartition) seedLogData(log, numRecords = 10, leaderEpoch = 4) val controllerEpoch = 0 @@ -1221,7 +1220,7 @@ class PartitionTest extends AbstractPartitionTest { @Test def testMaybeShrinkIsr(): Unit = { - val log = logManager.getOrCreateLog(topicPartition, () => logConfig) + val log = logManager.getOrCreateLog(topicPartition) seedLogData(log, numRecords = 10, leaderEpoch = 4) val controllerEpoch = 0 @@ -1268,7 +1267,7 @@ class PartitionTest extends AbstractPartitionTest { @Test def testShouldNotShrinkIsrIfPreviousFetchIsCaughtUp(): Unit = { - val log = logManager.getOrCreateLog(topicPartition, () => logConfig) + val log = logManager.getOrCreateLog(topicPartition) seedLogData(log, numRecords = 10, leaderEpoch = 4) val controllerEpoch = 0 @@ -1333,7 +1332,7 @@ class PartitionTest extends AbstractPartitionTest { @Test def testShouldNotShrinkIsrIfFollowerCaughtUpToLogEnd(): Unit = { - val log = logManager.getOrCreateLog(topicPartition, () => logConfig) + val log = logManager.getOrCreateLog(topicPartition) seedLogData(log, numRecords = 10, leaderEpoch = 4) val controllerEpoch = 0 @@ -1384,7 +1383,7 @@ class PartitionTest extends AbstractPartitionTest { @Test def testIsrNotShrunkIfUpdateFails(): Unit = { - val log = logManager.getOrCreateLog(topicPartition, () => logConfig) + val log = logManager.getOrCreateLog(topicPartition) seedLogData(log, numRecords = 10, leaderEpoch = 4) val controllerEpoch = 0 @@ -1463,7 +1462,7 @@ class PartitionTest extends AbstractPartitionTest { } def handleAlterIsrFailure(error: Errors, callback: (Int, Int, Partition) => Unit): Unit = { - val log = logManager.getOrCreateLog(topicPartition, () => logConfig) + val log = logManager.getOrCreateLog(topicPartition) seedLogData(log, numRecords = 10, leaderEpoch = 4) val controllerEpoch = 0 @@ -1510,7 +1509,7 @@ class PartitionTest extends AbstractPartitionTest { @Test def testSingleInFlightAlterIsr(): Unit = { - val log = logManager.getOrCreateLog(topicPartition, () => logConfig) + val log = logManager.getOrCreateLog(topicPartition) seedLogData(log, numRecords = 10, leaderEpoch = 4) val controllerEpoch = 0 @@ -1569,14 +1568,13 @@ class PartitionTest extends AbstractPartitionTest { interBrokerProtocolVersion = KAFKA_2_6_IV0, // shouldn't matter, but set this to a ZK isr version localBrokerId = brokerId, time, - topicConfigProvider, isrChangeListener, delayedOperations, metadataCache, logManager, zkIsrManager) - val log = logManager.getOrCreateLog(topicPartition, () => logConfig) + val log = logManager.getOrCreateLog(topicPartition) seedLogData(log, numRecords = 10, leaderEpoch = 4) val controllerEpoch = 0 @@ -1619,7 +1617,7 @@ class PartitionTest extends AbstractPartitionTest { @Test def testUseCheckpointToInitializeHighWatermark(): Unit = { - val log = logManager.getOrCreateLog(topicPartition, () => logConfig) + val log = logManager.getOrCreateLog(topicPartition) seedLogData(log, numRecords = 6, leaderEpoch = 5) when(offsetCheckpoints.fetch(logDir1.getAbsolutePath, topicPartition)) @@ -1689,7 +1687,7 @@ class PartitionTest extends AbstractPartitionTest { val topicPartition = new TopicPartition("test", 1) val partition = new Partition( topicPartition, 1000, ApiVersion.latestVersion, 0, - new SystemTime(), topicConfigProvider, mock(classOf[IsrChangeListener]), mock(classOf[DelayedOperations]), + new SystemTime(), mock(classOf[IsrChangeListener]), mock(classOf[DelayedOperations]), mock(classOf[MetadataCache]), mock(classOf[LogManager]), mock(classOf[AlterIsrManager])) val replicas = Seq(0, 1, 2, 3) @@ -1723,14 +1721,17 @@ class PartitionTest extends AbstractPartitionTest { */ @Test def testLogConfigNotDirty(): Unit = { + logManager.shutdown() + val spyConfigRepository = spy(configRepository) + logManager = TestUtils.createLogManager( + logDirs = Seq(logDir1, logDir2), defaultConfig = logConfig, configRepository = spyConfigRepository, + cleanerConfig = CleanerConfig(enableCleaner = false), time = time) val spyLogManager = spy(logManager) - val spyConfigProvider = spy(topicConfigProvider) val partition = new Partition(topicPartition, replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, interBrokerProtocolVersion = ApiVersion.latestVersion, localBrokerId = brokerId, time, - spyConfigProvider, isrChangeListener, delayedOperations, metadataCache, @@ -1741,12 +1742,10 @@ class PartitionTest extends AbstractPartitionTest { // Validate that initializingLog and finishedInitializingLog was called verify(spyLogManager).initializingLog(ArgumentMatchers.eq(topicPartition)) - verify(spyLogManager).finishedInitializingLog(ArgumentMatchers.eq(topicPartition), - ArgumentMatchers.any(), - ArgumentMatchers.any()) // This doesn't get evaluated, but needed to satisfy compilation + verify(spyLogManager).finishedInitializingLog(ArgumentMatchers.eq(topicPartition), ArgumentMatchers.any()) - // We should get config from ZK only once - verify(spyConfigProvider, times(1)).fetch() + // We should retrieve configs only once + verify(spyConfigRepository, times(1)).topicConfig(topicPartition.topic()) } /** @@ -1755,9 +1754,13 @@ class PartitionTest extends AbstractPartitionTest { */ @Test def testLogConfigDirtyAsTopicUpdated(): Unit = { - val spyConfigProvider = spy(topicConfigProvider) + logManager.shutdown() + val spyConfigRepository = spy(configRepository) + logManager = TestUtils.createLogManager( + logDirs = Seq(logDir1, logDir2), defaultConfig = logConfig, configRepository = spyConfigRepository, + cleanerConfig = CleanerConfig(enableCleaner = false), time = time) val spyLogManager = spy(logManager) - doAnswer((invocation: InvocationOnMock) => { + doAnswer((_: InvocationOnMock) => { logManager.initializingLog(topicPartition) logManager.topicConfigUpdated(topicPartition.topic()) }).when(spyLogManager).initializingLog(ArgumentMatchers.eq(topicPartition)) @@ -1767,7 +1770,6 @@ class PartitionTest extends AbstractPartitionTest { interBrokerProtocolVersion = ApiVersion.latestVersion, localBrokerId = brokerId, time, - spyConfigProvider, isrChangeListener, delayedOperations, metadataCache, @@ -1778,13 +1780,11 @@ class PartitionTest extends AbstractPartitionTest { // Validate that initializingLog and finishedInitializingLog was called verify(spyLogManager).initializingLog(ArgumentMatchers.eq(topicPartition)) - verify(spyLogManager).finishedInitializingLog(ArgumentMatchers.eq(topicPartition), - ArgumentMatchers.any(), - ArgumentMatchers.any()) // This doesn't get evaluated, but needed to satisfy compilation + verify(spyLogManager).finishedInitializingLog(ArgumentMatchers.eq(topicPartition), ArgumentMatchers.any()) - // We should get config from ZK twice, once before log is created, and second time once + // We should retrieve configs twice, once before log is created, and second time once // we find log config is dirty and refresh it. - verify(spyConfigProvider, times(2)).fetch() + verify(spyConfigRepository, times(2)).topicConfig(topicPartition.topic()) } /** @@ -1793,9 +1793,15 @@ class PartitionTest extends AbstractPartitionTest { */ @Test def testLogConfigDirtyAsBrokerUpdated(): Unit = { - val spyConfigProvider = spy(topicConfigProvider) + logManager.shutdown() + val spyConfigRepository = spy(configRepository) + logManager = TestUtils.createLogManager( + logDirs = Seq(logDir1, logDir2), defaultConfig = logConfig, configRepository = spyConfigRepository, + cleanerConfig = CleanerConfig(enableCleaner = false), time = time) + logManager.startup(Set.empty) + val spyLogManager = spy(logManager) - doAnswer((invocation: InvocationOnMock) => { + doAnswer((_: InvocationOnMock) => { logManager.initializingLog(topicPartition) logManager.brokerConfigUpdated() }).when(spyLogManager).initializingLog(ArgumentMatchers.eq(topicPartition)) @@ -1805,7 +1811,6 @@ class PartitionTest extends AbstractPartitionTest { interBrokerProtocolVersion = ApiVersion.latestVersion, localBrokerId = brokerId, time, - spyConfigProvider, isrChangeListener, delayedOperations, metadataCache, @@ -1816,13 +1821,11 @@ class PartitionTest extends AbstractPartitionTest { // Validate that initializingLog and finishedInitializingLog was called verify(spyLogManager).initializingLog(ArgumentMatchers.eq(topicPartition)) - verify(spyLogManager).finishedInitializingLog(ArgumentMatchers.eq(topicPartition), - ArgumentMatchers.any(), - ArgumentMatchers.any()) // This doesn't get evaluated, but needed to satisfy compilation + verify(spyLogManager).finishedInitializingLog(ArgumentMatchers.eq(topicPartition), ArgumentMatchers.any()) - // We should get config from ZK twice, once before log is created, and second time once + // We should get configs twice, once before log is created, and second time once // we find log config is dirty and refresh it. - verify(spyConfigProvider, times(2)).fetch() + verify(spyConfigRepository, times(2)).topicConfig(topicPartition.topic()) } private def seedLogData(log: Log, numRecords: Int, leaderEpoch: Int): Unit = { diff --git a/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala b/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala index a416c43c44..14e0f3fd7f 100644 --- a/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala @@ -158,7 +158,7 @@ object AbstractCoordinatorConcurrencyTest { } class TestReplicaManager extends ReplicaManager( - null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, None, null) { + null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, None, null, null) { var producePurgatory: DelayedOperationPurgatory[DelayedProduce] = _ var watchKeys: mutable.Set[TopicPartitionOperationKey] = _ diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorConcurrencyTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorConcurrencyTest.scala index a0b9f28713..4facf85dda 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorConcurrencyTest.scala @@ -83,8 +83,9 @@ class GroupCoordinatorConcurrencyTest extends AbstractCoordinatorConcurrencyTest heartbeatPurgatory = new DelayedOperationPurgatory[DelayedHeartbeat]("Heartbeat", timer, config.brokerId, reaperEnabled = false) joinPurgatory = new DelayedOperationPurgatory[DelayedJoin]("Rebalance", timer, config.brokerId, reaperEnabled = false) - groupCoordinator = GroupCoordinator(config, zkClient, replicaManager, heartbeatPurgatory, joinPurgatory, timer.time, new Metrics()) - groupCoordinator.startup(false) + groupCoordinator = GroupCoordinator(config, replicaManager, heartbeatPurgatory, joinPurgatory, timer.time, new Metrics()) + groupCoordinator.startup(() => zkClient.getTopicPartitionCount(Topic.GROUP_METADATA_TOPIC_NAME).getOrElse(config.offsetsTopicPartitions), + false) } @AfterEach @@ -148,9 +149,10 @@ class GroupCoordinatorConcurrencyTest extends AbstractCoordinatorConcurrencyTest if (groupCoordinator != null) groupCoordinator.shutdown() - groupCoordinator = GroupCoordinator(config, zkClient, replicaManager, heartbeatPurgatory, + groupCoordinator = GroupCoordinator(config, replicaManager, heartbeatPurgatory, joinPurgatory, timer.time, new Metrics()) - groupCoordinator.startup(false) + groupCoordinator.startup(() => zkClient.getTopicPartitionCount(Topic.GROUP_METADATA_TOPIC_NAME).getOrElse(config.offsetsTopicPartitions), + false) val members = new Group(s"group", nMembersPerGroup, groupCoordinator, replicaManager) .members diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala index 1a5e6d5f6f..3e1f82074e 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala @@ -116,8 +116,9 @@ class GroupCoordinatorTest { val heartbeatPurgatory = new DelayedOperationPurgatory[DelayedHeartbeat]("Heartbeat", timer, config.brokerId, reaperEnabled = false) val joinPurgatory = new DelayedOperationPurgatory[DelayedJoin]("Rebalance", timer, config.brokerId, reaperEnabled = false) - groupCoordinator = GroupCoordinator(config, zkClient, replicaManager, heartbeatPurgatory, joinPurgatory, timer.time, new Metrics()) - groupCoordinator.startup(enableMetadataExpiration = false) + groupCoordinator = GroupCoordinator(config, replicaManager, heartbeatPurgatory, joinPurgatory, timer.time, new Metrics()) + groupCoordinator.startup(() => zkClient.getTopicPartitionCount(Topic.GROUP_METADATA_TOPIC_NAME).getOrElse(config.offsetsTopicPartitions), + enableMetadataExpiration = false) // add the partition into the owned partition list groupPartitionId = groupCoordinator.partitionFor(groupId) diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala index 8ab9b6cefb..45eccbe952 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala @@ -31,7 +31,6 @@ import kafka.log.{AppendOrigin, Log, LogAppendInfo} import kafka.metrics.KafkaYammerMetrics import kafka.server.{FetchDataInfo, FetchLogEnd, HostedPartition, KafkaConfig, LogOffsetMetadata, ReplicaManager} import kafka.utils.{KafkaScheduler, MockTime, TestUtils} -import kafka.zk.KafkaZkClient import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Subscription import org.apache.kafka.clients.consumer.internals.ConsumerProtocol @@ -45,7 +44,7 @@ import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.utils.Utils import org.easymock.{Capture, EasyMock, IAnswer} import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.{BeforeEach, Test} +import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import scala.jdk.CollectionConverters._ import scala.collection._ @@ -68,6 +67,7 @@ class GroupMetadataManagerTest { val rebalanceTimeout = 60000 val sessionTimeout = 10000 val defaultRequireStable = false + val numOffsetsPartitions = 2 private val offsetConfig = { val config = KafkaConfig.fromProps(TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "")) @@ -83,14 +83,6 @@ class GroupMetadataManagerTest { offsetCommitRequiredAcks = config.offsetCommitRequiredAcks) } - private def mockKafkaZkClient: KafkaZkClient = { - // make two partitions of the group topic to make sure some partitions are not owned by the coordinator - val zkClient: KafkaZkClient = EasyMock.createNiceMock(classOf[KafkaZkClient]) - EasyMock.expect(zkClient.getTopicPartitionCount(Topic.GROUP_METADATA_TOPIC_NAME)).andReturn(Some(2)) - EasyMock.replay(zkClient) - zkClient - } - @BeforeEach def setUp(): Unit = { defaultOffsetRetentionMs = offsetConfig.offsetsRetentionMs @@ -98,28 +90,38 @@ class GroupMetadataManagerTest { time = new MockTime replicaManager = EasyMock.createNiceMock(classOf[ReplicaManager]) groupMetadataManager = new GroupMetadataManager(0, ApiVersion.latestVersion, offsetConfig, replicaManager, - mockKafkaZkClient, time, metrics) + time, metrics) + groupMetadataManager.startup(() => numOffsetsPartitions, false) partition = EasyMock.niceMock(classOf[Partition]) } + @AfterEach + def tearDown(): Unit = { + groupMetadataManager.shutdown() + } + @Test def testLogInfoFromCleanupGroupMetadata(): Unit = { var expiredOffsets: Int = 0 var infoCount = 0 - val gmm = new GroupMetadataManager(0, ApiVersion.latestVersion, offsetConfig, replicaManager, mockKafkaZkClient, time, metrics) { + val gmm = new GroupMetadataManager(0, ApiVersion.latestVersion, offsetConfig, replicaManager, time, metrics) { override def cleanupGroupMetadata(groups: Iterable[GroupMetadata], selector: GroupMetadata => Map[TopicPartition, OffsetAndMetadata]): Int = expiredOffsets override def info(msg: => String): Unit = infoCount += 1 } - - // if there are no offsets to expire, we skip to log - gmm.cleanupGroupMetadata() - assertEquals(0, infoCount) - // if there are offsets to expire, we should log info - expiredOffsets = 100 - gmm.cleanupGroupMetadata() - assertEquals(1, infoCount) + gmm.startup(() => numOffsetsPartitions, false) + try { + // if there are no offsets to expire, we skip to log + gmm.cleanupGroupMetadata() + assertEquals(0, infoCount) + // if there are offsets to expire, we should log info + expiredOffsets = 100 + gmm.cleanupGroupMetadata() + assertEquals(1, infoCount) + } finally { + gmm.shutdown() + } } @Test diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala index b0b4eefdc1..ad33703876 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala @@ -71,8 +71,9 @@ class TransactionCoordinatorConcurrencyTest extends AbstractCoordinatorConcurren .anyTimes() EasyMock.replay(zkClient) - txnStateManager = new TransactionStateManager(0, zkClient, scheduler, replicaManager, txnConfig, time, + txnStateManager = new TransactionStateManager(0, scheduler, replicaManager, txnConfig, time, new Metrics()) + txnStateManager.startup(() => zkClient.getTopicPartitionCount(TRANSACTION_STATE_TOPIC_NAME).get) for (i <- 0 until numPartitions) txnStateManager.addLoadedTransactionsToCache(i, coordinatorEpoch, new Pool[String, TransactionMetadata]()) diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala index 8c9421ccd7..b5381e2def 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala @@ -57,7 +57,7 @@ class TransactionCoordinatorTest { transactionMarkerChannelManager, time, new LogContext) - + val transactionStatePartitionCount = 1 var result: InitProducerIdResult = _ var error: Errors = Errors.NONE @@ -966,7 +966,7 @@ class TransactionCoordinatorTest { EasyMock.replay(transactionManager, transactionMarkerChannelManager) - coordinator.startup(false) + coordinator.startup(() => transactionStatePartitionCount, false) time.sleep(TransactionStateManager.DefaultAbortTimedOutTransactionsIntervalMs) scheduler.tick() EasyMock.verify(transactionManager) @@ -1011,7 +1011,7 @@ class TransactionCoordinatorTest { EasyMock.replay(transactionManager, transactionMarkerChannelManager) - coordinator.startup(false) + coordinator.startup(() => transactionStatePartitionCount, false) time.sleep(TransactionStateManager.DefaultAbortTimedOutTransactionsIntervalMs) scheduler.tick() EasyMock.verify(transactionManager) @@ -1049,7 +1049,7 @@ class TransactionCoordinatorTest { EasyMock.replay(transactionManager, transactionMarkerChannelManager) - coordinator.startup(false) + coordinator.startup(() => transactionStatePartitionCount, false) time.sleep(TransactionStateManager.DefaultAbortTimedOutTransactionsIntervalMs) scheduler.tick() EasyMock.verify(transactionManager) diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala index f6c5fc30da..20ca0c421d 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala @@ -63,7 +63,7 @@ class TransactionStateManagerTest { val metrics = new Metrics() val txnConfig = TransactionConfig() - val transactionManager: TransactionStateManager = new TransactionStateManager(0, zkClient, scheduler, + val transactionManager: TransactionStateManager = new TransactionStateManager(0, scheduler, replicaManager, txnConfig, time, metrics) val transactionalId1: String = "one" @@ -78,6 +78,7 @@ class TransactionStateManagerTest { @BeforeEach def setUp(): Unit = { + transactionManager.startup(() => numPartitions, false) // make sure the transactional id hashes to the assigning partition id assertEquals(partitionId, transactionManager.partitionFor(transactionalId1)) assertEquals(partitionId, transactionManager.partitionFor(transactionalId2)) diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index 6eb820f544..01ca38ce80 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -20,18 +20,18 @@ package kafka.log import com.yammer.metrics.core.MetricName import kafka.metrics.KafkaYammerMetrics import kafka.server.checkpoints.OffsetCheckpointFile +import kafka.server.metadata.{CachedConfigRepository, ConfigRepository} import kafka.server.{FetchDataInfo, FetchLogEnd} import kafka.utils._ import org.apache.directory.api.util.FileUtils import org.apache.kafka.common.errors.OffsetOutOfRangeException import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.{KafkaException, TopicPartition} -import org.easymock.EasyMock import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.mockito.ArgumentMatchers.any -import org.mockito.Mockito -import org.mockito.Mockito.{doAnswer, spy} +import org.mockito.{ArgumentMatchers, Mockito} +import org.mockito.Mockito.{doAnswer, mock, never, spy, times, verify} import java.io._ import java.nio.file.Files @@ -61,7 +61,7 @@ class LogManagerTest { def setUp(): Unit = { logDir = TestUtils.tempDir() logManager = createLogManager() - logManager.startup() + logManager.startup(Set.empty) } @AfterEach @@ -79,7 +79,7 @@ class LogManagerTest { */ @Test def testCreateLog(): Unit = { - val log = logManager.getOrCreateLog(new TopicPartition(name, 0), () => logConfig) + val log = logManager.getOrCreateLog(new TopicPartition(name, 0)) assertEquals(1, logManager.liveLogDirs.size) val logFile = new File(logDir, name + "-0") @@ -102,10 +102,10 @@ class LogManagerTest { logManagerForTest = Some(createLogManager(Seq(logDir1, logDir2))) assertEquals(2, logManagerForTest.get.liveLogDirs.size) - logManagerForTest.get.startup() + logManagerForTest.get.startup(Set.empty) - val log1 = logManagerForTest.get.getOrCreateLog(new TopicPartition(name, 0), () => logConfig) - val log2 = logManagerForTest.get.getOrCreateLog(new TopicPartition(name, 1), () => logConfig) + val log1 = logManagerForTest.get.getOrCreateLog(new TopicPartition(name, 0)) + val log2 = logManagerForTest.get.getOrCreateLog(new TopicPartition(name, 1)) val logFile1 = new File(logDir1, name + "-0") assertTrue(logFile1.exists) @@ -145,9 +145,9 @@ class LogManagerTest { logManager.shutdown() logManager = createLogManager(dirs) - logManager.startup() + logManager.startup(Set.empty) - val log = logManager.getOrCreateLog(new TopicPartition(name, 0), () => logConfig, isNew = true) + val log = logManager.getOrCreateLog(new TopicPartition(name, 0), isNew = true) val logFile = new File(logDir, name + "-0") assertTrue(logFile.exists) log.appendAsLeader(TestUtils.singletonRecords("test".getBytes()), leaderEpoch = 0) @@ -175,11 +175,11 @@ class LogManagerTest { invocation.callRealMethod().asInstanceOf[Try[File]] } }.when(logManager).createLogDirectory(any(), any()) - logManager.startup() + logManager.startup(Set.empty) // Request creating a new log. // LogManager should try using all configured log directories until one succeeds. - logManager.getOrCreateLog(new TopicPartition(name, 0), () => logConfig, isNew = true) + logManager.getOrCreateLog(new TopicPartition(name, 0), isNew = true) // Verify that half the directories were considered broken, assertEquals(dirs.length / 2, brokenDirs.size) @@ -208,7 +208,7 @@ class LogManagerTest { */ @Test def testCleanupExpiredSegments(): Unit = { - val log = logManager.getOrCreateLog(new TopicPartition(name, 0), () => logConfig) + val log = logManager.getOrCreateLog(new TopicPartition(name, 0)) var offset = 0L for(_ <- 0 until 200) { val set = TestUtils.singletonRecords("test".getBytes()) @@ -245,16 +245,16 @@ class LogManagerTest { def testCleanupSegmentsToMaintainSize(): Unit = { val setSize = TestUtils.singletonRecords("test".getBytes()).sizeInBytes logManager.shutdown() - val logProps = new Properties() - logProps.put(LogConfig.SegmentBytesProp, 10 * setSize: java.lang.Integer) - logProps.put(LogConfig.RetentionBytesProp, 5L * 10L * setSize + 10L: java.lang.Long) - val config = LogConfig.fromProps(logConfig.originals, logProps) + val configRepository = new CachedConfigRepository + val segmentBytes = 10 * setSize + configRepository.setTopicConfig(name, LogConfig.SegmentBytesProp, segmentBytes.toString) + configRepository.setTopicConfig(name, LogConfig.RetentionBytesProp, (5L * 10L * setSize + 10L).toString) - logManager = createLogManager() - logManager.startup() + logManager = createLogManager(configRepository = configRepository) + logManager.startup(Set.empty) // create a log - val log = logManager.getOrCreateLog(new TopicPartition(name, 0), () => config) + val log = logManager.getOrCreateLog(new TopicPartition(name, 0)) var offset = 0L // add a bunch of messages that should be larger than the retentionSize @@ -266,7 +266,7 @@ class LogManagerTest { } log.updateHighWatermark(log.logEndOffset) - assertEquals(numMessages * setSize / config.segmentSize, log.numberOfSegments, "Check we have the expected number of segments.") + assertEquals(numMessages * setSize / segmentBytes, log.numberOfSegments, "Check we have the expected number of segments.") // this cleanup shouldn't find any expired segments but should delete some to reduce size time.sleep(logManager.InitialTaskDelayMs) @@ -301,9 +301,12 @@ class LogManagerTest { } private def testDoesntCleanLogs(policy: String): Unit = { - val logProps = new Properties() - logProps.put(LogConfig.CleanupPolicyProp, policy) - val log = logManager.getOrCreateLog(new TopicPartition(name, 0), () => LogConfig.fromProps(logConfig.originals, logProps)) + logManager.shutdown() + val configRepository = new CachedConfigRepository + configRepository.setTopicConfig(name, LogConfig.CleanupPolicyProp, policy) + + logManager = createLogManager(configRepository = configRepository) + val log = logManager.getOrCreateLog(new TopicPartition(name, 0)) var offset = 0L for (_ <- 0 until 200) { val set = TestUtils.singletonRecords("test".getBytes(), key="test".getBytes()) @@ -326,13 +329,12 @@ class LogManagerTest { @Test def testTimeBasedFlush(): Unit = { logManager.shutdown() - val logProps = new Properties() - logProps.put(LogConfig.FlushMsProp, 1000: java.lang.Integer) - val config = LogConfig.fromProps(logConfig.originals, logProps) + val configRepository = new CachedConfigRepository + configRepository.setTopicConfig(name, LogConfig.FlushMsProp, "1000") - logManager = createLogManager() - logManager.startup() - val log = logManager.getOrCreateLog(new TopicPartition(name, 0), () => config) + logManager = createLogManager(configRepository = configRepository) + logManager.startup(Set.empty) + val log = logManager.getOrCreateLog(new TopicPartition(name, 0)) val lastFlush = log.lastFlushTime for (_ <- 0 until 200) { val set = TestUtils.singletonRecords("test".getBytes()) @@ -356,7 +358,7 @@ class LogManagerTest { // verify that logs are always assigned to the least loaded partition for(partition <- 0 until 20) { - logManager.getOrCreateLog(new TopicPartition("test", partition), () => logConfig) + logManager.getOrCreateLog(new TopicPartition("test", partition)) assertEquals(partition + 1, logManager.allLogs.size, "We should have created the right number of logs") val counts = logManager.allLogs.groupBy(_.dir.getParent).values.map(_.size) assertTrue(counts.max <= counts.min + 1, "Load should balance evenly") @@ -386,7 +388,7 @@ class LogManagerTest { def testRecoveryDirectoryMappingWithTrailingSlash(): Unit = { logManager.shutdown() logManager = TestUtils.createLogManager(logDirs = Seq(new File(TestUtils.tempDir().getAbsolutePath + File.separator))) - logManager.startup() + logManager.startup(Set.empty) verifyCheckpointRecovery(Seq(new TopicPartition("test-a", 1)), logManager, logManager.liveLogDirs.head) } @@ -397,12 +399,12 @@ class LogManagerTest { def testRecoveryDirectoryMappingWithRelativeDirectory(): Unit = { logManager.shutdown() logManager = createLogManager(Seq(new File("data", logDir.getName).getAbsoluteFile)) - logManager.startup() + logManager.startup(Set.empty) verifyCheckpointRecovery(Seq(new TopicPartition("test-a", 1)), logManager, logManager.liveLogDirs.head) } private def verifyCheckpointRecovery(topicPartitions: Seq[TopicPartition], logManager: LogManager, logDir: File): Unit = { - val logs = topicPartitions.map(logManager.getOrCreateLog(_, () => logConfig)) + val logs = topicPartitions.map(logManager.getOrCreateLog(_)) logs.foreach { log => for (_ <- 0 until 50) log.appendAsLeader(TestUtils.singletonRecords("test".getBytes()), leaderEpoch = 0) @@ -418,16 +420,18 @@ class LogManagerTest { } } - private def createLogManager(logDirs: Seq[File] = Seq(this.logDir)): LogManager = { + private def createLogManager(logDirs: Seq[File] = Seq(this.logDir), + configRepository: ConfigRepository = new CachedConfigRepository): LogManager = { TestUtils.createLogManager( defaultConfig = logConfig, + configRepository = configRepository, logDirs = logDirs, time = this.time) } @Test def testFileReferencesAfterAsyncDelete(): Unit = { - val log = logManager.getOrCreateLog(new TopicPartition(name, 0), () => logConfig) + val log = logManager.getOrCreateLog(new TopicPartition(name, 0)) val activeSegment = log.activeSegment val logName = activeSegment.log.file.getName val indexName = activeSegment.offsetIndex.file.getName @@ -464,7 +468,7 @@ class LogManagerTest { @Test def testCreateAndDeleteOverlyLongTopic(): Unit = { val invalidTopicName = String.join("", Collections.nCopies(253, "x")) - logManager.getOrCreateLog(new TopicPartition(invalidTopicName, 0), () => logConfig) + logManager.getOrCreateLog(new TopicPartition(invalidTopicName, 0)) logManager.asyncDelete(new TopicPartition(invalidTopicName, 0)) } @@ -477,7 +481,7 @@ class LogManagerTest { new TopicPartition("test-b", 0), new TopicPartition("test-b", 1)) - val allLogs = tps.map(logManager.getOrCreateLog(_, () => logConfig)) + val allLogs = tps.map(logManager.getOrCreateLog(_)) allLogs.foreach { log => for (_ <- 0 until 50) log.appendAsLeader(TestUtils.singletonRecords("test".getBytes), leaderEpoch = 0) @@ -504,52 +508,53 @@ class LogManagerTest { */ @Test def testTopicConfigChangeUpdatesLogConfig(): Unit = { + logManager.shutdown() + val spyConfigRepository = spy(new CachedConfigRepository) + logManager = createLogManager(configRepository = spyConfigRepository) + val spyLogManager = spy(logManager) + val mockLog = mock(classOf[Log]) + val testTopicOne = "test-topic-one" val testTopicTwo = "test-topic-two" - val testTopicOnePartition: TopicPartition = new TopicPartition(testTopicOne, 1) - val testTopicTwoPartition: TopicPartition = new TopicPartition(testTopicTwo, 1) - val mockLog: Log = EasyMock.mock(classOf[Log]) + val testTopicOnePartition = new TopicPartition(testTopicOne, 1) + val testTopicTwoPartition = new TopicPartition(testTopicTwo, 1) - logManager.initializingLog(testTopicOnePartition) - logManager.initializingLog(testTopicTwoPartition) + spyLogManager.initializingLog(testTopicOnePartition) + spyLogManager.initializingLog(testTopicTwoPartition) - logManager.topicConfigUpdated(testTopicOne) + spyLogManager.topicConfigUpdated(testTopicOne) - val logConfig: LogConfig = null - var configUpdated = false - logManager.finishedInitializingLog(testTopicOnePartition, Some(mockLog), () => { - configUpdated = true - logConfig - }) - assertTrue(configUpdated) + spyLogManager.finishedInitializingLog(testTopicOnePartition, Some(mockLog)) + spyLogManager.finishedInitializingLog(testTopicTwoPartition, Some(mockLog)) - var configNotUpdated = true - logManager.finishedInitializingLog(testTopicTwoPartition, Some(mockLog), () => { - configNotUpdated = false - logConfig - }) - assertTrue(configNotUpdated) + // testTopicOne configs loaded again due to the update + verify(spyLogManager).initializingLog(ArgumentMatchers.eq(testTopicOnePartition)) + verify(spyLogManager).finishedInitializingLog(ArgumentMatchers.eq(testTopicOnePartition), ArgumentMatchers.any()) + verify(spyConfigRepository, times(1)).topicConfig(testTopicOne) + + // testTopicTwo configs not loaded again since there was no update + verify(spyLogManager).initializingLog(ArgumentMatchers.eq(testTopicTwoPartition)) + verify(spyLogManager).finishedInitializingLog(ArgumentMatchers.eq(testTopicTwoPartition), ArgumentMatchers.any()) + verify(spyConfigRepository, never).topicConfig(testTopicTwo) } /** * Test if an error occurs when creating log, log manager removes corresponding - * topic partition from the list of initializing partitions. + * topic partition from the list of initializing partitions and no configs are retrieved. */ @Test def testConfigChangeGetsCleanedUp(): Unit = { - val testTopicPartition: TopicPartition = new TopicPartition("test-topic", 1) - - logManager.initializingLog(testTopicPartition) + logManager.shutdown() + val spyConfigRepository = spy(new CachedConfigRepository) + logManager = createLogManager(configRepository = spyConfigRepository) + val spyLogManager = spy(logManager) - val logConfig: LogConfig = null - var configUpdateNotCalled = true - logManager.finishedInitializingLog(testTopicPartition, None, () => { - configUpdateNotCalled = false - logConfig - }) + val testTopicPartition = new TopicPartition("test-topic", 1) + spyLogManager.initializingLog(testTopicPartition) + spyLogManager.finishedInitializingLog(testTopicPartition, None) assertTrue(logManager.partitionsInitializing.isEmpty) - assertTrue(configUpdateNotCalled) + verify(spyConfigRepository, never).topicConfig(testTopicPartition.topic) } /** @@ -558,29 +563,27 @@ class LogManagerTest { */ @Test def testBrokerConfigChangeDeliveredToAllLogs(): Unit = { + logManager.shutdown() + val spyConfigRepository = spy(new CachedConfigRepository) + logManager = createLogManager(configRepository = spyConfigRepository) + val spyLogManager = spy(logManager) + val mockLog = mock(classOf[Log]) + val testTopicOne = "test-topic-one" val testTopicTwo = "test-topic-two" - val testTopicOnePartition: TopicPartition = new TopicPartition(testTopicOne, 1) - val testTopicTwoPartition: TopicPartition = new TopicPartition(testTopicTwo, 1) - val mockLog: Log = EasyMock.mock(classOf[Log]) + val testTopicOnePartition = new TopicPartition(testTopicOne, 1) + val testTopicTwoPartition = new TopicPartition(testTopicTwo, 1) - logManager.initializingLog(testTopicOnePartition) - logManager.initializingLog(testTopicTwoPartition) + spyLogManager.initializingLog(testTopicOnePartition) + spyLogManager.initializingLog(testTopicTwoPartition) - logManager.brokerConfigUpdated() + spyLogManager.brokerConfigUpdated() - val logConfig: LogConfig = null - var totalChanges = 0 - logManager.finishedInitializingLog(testTopicOnePartition, Some(mockLog), () => { - totalChanges += 1 - logConfig - }) - logManager.finishedInitializingLog(testTopicTwoPartition, Some(mockLog), () => { - totalChanges += 1 - logConfig - }) + spyLogManager.finishedInitializingLog(testTopicOnePartition, Some(mockLog)) + spyLogManager.finishedInitializingLog(testTopicTwoPartition, Some(mockLog)) - assertEquals(2, totalChanges) + verify(spyConfigRepository, times(1)).topicConfig(testTopicOne) + verify(spyConfigRepository, times(1)).topicConfig(testTopicTwo) } /** @@ -613,7 +616,7 @@ class LogManagerTest { } // Create the Log and assert that the metrics are present - logManager.getOrCreateLog(tp, () => logConfig) + logManager.getOrCreateLog(tp) verifyMetrics() // Trigger the deletion and assert that the metrics have been removed @@ -621,7 +624,7 @@ class LogManagerTest { assertTrue(logMetrics.isEmpty) // Recreate the Log and assert that the metrics are present - logManager.getOrCreateLog(tp, () => logConfig) + logManager.getOrCreateLog(tp) verifyMetrics() // Advance time past the file deletion delay and assert that the removed log has been deleted but the metrics @@ -636,7 +639,7 @@ class LogManagerTest { val dir1 = TestUtils.tempDir() val dir2 = TestUtils.tempDir() logManager = createLogManager(Seq(dir1, dir2)) - logManager.startup() + logManager.startup(Set.empty) val topicName = "future-log" def logMetrics: mutable.Set[MetricName] = KafkaYammerMetrics.defaultRegistry.allMetrics.keySet.asScala. @@ -654,9 +657,9 @@ class LogManagerTest { // Create the current and future logs and verify that metrics are present for both current and future logs logManager.maybeUpdatePreferredLogDir(tp, dir1.getAbsolutePath) - logManager.getOrCreateLog(tp, () => logConfig) + logManager.getOrCreateLog(tp) logManager.maybeUpdatePreferredLogDir(tp, dir2.getAbsolutePath) - logManager.getOrCreateLog(tp, () => logConfig, isFuture = true) + logManager.getOrCreateLog(tp, isFuture = true) verifyMetrics(2) // Replace the current log with the future one and verify that only one set of metrics are present diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index e9694103d7..b39dd7aeb9 100755 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -20,7 +20,6 @@ package kafka.log import java.io._ import java.nio.ByteBuffer import java.nio.file.{Files, Paths} -import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.{Callable, Executors} import java.util.regex.Pattern import java.util.{Collections, Optional, Properties} @@ -31,6 +30,7 @@ import kafka.log.Log.DeleteDirSuffix import kafka.metrics.KafkaYammerMetrics import kafka.server.checkpoints.LeaderEpochCheckpointFile import kafka.server.epoch.{EpochEntry, LeaderEpochFileCache} +import kafka.server.metadata.CachedConfigRepository import kafka.server.{BrokerTopicStats, FetchDataInfo, FetchHighWatermark, FetchIsolation, FetchLogEnd, FetchTxnCommitted, KafkaConfig, LogDirFailureChannel, LogOffsetMetadata, PartitionMetadataFile} import kafka.utils._ import org.apache.kafka.common.{InvalidRecordException, KafkaException, TopicPartition, Uuid} @@ -42,7 +42,6 @@ import org.apache.kafka.common.record._ import org.apache.kafka.common.requests.FetchResponse.AbortedTransaction import org.apache.kafka.common.requests.{ListOffsetsRequest, ListOffsetsResponse} import org.apache.kafka.common.utils.{Time, Utils} -import org.apache.kafka.metadata.BrokerState import org.easymock.EasyMock import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} @@ -95,15 +94,14 @@ class LogTest { // Create a LogManager with some overridden methods to facilitate interception of clean shutdown // flag and to inject a runtime error def interceptedLogManager(logConfig: LogConfig, logDirs: Seq[File]): LogManager = { - new LogManager(logDirs = logDirs.map(_.getAbsoluteFile), initialOfflineDirs = Array.empty[File], topicConfigs = Map(), + new LogManager(logDirs = logDirs.map(_.getAbsoluteFile), initialOfflineDirs = Array.empty[File], new CachedConfigRepository(), initialDefaultConfig = logConfig, cleanerConfig = CleanerConfig(enableCleaner = false), recoveryThreadsPerDataDir = 4, flushCheckMs = 1000L, flushRecoveryOffsetCheckpointMs = 10000L, flushStartOffsetCheckpointMs = 10000L, retentionCheckMs = 1000L, maxPidExpirationMs = 60 * 60 * 1000, scheduler = time.scheduler, time = time, - brokerState = new AtomicReference[BrokerState](BrokerState.NOT_RUNNING), brokerTopicStats = new BrokerTopicStats, logDirFailureChannel = new LogDirFailureChannel(logDirs.size)) { override def loadLog(logDir: File, hadCleanShutdown: Boolean, recoveryPoints: Map[TopicPartition, Long], - logStartOffsets: Map[TopicPartition, Long]): Log = { + logStartOffsets: Map[TopicPartition, Long], topicConfigs: Map[String, LogConfig]): Log = { val topicPartition = Log.parseTopicPartitionName(logDir) val config = topicConfigs.getOrElse(topicPartition.topic, currentDefaultConfig) @@ -130,28 +128,28 @@ class LogTest { val cleanShutdownFile = new File(logDir, Log.CleanShutdownFile) val logManager: LogManager = interceptedLogManager(logConfig, logDirs) - log = logManager.getOrCreateLog(topicPartition, () => logConfig, isNew = true) + log = logManager.getOrCreateLog(topicPartition, isNew = true) // Load logs after a clean shutdown Files.createFile(cleanShutdownFile.toPath) cleanShutdownInterceptedValue = false - logManager.loadLogs() + logManager.loadLogs(logManager.fetchTopicConfigOverrides(Set.empty)) assertTrue(cleanShutdownInterceptedValue, "Unexpected value intercepted for clean shutdown flag") assertFalse(cleanShutdownFile.exists(), "Clean shutdown file must not exist after loadLogs has completed") // Load logs without clean shutdown file cleanShutdownInterceptedValue = true - logManager.loadLogs() + logManager.loadLogs(logManager.fetchTopicConfigOverrides(Set.empty)) assertFalse(cleanShutdownInterceptedValue, "Unexpected value intercepted for clean shutdown flag") assertFalse(cleanShutdownFile.exists(), "Clean shutdown file must not exist after loadLogs has completed") // Create clean shutdown file and then simulate error while loading logs such that log loading does not complete. Files.createFile(cleanShutdownFile.toPath) simulateError = true - assertThrows(classOf[RuntimeException], () => logManager.loadLogs()) + assertThrows(classOf[RuntimeException], () => logManager.loadLogs(logManager.fetchTopicConfigOverrides(Set.empty))) assertFalse(cleanShutdownFile.exists(), "Clean shutdown file must not have existed") // Do not simulate error on next call to LogManager#loadLogs. LogManager must understand that log had unclean shutdown the last time. simulateError = false cleanShutdownInterceptedValue = true - logManager.loadLogs() + logManager.loadLogs(logManager.fetchTopicConfigOverrides(Set.empty)) assertFalse(cleanShutdownInterceptedValue, "Unexpected value for clean shutdown flag") } diff --git a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala index f5a96c7a1c..b8b904099d 100644 --- a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala +++ b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala @@ -33,14 +33,16 @@ import scala.jdk.CollectionConverters._ import kafka.log.LogConfig import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.metrics.JmxReporter +import org.apache.kafka.common.utils.Time class MetricsTest extends KafkaServerTestHarness with Logging { val numNodes = 2 val numParts = 2 + val requiredKafkaServerPrefix = "kafka.server:type=KafkaServer,name" val overridingProps = new Properties overridingProps.put(KafkaConfig.NumPartitionsProp, numParts.toString) - overridingProps.put(JmxReporter.EXCLUDE_CONFIG, "kafka.server:type=KafkaServer,name=ClusterId") + overridingProps.put(JmxReporter.EXCLUDE_CONFIG, s"$requiredKafkaServerPrefix=ClusterId") def generateConfigs = TestUtils.createBrokerConfigs(numNodes, zkConnect).map(KafkaConfig.fromProps(_, overridingProps)) @@ -74,7 +76,32 @@ class MetricsTest extends KafkaServerTestHarness with Logging { def testClusterIdMetric(): Unit = { // Check if clusterId metric exists. val metrics = KafkaYammerMetrics.defaultRegistry.allMetrics - assertEquals(metrics.keySet.asScala.count(_.getMBeanName == "kafka.server:type=KafkaServer,name=ClusterId"), 1) + assertEquals(metrics.keySet.asScala.count(_.getMBeanName == s"$requiredKafkaServerPrefix=ClusterId"), 1) + } + + @Test + def testBrokerStateMetric(): Unit = { + // Check if BrokerState metric exists. + val metrics = KafkaYammerMetrics.defaultRegistry.allMetrics + assertEquals(metrics.keySet.asScala.count(_.getMBeanName == s"$requiredKafkaServerPrefix=BrokerState"), 1) + } + + @Test + def testYammerMetricsCountMetric(): Unit = { + // Check if yammer-metrics-count metric exists. + val metrics = KafkaYammerMetrics.defaultRegistry.allMetrics + assertEquals(metrics.keySet.asScala.count(_.getMBeanName == s"$requiredKafkaServerPrefix=yammer-metrics-count"), 1) + } + + @Test + def testLinuxIoMetrics(): Unit = { + // Check if linux-disk-{read,write}-bytes metrics either do or do not exist depending on whether we are or are not + // able to collect those metrics on the platform where this test is running. + val usable = new LinuxIoMetricsCollector("/proc", Time.SYSTEM, logger.underlying).usable() + val expectedCount = if (usable) 1 else 0 + val metrics = KafkaYammerMetrics.defaultRegistry.allMetrics + Set("linux-disk-read-bytes", "linux-disk-write-bytes").foreach(name => + assertEquals(metrics.keySet.asScala.count(_.getMBeanName == s"$requiredKafkaServerPrefix=$name"), expectedCount)) } @Test @@ -83,7 +110,7 @@ class MetricsTest extends KafkaServerTestHarness with Logging { assertTrue(ManagementFactory.getPlatformMBeanServer .isRegistered(new ObjectName("kafka.controller:type=KafkaController,name=ActiveControllerCount"))) assertFalse(ManagementFactory.getPlatformMBeanServer - .isRegistered(new ObjectName("kafka.server:type=KafkaServer,name=ClusterId"))) + .isRegistered(new ObjectName(s"$requiredKafkaServerPrefix=ClusterId"))) } @Test @@ -95,7 +122,7 @@ class MetricsTest extends KafkaServerTestHarness with Logging { assertFalse(ManagementFactory.getPlatformMBeanServer .isRegistered(new ObjectName("kafka.controller:type=KafkaController,name=ActiveControllerCount"))) assertTrue(ManagementFactory.getPlatformMBeanServer - .isRegistered(new ObjectName("kafka.server:type=KafkaServer,name=ClusterId"))) + .isRegistered(new ObjectName(s"$requiredKafkaServerPrefix=ClusterId"))) } @Test diff --git a/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala index d71764e433..f7163cad10 100644 --- a/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala @@ -16,21 +16,28 @@ */ package kafka.server -import java.util.Properties +import integration.kafka.server.IntegrationTestUtils +import kafka.test.ClusterInstance import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersion import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.requests.{ApiVersionsRequest, ApiVersionsResponse} import org.junit.jupiter.api.Assertions._ +import java.util.Properties import scala.jdk.CollectionConverters._ -abstract class AbstractApiVersionsRequestTest extends BaseRequestTest { +abstract class AbstractApiVersionsRequestTest(cluster: ClusterInstance) { + + def sendApiVersionsRequest(request: ApiVersionsRequest, listenerName: ListenerName): ApiVersionsResponse = { + IntegrationTestUtils.connectAndReceive[ApiVersionsResponse](request, cluster.brokerSocketServers().asScala.head, listenerName) + } def controlPlaneListenerName = new ListenerName("CONTROLLER") // Configure control plane listener to make sure we have separate listeners for testing. - override def brokerPropertyOverrides(properties: Properties): Unit = { + def brokerPropertyOverrides(properties: Properties): Unit = { + val securityProtocol = cluster.config().securityProtocol() properties.setProperty(KafkaConfig.ControlPlaneListenerNameProp, controlPlaneListenerName.value()) properties.setProperty(KafkaConfig.ListenerSecurityProtocolMapProp, s"${controlPlaneListenerName.value()}:$securityProtocol,$securityProtocol:$securityProtocol") properties.setProperty("listeners", s"$securityProtocol://localhost:0,${controlPlaneListenerName.value()}://localhost:0") @@ -38,15 +45,15 @@ abstract class AbstractApiVersionsRequestTest extends BaseRequestTest { } def sendUnsupportedApiVersionRequest(request: ApiVersionsRequest): ApiVersionsResponse = { - val overrideHeader = nextRequestHeader(ApiKeys.API_VERSIONS, Short.MaxValue) - val socket = connect(anySocketServer) + val overrideHeader = IntegrationTestUtils.nextRequestHeader(ApiKeys.API_VERSIONS, Short.MaxValue) + val socket = IntegrationTestUtils.connect(cluster.brokerSocketServers().asScala.head, cluster.clientListener()) try { - sendWithHeader(request, overrideHeader, socket) - receive[ApiVersionsResponse](socket, ApiKeys.API_VERSIONS, 0.toShort) + IntegrationTestUtils.sendWithHeader(request, overrideHeader, socket) + IntegrationTestUtils.receive[ApiVersionsResponse](socket, ApiKeys.API_VERSIONS, 0.toShort) } finally socket.close() } - def validateApiVersionsResponse(apiVersionsResponse: ApiVersionsResponse, listenerName: ListenerName = interBrokerListenerName): Unit = { + def validateApiVersionsResponse(apiVersionsResponse: ApiVersionsResponse, listenerName: ListenerName): Unit = { val expectedApis = ApiKeys.brokerApis() if (listenerName == controlPlaneListenerName) { expectedApis.add(ApiKeys.ENVELOPE) diff --git a/core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala index 9d21fc2b41..b6036b7c5e 100644 --- a/core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala @@ -100,7 +100,7 @@ abstract class AbstractCreateTopicsRequestTest extends BaseRequestTest { request.data.topics.forEach { topic => def verifyMetadata(socketServer: SocketServer) = { val metadata = sendMetadataRequest( - new MetadataRequest.Builder(List(topic.name()).asJava, true).build()).topicMetadata.asScala + new MetadataRequest.Builder(List(topic.name()).asJava, false).build()).topicMetadata.asScala val metadataForTopic = metadata.filter(_.topic == topic.name()).head val partitions = if (!topic.assignments().isEmpty) diff --git a/core/src/test/scala/unit/kafka/server/AbstractMetadataRequestTest.scala b/core/src/test/scala/unit/kafka/server/AbstractMetadataRequestTest.scala new file mode 100644 index 0000000000..b14085137c --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/AbstractMetadataRequestTest.scala @@ -0,0 +1,61 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server + +import java.util.Properties + +import kafka.network.SocketServer +import kafka.utils.TestUtils +import org.apache.kafka.common.message.MetadataRequestData +import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.requests.{MetadataRequest, MetadataResponse} +import org.junit.jupiter.api.Assertions.assertEquals + +abstract class AbstractMetadataRequestTest extends BaseRequestTest { + + override def brokerPropertyOverrides(properties: Properties): Unit = { + properties.setProperty(KafkaConfig.OffsetsTopicPartitionsProp, "1") + properties.setProperty(KafkaConfig.DefaultReplicationFactorProp, "2") + properties.setProperty(KafkaConfig.RackProp, s"rack/${properties.getProperty(KafkaConfig.BrokerIdProp)}") + } + + protected def requestData(topics: List[String], allowAutoTopicCreation: Boolean): MetadataRequestData = { + val data = new MetadataRequestData + if (topics == null) + data.setTopics(null) + else + topics.foreach(topic => + data.topics.add( + new MetadataRequestData.MetadataRequestTopic() + .setName(topic))) + + data.setAllowAutoTopicCreation(allowAutoTopicCreation) + data + } + + protected def sendMetadataRequest(request: MetadataRequest, destination: Option[SocketServer] = None): MetadataResponse = { + connectAndReceive[MetadataResponse](request, destination = destination.getOrElse(anySocketServer)) + } + + protected def checkAutoCreatedTopic(autoCreatedTopic: String, response: MetadataResponse): Unit = { + assertEquals(Errors.LEADER_NOT_AVAILABLE, response.errors.get(autoCreatedTopic)) + assertEquals(Some(servers.head.config.numPartitions), zkClient.getTopicPartitionCount(autoCreatedTopic)) + for (i <- 0 until servers.head.config.numPartitions) + TestUtils.waitForPartitionMetadata(servers, autoCreatedTopic, i) + } +} diff --git a/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala b/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala index 0cdae57b9b..dc35bae4ab 100644 --- a/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala @@ -17,32 +17,40 @@ package kafka.server +import kafka.test.{ClusterConfig, ClusterInstance} import org.apache.kafka.common.message.ApiVersionsRequestData -import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.{ApiKeys, Errors} -import org.apache.kafka.common.requests.{ApiVersionsRequest, ApiVersionsResponse} +import org.apache.kafka.common.requests.ApiVersionsRequest +import kafka.test.annotation.ClusterTest +import kafka.test.junit.ClusterTestExtensions import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.Test +import org.junit.jupiter.api.BeforeEach +import org.junit.jupiter.api.extension.ExtendWith -class ApiVersionsRequestTest extends AbstractApiVersionsRequestTest { - override def brokerCount: Int = 1 +@ExtendWith(value = Array(classOf[ClusterTestExtensions])) +class ApiVersionsRequestTest(cluster: ClusterInstance) extends AbstractApiVersionsRequestTest(cluster) { - @Test + @BeforeEach + def setup(config: ClusterConfig): Unit = { + super.brokerPropertyOverrides(config.serverProperties()) + } + + @ClusterTest def testApiVersionsRequest(): Unit = { val request = new ApiVersionsRequest.Builder().build() - val apiVersionsResponse = sendApiVersionsRequest(request) - validateApiVersionsResponse(apiVersionsResponse) + val apiVersionsResponse = sendApiVersionsRequest(request, cluster.clientListener()) + validateApiVersionsResponse(apiVersionsResponse, cluster.clientListener()) } - @Test + @ClusterTest def testApiVersionsRequestThroughControlPlaneListener(): Unit = { val request = new ApiVersionsRequest.Builder().build() val apiVersionsResponse = sendApiVersionsRequest(request, super.controlPlaneListenerName) validateApiVersionsResponse(apiVersionsResponse, super.controlPlaneListenerName) } - @Test + @ClusterTest def testApiVersionsRequestWithUnsupportedVersion(): Unit = { val apiVersionsRequest = new ApiVersionsRequest.Builder().build() val apiVersionsResponse = sendUnsupportedApiVersionRequest(apiVersionsRequest) @@ -54,30 +62,25 @@ class ApiVersionsRequestTest extends AbstractApiVersionsRequestTest { assertEquals(ApiKeys.API_VERSIONS.latestVersion(), apiVersion.maxVersion()) } - @Test + @ClusterTest def testApiVersionsRequestValidationV0(): Unit = { val apiVersionsRequest = new ApiVersionsRequest.Builder().build(0.asInstanceOf[Short]) - val apiVersionsResponse = sendApiVersionsRequest(apiVersionsRequest) - validateApiVersionsResponse(apiVersionsResponse) + val apiVersionsResponse = sendApiVersionsRequest(apiVersionsRequest, cluster.clientListener()) + validateApiVersionsResponse(apiVersionsResponse, cluster.clientListener()) } - @Test + @ClusterTest def testApiVersionsRequestValidationV0ThroughControlPlaneListener(): Unit = { val apiVersionsRequest = new ApiVersionsRequest.Builder().build(0.asInstanceOf[Short]) val apiVersionsResponse = sendApiVersionsRequest(apiVersionsRequest, super.controlPlaneListenerName) validateApiVersionsResponse(apiVersionsResponse, super.controlPlaneListenerName) } - @Test + @ClusterTest def testApiVersionsRequestValidationV3(): Unit = { // Invalid request because Name and Version are empty by default val apiVersionsRequest = new ApiVersionsRequest(new ApiVersionsRequestData(), 3.asInstanceOf[Short]) - val apiVersionsResponse = sendApiVersionsRequest(apiVersionsRequest) + val apiVersionsResponse = sendApiVersionsRequest(apiVersionsRequest, cluster.clientListener()) assertEquals(Errors.INVALID_REQUEST.code(), apiVersionsResponse.data.errorCode()) } - - private def sendApiVersionsRequest(request: ApiVersionsRequest, - listenerName: ListenerName = super.listenerName): ApiVersionsResponse = { - connectAndReceive[ApiVersionsResponse](request, listenerName = listenerName) - } } diff --git a/core/src/test/scala/unit/kafka/server/AutoTopicCreationManagerTest.scala b/core/src/test/scala/unit/kafka/server/AutoTopicCreationManagerTest.scala new file mode 100644 index 0000000000..dc4dd06ef6 --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/AutoTopicCreationManagerTest.scala @@ -0,0 +1,194 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server + +import java.util.Properties + +import kafka.controller.KafkaController +import kafka.coordinator.group.GroupCoordinator +import kafka.coordinator.transaction.TransactionCoordinator +import kafka.utils.TestUtils +import kafka.utils.TestUtils.createBroker +import org.apache.kafka.common.internals.Topic.{GROUP_METADATA_TOPIC_NAME, TRANSACTION_STATE_TOPIC_NAME} +import org.apache.kafka.common.message.CreateTopicsRequestData +import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic +import org.apache.kafka.common.message.MetadataResponseData.MetadataResponseTopic +import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.requests._ +import org.junit.jupiter.api.Assertions.assertEquals +import org.junit.jupiter.api.{BeforeEach, Test} +import org.mockito.ArgumentMatchers.any +import org.mockito.{ArgumentMatchers, Mockito} + +import scala.collection.{Map, Seq} + +class AutoTopicCreationManagerTest { + + private val requestTimeout = 100 + private var config: KafkaConfig = _ + private val metadataCache = Mockito.mock(classOf[MetadataCache]) + private val brokerToController = Mockito.mock(classOf[BrokerToControllerChannelManager]) + private val adminManager = Mockito.mock(classOf[ZkAdminManager]) + private val controller = Mockito.mock(classOf[KafkaController]) + private val groupCoordinator = Mockito.mock(classOf[GroupCoordinator]) + private val transactionCoordinator = Mockito.mock(classOf[TransactionCoordinator]) + private var autoTopicCreationManager: AutoTopicCreationManager = _ + + private val internalTopicPartitions = 2 + private val internalTopicReplicationFactor: Short = 2 + + @BeforeEach + def setup(): Unit = { + val props = TestUtils.createBrokerConfig(1, "localhost") + props.setProperty(KafkaConfig.RequestTimeoutMsProp, requestTimeout.toString) + + props.setProperty(KafkaConfig.OffsetsTopicReplicationFactorProp, internalTopicPartitions.toString) + props.setProperty(KafkaConfig.TransactionsTopicReplicationFactorProp, internalTopicPartitions.toString) + + props.setProperty(KafkaConfig.OffsetsTopicPartitionsProp, internalTopicReplicationFactor.toString) + props.setProperty(KafkaConfig.TransactionsTopicPartitionsProp, internalTopicReplicationFactor.toString) + + config = KafkaConfig.fromProps(props) + val aliveBrokers = Seq(createBroker(0, "host0", 0), createBroker(1, "host1", 1)) + + Mockito.reset(metadataCache, controller, brokerToController, groupCoordinator, transactionCoordinator) + + Mockito.when(metadataCache.getAliveBrokers).thenReturn(aliveBrokers) + } + + @Test + def testCreateOffsetTopic(): Unit = { + Mockito.when(groupCoordinator.offsetsTopicConfigs).thenReturn(new Properties) + testCreateTopic(GROUP_METADATA_TOPIC_NAME, true, internalTopicPartitions, internalTopicReplicationFactor) + } + + @Test + def testCreateTxnTopic(): Unit = { + Mockito.when(transactionCoordinator.transactionTopicConfigs).thenReturn(new Properties) + testCreateTopic(TRANSACTION_STATE_TOPIC_NAME, true, internalTopicPartitions, internalTopicReplicationFactor) + } + + @Test + def testCreateNonInternalTopic(): Unit = { + testCreateTopic("topic", false) + } + + private def testCreateTopic(topicName: String, + isInternal: Boolean, + numPartitions: Int = 1, + replicationFactor: Short = 1): Unit = { + autoTopicCreationManager = new DefaultAutoTopicCreationManager( + config, + metadataCache, + Some(brokerToController), + adminManager, + controller, + groupCoordinator, + transactionCoordinator) + + val topicsCollection = new CreateTopicsRequestData.CreatableTopicCollection + topicsCollection.add(getNewTopic(topicName, numPartitions, replicationFactor)) + val requestBody = new CreateTopicsRequest.Builder( + new CreateTopicsRequestData() + .setTopics(topicsCollection) + .setTimeoutMs(requestTimeout)) + + Mockito.when(controller.isActive).thenReturn(false) + + // Calling twice with the same topic will only trigger one forwarding. + createTopicAndVerifyResult(Errors.UNKNOWN_TOPIC_OR_PARTITION, topicName, isInternal) + createTopicAndVerifyResult(Errors.UNKNOWN_TOPIC_OR_PARTITION, topicName, isInternal) + + Mockito.verify(brokerToController).sendRequest( + ArgumentMatchers.eq(requestBody), + any(classOf[ControllerRequestCompletionHandler])) + } + + @Test + def testCreateTopicsWithForwardingDisabled(): Unit = { + autoTopicCreationManager = new DefaultAutoTopicCreationManager( + config, + metadataCache, + None, + adminManager, + controller, + groupCoordinator, + transactionCoordinator) + + val topicName = "topic" + + Mockito.when(controller.isActive).thenReturn(false) + + createTopicAndVerifyResult(Errors.UNKNOWN_TOPIC_OR_PARTITION, topicName, false) + + Mockito.verify(adminManager).createTopics( + ArgumentMatchers.eq(0), + ArgumentMatchers.eq(false), + ArgumentMatchers.eq(Map(topicName -> getNewTopic(topicName))), + ArgumentMatchers.eq(Map.empty), + any(classOf[ControllerMutationQuota]), + any(classOf[Map[String, ApiError] => Unit])) + } + + @Test + def testNotEnoughLiveBrokers(): Unit = { + val props = TestUtils.createBrokerConfig(1, "localhost") + props.setProperty(KafkaConfig.DefaultReplicationFactorProp, 3.toString) + config = KafkaConfig.fromProps(props) + + autoTopicCreationManager = new DefaultAutoTopicCreationManager( + config, + metadataCache, + Some(brokerToController), + adminManager, + controller, + groupCoordinator, + transactionCoordinator) + + val topicName = "topic" + + Mockito.when(controller.isActive).thenReturn(false) + + createTopicAndVerifyResult(Errors.INVALID_REPLICATION_FACTOR, topicName, false) + + Mockito.verify(brokerToController, Mockito.never()).sendRequest( + any(), + any(classOf[ControllerRequestCompletionHandler])) + } + + private def createTopicAndVerifyResult(error: Errors, + topicName: String, + isInternal: Boolean): Unit = { + val topicResponses = autoTopicCreationManager.createTopics( + Set(topicName), UnboundedControllerMutationQuota) + + val expectedResponses = Seq(new MetadataResponseTopic() + .setErrorCode(error.code()) + .setIsInternal(isInternal) + .setName(topicName)) + + assertEquals(expectedResponses, topicResponses) + } + + private def getNewTopic(topicName: String, numPartitions: Int = 1, replicationFactor: Short = 1): CreatableTopic = { + new CreatableTopic() + .setName(topicName) + .setNumPartitions(numPartitions) + .setReplicationFactor(replicationFactor) + } +} diff --git a/core/src/test/scala/unit/kafka/server/ClientQuotasRequestTest.scala b/core/src/test/scala/unit/kafka/server/ClientQuotasRequestTest.scala index a011a97cc9..61bd02a063 100644 --- a/core/src/test/scala/unit/kafka/server/ClientQuotasRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ClientQuotasRequestTest.scala @@ -17,8 +17,10 @@ package kafka.server -import java.net.InetAddress +import integration.kafka.server.IntegrationTestUtils +import kafka.test.ClusterInstance +import java.net.InetAddress import org.apache.kafka.clients.admin.{ScramCredentialInfo, ScramMechanism, UserScramCredentialUpsertion} import org.apache.kafka.common.config.internals.QuotaConfigs import org.apache.kafka.common.errors.{InvalidRequestException, UnsupportedVersionException} @@ -26,23 +28,25 @@ import org.apache.kafka.common.internals.KafkaFutureImpl import org.apache.kafka.common.quota.{ClientQuotaAlteration, ClientQuotaEntity, ClientQuotaFilter, ClientQuotaFilterComponent} import org.apache.kafka.common.requests.{AlterClientQuotasRequest, AlterClientQuotasResponse, DescribeClientQuotasRequest, DescribeClientQuotasResponse} import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.Test +import org.junit.jupiter.api.extension.ExtendWith + import java.util import java.util.concurrent.{ExecutionException, TimeUnit} - import kafka.utils.TestUtils +import kafka.test.annotation.{ClusterTest, ClusterTestDefaults, Type} +import kafka.test.junit.ClusterTestExtensions import scala.jdk.CollectionConverters._ -class ClientQuotasRequestTest extends BaseRequestTest { +@ClusterTestDefaults(clusterType = Type.ZK) +@ExtendWith(value = Array(classOf[ClusterTestExtensions])) +class ClientQuotasRequestTest(cluster: ClusterInstance) { private val ConsumerByteRateProp = QuotaConfigs.CONSUMER_BYTE_RATE_OVERRIDE_CONFIG private val ProducerByteRateProp = QuotaConfigs.PRODUCER_BYTE_RATE_OVERRIDE_CONFIG private val RequestPercentageProp = QuotaConfigs.REQUEST_PERCENTAGE_OVERRIDE_CONFIG private val IpConnectionRateProp = QuotaConfigs.IP_CONNECTION_RATE_OVERRIDE_CONFIG - override val brokerCount = 1 - - @Test + @ClusterTest def testAlterClientQuotasRequest(): Unit = { val entity = new ClientQuotaEntity(Map((ClientQuotaEntity.USER -> "user"), (ClientQuotaEntity.CLIENT_ID -> "client-id")).asJava) @@ -112,7 +116,7 @@ class ClientQuotasRequestTest extends BaseRequestTest { )) } - @Test + @ClusterTest def testAlterClientQuotasRequestValidateOnly(): Unit = { val entity = new ClientQuotaEntity(Map((ClientQuotaEntity.USER -> "user")).asJava) @@ -170,11 +174,11 @@ class ClientQuotasRequestTest extends BaseRequestTest { )) } - @Test + @ClusterTest def testClientQuotasForScramUsers(): Unit = { val userName = "user" - val results = createAdminClient().alterUserScramCredentials(util.Arrays.asList( + val results = cluster.createAdminClient().alterUserScramCredentials(util.Arrays.asList( new UserScramCredentialUpsertion(userName, new ScramCredentialInfo(ScramMechanism.SCRAM_SHA_256, 4096), "password"))) results.all.get @@ -193,7 +197,7 @@ class ClientQuotasRequestTest extends BaseRequestTest { )) } - @Test + @ClusterTest def testAlterIpQuotasRequest(): Unit = { val knownHost = "1.2.3.4" val unknownHost = "2.3.4.5" @@ -218,7 +222,7 @@ class ClientQuotasRequestTest extends BaseRequestTest { var currentServerQuota = 0 TestUtils.waitUntilTrue( () => { - currentServerQuota = servers.head.socketServer.connectionQuotas.connectionRateForIp(entityIp) + currentServerQuota = cluster.brokerSocketServers().asScala.head.connectionQuotas.connectionRateForIp(entityIp) Math.abs(expectedMatches(entity) - currentServerQuota) < 0.01 }, s"Connection quota of $entity is not ${expectedMatches(entity)} but $currentServerQuota") } @@ -251,40 +255,25 @@ class ClientQuotasRequestTest extends BaseRequestTest { verifyIpQuotas(allIpEntityFilter, Map.empty) } - @Test - def testAlterClientQuotasBadUser(): Unit = { - val entity = new ClientQuotaEntity(Map((ClientQuotaEntity.USER -> "")).asJava) - assertThrows(classOf[InvalidRequestException], () => alterEntityQuotas(entity, Map(RequestPercentageProp -> Some(12.34)), validateOnly = true)) - } + @ClusterTest + def testAlterClientQuotasInvalidRequests(): Unit = { + var entity = new ClientQuotaEntity(Map((ClientQuotaEntity.USER -> "")).asJava) + assertThrows(classOf[InvalidRequestException], () => alterEntityQuotas(entity, Map((RequestPercentageProp -> Some(12.34))), validateOnly = true)) - @Test - def testAlterClientQuotasBadClientId(): Unit = { - val entity = new ClientQuotaEntity(Map((ClientQuotaEntity.CLIENT_ID -> "")).asJava) - assertThrows(classOf[InvalidRequestException], () => alterEntityQuotas(entity, Map(RequestPercentageProp -> Some(12.34)), validateOnly = true)) - } + entity = new ClientQuotaEntity(Map((ClientQuotaEntity.CLIENT_ID -> "")).asJava) + assertThrows(classOf[InvalidRequestException], () => alterEntityQuotas(entity, Map((RequestPercentageProp -> Some(12.34))), validateOnly = true)) - @Test - def testAlterClientQuotasBadEntityType(): Unit = { - val entity = new ClientQuotaEntity(Map(("" -> "name")).asJava) - assertThrows(classOf[InvalidRequestException], () => alterEntityQuotas(entity, Map(RequestPercentageProp -> Some(12.34)), validateOnly = true)) - } + entity = new ClientQuotaEntity(Map(("" -> "name")).asJava) + assertThrows(classOf[InvalidRequestException], () => alterEntityQuotas(entity, Map((RequestPercentageProp -> Some(12.34))), validateOnly = true)) - @Test - def testAlterClientQuotasEmptyEntity(): Unit = { - val entity = new ClientQuotaEntity(Map.empty.asJava) - assertThrows(classOf[InvalidRequestException], () => alterEntityQuotas(entity, Map(ProducerByteRateProp -> Some(10000.5)), validateOnly = true)) - } + entity = new ClientQuotaEntity(Map.empty.asJava) + assertThrows(classOf[InvalidRequestException], () => alterEntityQuotas(entity, Map((ProducerByteRateProp -> Some(10000.5))), validateOnly = true)) - @Test - def testAlterClientQuotasBadConfigKey(): Unit = { - val entity = new ClientQuotaEntity(Map(ClientQuotaEntity.USER -> "user").asJava) - assertThrows(classOf[InvalidRequestException], () => alterEntityQuotas(entity, Map("bad" -> Some(1.0)), validateOnly = true)) - } + entity = new ClientQuotaEntity(Map((ClientQuotaEntity.USER -> "user")).asJava) + assertThrows(classOf[InvalidRequestException], () => alterEntityQuotas(entity, Map(("bad" -> Some(1.0))), validateOnly = true)) - @Test - def testAlterClientQuotasBadConfigValue(): Unit = { - val entity = new ClientQuotaEntity(Map(ClientQuotaEntity.USER -> "user").asJava) - assertThrows(classOf[InvalidRequestException], () => alterEntityQuotas(entity, Map(ProducerByteRateProp -> Some(10000.5)), validateOnly = true)) + entity = new ClientQuotaEntity(Map((ClientQuotaEntity.USER -> "user")).asJava) + assertThrows(classOf[InvalidRequestException], () => alterEntityQuotas(entity, Map((ProducerByteRateProp -> Some(10000.5))), validateOnly = true)) } private def expectInvalidRequestWithMessage(runnable: => Unit, expectedMessage: String): Unit = { @@ -292,7 +281,7 @@ class ClientQuotasRequestTest extends BaseRequestTest { assertTrue(exception.getMessage.contains(expectedMessage), s"Expected message $exception to contain $expectedMessage") } - @Test + @ClusterTest def testAlterClientQuotasInvalidEntityCombination(): Unit = { val userAndIpEntity = new ClientQuotaEntity(Map(ClientQuotaEntity.USER -> "user", ClientQuotaEntity.IP -> "1.2.3.4").asJava) val clientAndIpEntity = new ClientQuotaEntity(Map(ClientQuotaEntity.CLIENT_ID -> "client", ClientQuotaEntity.IP -> "1.2.3.4").asJava) @@ -303,7 +292,7 @@ class ClientQuotasRequestTest extends BaseRequestTest { validateOnly = true), expectedExceptionMessage) } - @Test + @ClusterTest def testAlterClientQuotasBadIp(): Unit = { val invalidHostPatternEntity = new ClientQuotaEntity(Map(ClientQuotaEntity.IP -> "abc-123").asJava) val unresolvableHostEntity = new ClientQuotaEntity(Map(ClientQuotaEntity.IP -> "ip").asJava) @@ -314,7 +303,7 @@ class ClientQuotasRequestTest extends BaseRequestTest { validateOnly = true), expectedExceptionMessage) } - @Test + @ClusterTest def testDescribeClientQuotasInvalidFilterCombination(): Unit = { val ipFilterComponent = ClientQuotaFilterComponent.ofEntityType(ClientQuotaEntity.IP) val userFilterComponent = ClientQuotaFilterComponent.ofEntityType(ClientQuotaEntity.USER) @@ -357,7 +346,7 @@ class ClientQuotasRequestTest extends BaseRequestTest { (matchUserClientEntities ++ matchIpEntities).foreach(e => result(e._1).get(10, TimeUnit.SECONDS)) } - @Test + @ClusterTest def testDescribeClientQuotasMatchExact(): Unit = { setupDescribeClientQuotasMatchTest() @@ -402,7 +391,7 @@ class ClientQuotasRequestTest extends BaseRequestTest { } } - @Test + @ClusterTest def testDescribeClientQuotasMatchPartial(): Unit = { setupDescribeClientQuotasMatchTest() @@ -509,13 +498,13 @@ class ClientQuotasRequestTest extends BaseRequestTest { testMatchEntities(ClientQuotaFilter.containsOnly(List.empty.asJava), 0, entity => false) } - @Test + @ClusterTest def testClientQuotasUnsupportedEntityTypes(): Unit = { val entity = new ClientQuotaEntity(Map(("other" -> "name")).asJava) assertThrows(classOf[UnsupportedVersionException], () => verifyDescribeEntityQuotas(entity, Map.empty)) } - @Test + @ClusterTest def testClientQuotasSanitized(): Unit = { // An entity with name that must be sanitized when writing to Zookeeper. val entity = new ClientQuotaEntity(Map((ClientQuotaEntity.USER -> "user with spaces")).asJava) @@ -529,7 +518,7 @@ class ClientQuotasRequestTest extends BaseRequestTest { )) } - @Test + @ClusterTest def testClientQuotasWithDefaultName(): Unit = { // An entity using the name associated with the default entity name. The entity's name should be sanitized so // that it does not conflict with the default entity name. @@ -580,7 +569,9 @@ class ClientQuotasRequestTest extends BaseRequestTest { private def sendDescribeClientQuotasRequest(filter: ClientQuotaFilter): DescribeClientQuotasResponse = { val request = new DescribeClientQuotasRequest.Builder(filter).build() - connectAndReceive[DescribeClientQuotasResponse](request, destination = controllerSocketServer) + IntegrationTestUtils.connectAndReceive[DescribeClientQuotasResponse](request, + destination = cluster.anyControllerSocketServer(), + listenerName = cluster.clientListener()) } private def alterEntityQuotas(entity: ClientQuotaEntity, alter: Map[String, Option[Double]], validateOnly: Boolean) = @@ -606,7 +597,9 @@ class ClientQuotasRequestTest extends BaseRequestTest { private def sendAlterClientQuotasRequest(entries: Iterable[ClientQuotaAlteration], validateOnly: Boolean): AlterClientQuotasResponse = { val request = new AlterClientQuotasRequest.Builder(entries.asJavaCollection, validateOnly).build() - connectAndReceive[AlterClientQuotasResponse](request, destination = controllerSocketServer) + IntegrationTestUtils.connectAndReceive[AlterClientQuotasResponse](request, + destination = cluster.anyControllerSocketServer(), + listenerName = cluster.clientListener()) } } diff --git a/core/src/test/scala/unit/kafka/server/CreateTopicsRequestWithForwardingTest.scala b/core/src/test/scala/unit/kafka/server/CreateTopicsRequestWithForwardingTest.scala index a538cebcc7..f55aa34d0c 100644 --- a/core/src/test/scala/unit/kafka/server/CreateTopicsRequestWithForwardingTest.scala +++ b/core/src/test/scala/unit/kafka/server/CreateTopicsRequestWithForwardingTest.scala @@ -34,5 +34,4 @@ class CreateTopicsRequestWithForwardingTest extends AbstractCreateTopicsRequestT // With forwarding enabled, request could be forwarded to the active controller. assertEquals(Map(Errors.NONE -> 1), response.errorCounts().asScala) } - } diff --git a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala index 178cbd4d62..c39ae7cf03 100755 --- a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala +++ b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala @@ -21,14 +21,13 @@ import java.io.File import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.utils.Utils -import org.easymock.EasyMock import org.junit.jupiter.api._ import org.junit.jupiter.api.Assertions._ import kafka.utils.{KafkaScheduler, MockTime, TestUtils} -import kafka.zk.KafkaZkClient import java.util.concurrent.atomic.AtomicBoolean import kafka.cluster.Partition +import kafka.server.metadata.CachedConfigRepository import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.record.SimpleRecord @@ -36,7 +35,7 @@ class HighwatermarkPersistenceTest { val configs = TestUtils.createBrokerConfigs(2, TestUtils.MockZkConnect).map(KafkaConfig.fromProps) val topic = "foo" - val zkClient: KafkaZkClient = EasyMock.createMock(classOf[KafkaZkClient]) + val configRepository = new CachedConfigRepository() val logManagers = configs map { config => TestUtils.createLogManager( logDirs = config.logDirs.map(new File(_)), @@ -57,9 +56,6 @@ class HighwatermarkPersistenceTest { @Test def testHighWatermarkPersistenceSinglePartition(): Unit = { - // mock zkclient - EasyMock.replay(zkClient) - // create kafka scheduler val scheduler = new KafkaScheduler(2) scheduler.startup() @@ -67,9 +63,10 @@ class HighwatermarkPersistenceTest { val time = new MockTime val quotaManager = QuotaFactory.instantiate(configs.head, metrics, time, "") // create replica manager - val replicaManager = new ReplicaManager(configs.head, metrics, time, zkClient, scheduler, + val replicaManager = new ReplicaManager(configs.head, metrics, time, None, scheduler, logManagers.head, new AtomicBoolean(false), quotaManager, - new BrokerTopicStats, new MetadataCache(configs.head.brokerId), logDirFailureChannels.head, alterIsrManager) + new BrokerTopicStats, MetadataCache.zkMetadataCache(configs.head.brokerId), logDirFailureChannels.head, alterIsrManager, + configRepository) replicaManager.startup() try { replicaManager.checkpointHighWatermarks() @@ -78,7 +75,7 @@ class HighwatermarkPersistenceTest { val tp0 = new TopicPartition(topic, 0) val partition0 = replicaManager.createPartition(tp0) // create leader and follower replicas - val log0 = logManagers.head.getOrCreateLog(new TopicPartition(topic, 0), () => LogConfig()) + val log0 = logManagers.head.getOrCreateLog(new TopicPartition(topic, 0)) partition0.setLog(log0, isFutureLog = false) partition0.updateAssignmentAndIsr( @@ -96,7 +93,6 @@ class HighwatermarkPersistenceTest { replicaManager.checkpointHighWatermarks() fooPartition0Hw = hwmFor(replicaManager, topic, 0) assertEquals(log0.highWatermark, fooPartition0Hw) - EasyMock.verify(zkClient) } finally { // shutdown the replica manager upon test completion replicaManager.shutdown(false) @@ -110,8 +106,6 @@ class HighwatermarkPersistenceTest { def testHighWatermarkPersistenceMultiplePartitions(): Unit = { val topic1 = "foo1" val topic2 = "foo2" - // mock zkclient - EasyMock.replay(zkClient) // create kafka scheduler val scheduler = new KafkaScheduler(2) scheduler.startup() @@ -119,9 +113,9 @@ class HighwatermarkPersistenceTest { val time = new MockTime val quotaManager = QuotaFactory.instantiate(configs.head, metrics, time, "") // create replica manager - val replicaManager = new ReplicaManager(configs.head, metrics, time, zkClient, + val replicaManager = new ReplicaManager(configs.head, metrics, time, None, scheduler, logManagers.head, new AtomicBoolean(false), quotaManager, - new BrokerTopicStats, new MetadataCache(configs.head.brokerId), logDirFailureChannels.head, alterIsrManager) + new BrokerTopicStats, MetadataCache.zkMetadataCache(configs.head.brokerId), logDirFailureChannels.head, alterIsrManager, configRepository) replicaManager.startup() try { replicaManager.checkpointHighWatermarks() @@ -130,7 +124,7 @@ class HighwatermarkPersistenceTest { val t1p0 = new TopicPartition(topic1, 0) val topic1Partition0 = replicaManager.createPartition(t1p0) // create leader log - val topic1Log0 = logManagers.head.getOrCreateLog(t1p0, () => LogConfig()) + val topic1Log0 = logManagers.head.getOrCreateLog(t1p0) // create a local replica for topic1 topic1Partition0.setLog(topic1Log0, isFutureLog = false) replicaManager.checkpointHighWatermarks() @@ -147,7 +141,7 @@ class HighwatermarkPersistenceTest { val t2p0 = new TopicPartition(topic2, 0) val topic2Partition0 = replicaManager.createPartition(t2p0) // create leader log - val topic2Log0 = logManagers.head.getOrCreateLog(t2p0, () => LogConfig()) + val topic2Log0 = logManagers.head.getOrCreateLog(t2p0) // create a local replica for topic2 topic2Partition0.setLog(topic2Log0, isFutureLog = false) replicaManager.checkpointHighWatermarks() @@ -168,7 +162,6 @@ class HighwatermarkPersistenceTest { // verify checkpointed hw for topic 1 topic1Partition0Hw = hwmFor(replicaManager, topic1, 0) assertEquals(10L, topic1Partition0Hw) - EasyMock.verify(zkClient) } finally { // shutdown the replica manager upon test completion replicaManager.shutdown(false) diff --git a/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala b/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala index 32fb360d18..7e1069ed09 100644 --- a/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala +++ b/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala @@ -23,6 +23,7 @@ import java.util.concurrent.atomic.AtomicBoolean import kafka.cluster.Partition import kafka.log.{Log, LogManager} import kafka.server.QuotaFactory.QuotaManagers +import kafka.server.metadata.CachedConfigRepository import kafka.utils.TestUtils.MockAlterIsrManager import kafka.utils._ import org.apache.kafka.common.TopicPartition @@ -65,9 +66,9 @@ class IsrExpirationTest { alterIsrManager = TestUtils.createAlterIsrManager() quotaManager = QuotaFactory.instantiate(configs.head, metrics, time, "") - replicaManager = new ReplicaManager(configs.head, metrics, time, null, null, logManager, new AtomicBoolean(false), - quotaManager, new BrokerTopicStats, new MetadataCache(configs.head.brokerId), - new LogDirFailureChannel(configs.head.logDirs.size), alterIsrManager) + replicaManager = new ReplicaManager(configs.head, metrics, time, None, null, logManager, new AtomicBoolean(false), + quotaManager, new BrokerTopicStats, MetadataCache.zkMetadataCache(configs.head.brokerId), + new LogDirFailureChannel(configs.head.logDirs.size), alterIsrManager, new CachedConfigRepository()) } @AfterEach diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index 4992892b42..88bf8ebc3a 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -34,6 +34,7 @@ import kafka.log.AppendOrigin import kafka.network.RequestChannel import kafka.network.RequestChannel.{CloseConnectionResponse, SendResponse} import kafka.server.QuotaFactory.QuotaManagers +import kafka.server.metadata.{ConfigRepository, CachedConfigRepository} import kafka.utils.{MockTime, TestUtils} import kafka.zk.KafkaZkClient import org.apache.kafka.clients.NodeApiVersions @@ -45,6 +46,7 @@ import org.apache.kafka.common.errors.UnsupportedVersionException import org.apache.kafka.common.internals.{KafkaFutureImpl, Topic} import org.apache.kafka.common.memory.MemoryPool import org.apache.kafka.common.message.CreateTopicsRequestData.{CreatableTopic, CreatableTopicCollection} +import org.apache.kafka.common.message.DescribeConfigsResponseData.DescribeConfigsResult import org.apache.kafka.common.message.JoinGroupRequestData.JoinGroupRequestProtocol import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity import org.apache.kafka.common.message.ListOffsetsRequestData.{ListOffsetsPartition, ListOffsetsTopic} @@ -60,6 +62,8 @@ import org.apache.kafka.common.quota.{ClientQuotaAlteration, ClientQuotaEntity} import org.apache.kafka.common.record.FileRecords.TimestampAndOffset import org.apache.kafka.common.record._ import org.apache.kafka.common.replica.ClientMetadata +import org.apache.kafka.common.requests.FindCoordinatorRequest.CoordinatorType +import org.apache.kafka.common.requests.MetadataResponse.TopicMetadata import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.requests.WriteTxnMarkersRequest.TxnMarkerEntry import org.apache.kafka.common.requests.{FetchMetadata => JFetchMetadata, _} @@ -88,6 +92,8 @@ class KafkaApisTest { private val txnCoordinator: TransactionCoordinator = EasyMock.createNiceMock(classOf[TransactionCoordinator]) private val controller: KafkaController = EasyMock.createNiceMock(classOf[KafkaController]) private val forwardingManager: ForwardingManager = EasyMock.createNiceMock(classOf[ForwardingManager]) + private val autoTopicCreationManager: AutoTopicCreationManager = EasyMock.createNiceMock(classOf[AutoTopicCreationManager]) + private val hostAddress: Array[Byte] = InetAddress.getByName("192.168.1.1").getAddress private val kafkaPrincipalSerde = new KafkaPrincipalSerde { override def serialize(principal: KafkaPrincipal): Array[Byte] = Utils.utf8(principal.toString) @@ -96,7 +102,7 @@ class KafkaApisTest { private val zkClient: KafkaZkClient = EasyMock.createNiceMock(classOf[KafkaZkClient]) private val metrics = new Metrics() private val brokerId = 1 - private var metadataCache: MetadataCache = new MetadataCache(brokerId) + private var metadataCache: MetadataCache = MetadataCache.zkMetadataCache(brokerId) private val clientQuotaManager: ClientQuotaManager = EasyMock.createNiceMock(classOf[ClientQuotaManager]) private val clientRequestQuotaManager: ClientRequestQuotaManager = EasyMock.createNiceMock(classOf[ClientRequestQuotaManager]) private val clientControllerQuotaManager: ControllerMutationQuotaManager = EasyMock.createNiceMock(classOf[ControllerMutationQuotaManager]) @@ -118,10 +124,22 @@ class KafkaApisTest { def createKafkaApis(interBrokerProtocolVersion: ApiVersion = ApiVersion.latestVersion, authorizer: Option[Authorizer] = None, - enableForwarding: Boolean = false): KafkaApis = { + enableForwarding: Boolean = false, + configRepository: ConfigRepository = new CachedConfigRepository(), + raftSupport: Boolean = false, + overrideProperties: Map[String, String] = Map.empty): KafkaApis = { + val brokerFeatures = BrokerFeatures.createDefault() val cache = new FinalizedFeatureCache(brokerFeatures) - val properties = TestUtils.createBrokerConfig(brokerId, "zk") + val properties = if (raftSupport) { + val properties = TestUtils.createBrokerConfig(brokerId, "") + properties.put(KafkaConfig.NodeIdProp, brokerId.toString) + properties.put(KafkaConfig.ProcessRolesProp, "broker") + properties + } else { + TestUtils.createBrokerConfig(brokerId, "zk") + } + overrideProperties.foreach( p => properties.put(p._1, p._2)) properties.put(KafkaConfig.InterBrokerProtocolVersionProp, interBrokerProtocolVersion.toString) properties.put(KafkaConfig.LogMessageFormatVersionProp, interBrokerProtocolVersion.toString) @@ -131,15 +149,14 @@ class KafkaApisTest { None new KafkaApis(requestChannel, + if (raftSupport) RaftSupport(forwardingManager) else ZkSupport(adminManager, controller, zkClient, forwardingManagerOpt), replicaManager, - adminManager, groupCoordinator, txnCoordinator, - controller, - forwardingManagerOpt, - zkClient, + autoTopicCreationManager, brokerId, new KafkaConfig(properties), + configRepository, metadataCache, metrics, authorizer, @@ -173,30 +190,50 @@ class KafkaApisTest { .andReturn(Seq(AuthorizationResult.ALLOWED).asJava) .once() - expectNoThrottling() + val capturedResponse = expectNoThrottling() - val configResource = new ConfigResource(ConfigResource.Type.TOPIC, resourceName) - EasyMock.expect(adminManager.describeConfigs(anyObject(), EasyMock.eq(true), EasyMock.eq(false))) - .andReturn( - List(new DescribeConfigsResponseData.DescribeConfigsResult() - .setResourceName(configResource.name) - .setResourceType(configResource.`type`.id) - .setErrorCode(Errors.NONE.code) - .setConfigs(Collections.emptyList()))) + val configRepository: ConfigRepository = EasyMock.strictMock(classOf[ConfigRepository]) + val topicConfigs = new Properties() + val propName = "min.insync.replicas" + val propValue = "3" + topicConfigs.put(propName, propValue) + EasyMock.expect(configRepository.topicConfig(resourceName)).andReturn(topicConfigs) - EasyMock.replay(replicaManager, clientRequestQuotaManager, requestChannel, authorizer, - adminManager) + metadataCache = + EasyMock.partialMockBuilder(classOf[ZkMetadataCache]) + .withConstructor(classOf[Int]) + .withArgs(Int.box(brokerId)) // Need to box it for Scala 2.12 and before + .addMockedMethod("contains", classOf[String]) + .createMock() - val request = buildRequest(new DescribeConfigsRequest.Builder(new DescribeConfigsRequestData() + expect(metadataCache.contains(resourceName)).andReturn(true) + + EasyMock.replay(metadataCache, replicaManager, clientRequestQuotaManager, requestChannel, authorizer, configRepository, adminManager) + + val describeConfigsRequest = new DescribeConfigsRequest.Builder(new DescribeConfigsRequestData() .setIncludeSynonyms(true) .setResources(List(new DescribeConfigsRequestData.DescribeConfigsResource() - .setResourceName("topic-1") + .setResourceName(resourceName) .setResourceType(ConfigResource.Type.TOPIC.id)).asJava)) - .build(requestHeader.apiVersion), + .build(requestHeader.apiVersion) + val request = buildRequest(describeConfigsRequest, requestHeader = Option(requestHeader)) - createKafkaApis(authorizer = Some(authorizer)).handleDescribeConfigsRequest(request) + createKafkaApis(authorizer = Some(authorizer), configRepository = configRepository).handleDescribeConfigsRequest(request) - verify(authorizer, adminManager) + verify(authorizer, replicaManager) + + val response = readResponse(describeConfigsRequest, capturedResponse) + .asInstanceOf[DescribeConfigsResponse] + val results = response.data().results() + assertEquals(1, results.size()) + val describeConfigsResult: DescribeConfigsResult = results.get(0) + assertEquals(ConfigResource.Type.TOPIC.id, describeConfigsResult.resourceType()) + assertEquals(resourceName, describeConfigsResult.resourceName()) + val configs = describeConfigsResult.configs().asScala.filter(_.name() == propName) + assertEquals(1, configs.length) + val describeConfigsResponseData = configs(0) + assertEquals(propName, describeConfigsResponseData.name()) + assertEquals(propValue, describeConfigsResponseData.value()) } @Test @@ -857,6 +894,223 @@ class KafkaApisTest { testForwardableAPI(ApiKeys.ALTER_USER_SCRAM_CREDENTIALS, requestBuilder) } + @Test + def testFindCoordinatorAutoTopicCreationForOffsetTopic(): Unit = { + testFindCoordinatorWithTopicCreation(CoordinatorType.GROUP) + } + + @Test + def testFindCoordinatorAutoTopicCreationForTxnTopic(): Unit = { + testFindCoordinatorWithTopicCreation(CoordinatorType.TRANSACTION) + } + + @Test + def testFindCoordinatorNotEnoughBrokersForOffsetTopic(): Unit = { + testFindCoordinatorWithTopicCreation(CoordinatorType.GROUP, hasEnoughLiveBrokers = false) + } + + @Test + def testFindCoordinatorNotEnoughBrokersForTxnTopic(): Unit = { + testFindCoordinatorWithTopicCreation(CoordinatorType.TRANSACTION, hasEnoughLiveBrokers = false) + } + + private def testFindCoordinatorWithTopicCreation(coordinatorType: CoordinatorType, + hasEnoughLiveBrokers: Boolean = true): Unit = { + val authorizer: Authorizer = EasyMock.niceMock(classOf[Authorizer]) + + val requestHeader = new RequestHeader(ApiKeys.FIND_COORDINATOR, ApiKeys.FIND_COORDINATOR.latestVersion, + clientId, 0) + + val numBrokersNeeded = 3 + + setupBrokerMetadata(hasEnoughLiveBrokers, numBrokersNeeded) + + val requestTimeout = 10 + val topicConfigOverride = mutable.Map.empty[String, String] + topicConfigOverride.put(KafkaConfig.RequestTimeoutMsProp, requestTimeout.toString) + + val groupId = "group" + val topicName = + coordinatorType match { + case CoordinatorType.GROUP => + topicConfigOverride.put(KafkaConfig.OffsetsTopicPartitionsProp, numBrokersNeeded.toString) + topicConfigOverride.put(KafkaConfig.OffsetsTopicReplicationFactorProp, numBrokersNeeded.toString) + EasyMock.expect(groupCoordinator.offsetsTopicConfigs).andReturn(new Properties) + authorizeResource(authorizer, AclOperation.DESCRIBE, ResourceType.GROUP, + groupId, AuthorizationResult.ALLOWED) + Topic.GROUP_METADATA_TOPIC_NAME + case CoordinatorType.TRANSACTION => + topicConfigOverride.put(KafkaConfig.TransactionsTopicPartitionsProp, numBrokersNeeded.toString) + topicConfigOverride.put(KafkaConfig.TransactionsTopicReplicationFactorProp, numBrokersNeeded.toString) + EasyMock.expect(txnCoordinator.transactionTopicConfigs).andReturn(new Properties) + authorizeResource(authorizer, AclOperation.DESCRIBE, ResourceType.TRANSACTIONAL_ID, + groupId, AuthorizationResult.ALLOWED) + Topic.TRANSACTION_STATE_TOPIC_NAME + case _ => + throw new IllegalStateException(s"Unknown coordinator type $coordinatorType") + } + + val findCoordinatorRequest = new FindCoordinatorRequest.Builder( + new FindCoordinatorRequestData() + .setKeyType(coordinatorType.id()) + .setKey(groupId) + ).build(requestHeader.apiVersion) + val request = buildRequest(findCoordinatorRequest) + + val capturedResponse = expectNoThrottling() + + verifyTopicCreation(topicName, true, true, request) + + EasyMock.replay(replicaManager, clientRequestQuotaManager, requestChannel, authorizer, + autoTopicCreationManager, forwardingManager, controller, clientControllerQuotaManager, groupCoordinator, txnCoordinator) + + createKafkaApis(authorizer = Some(authorizer), + overrideProperties = topicConfigOverride).handleFindCoordinatorRequest(request) + + val response = readResponse(findCoordinatorRequest, capturedResponse) + .asInstanceOf[FindCoordinatorResponse] + assertEquals(Errors.COORDINATOR_NOT_AVAILABLE, response.error()) + + verify(authorizer, autoTopicCreationManager) + } + + @Test + def testMetadataAutoTopicCreationForOffsetTopic(): Unit = { + testMetadataAutoTopicCreation(Topic.GROUP_METADATA_TOPIC_NAME, enableAutoTopicCreation = true, + expectedError = Errors.UNKNOWN_TOPIC_OR_PARTITION) + } + + @Test + def testMetadataAutoTopicCreationForTxnTopic(): Unit = { + testMetadataAutoTopicCreation(Topic.TRANSACTION_STATE_TOPIC_NAME, enableAutoTopicCreation = true, + expectedError = Errors.UNKNOWN_TOPIC_OR_PARTITION) + } + + @Test + def testMetadataAutoTopicCreationForNonInternalTopic(): Unit = { + testMetadataAutoTopicCreation("topic", enableAutoTopicCreation = true, + expectedError = Errors.UNKNOWN_TOPIC_OR_PARTITION) + } + + @Test + def testMetadataAutoTopicCreationDisabledForOffsetTopic(): Unit = { + testMetadataAutoTopicCreation(Topic.GROUP_METADATA_TOPIC_NAME, enableAutoTopicCreation = false, + expectedError = Errors.UNKNOWN_TOPIC_OR_PARTITION) + } + + @Test + def testMetadataAutoTopicCreationDisabledForTxnTopic(): Unit = { + testMetadataAutoTopicCreation(Topic.TRANSACTION_STATE_TOPIC_NAME, enableAutoTopicCreation = false, + expectedError = Errors.UNKNOWN_TOPIC_OR_PARTITION) + } + + @Test + def testMetadataAutoTopicCreationDisabledForNonInternalTopic(): Unit = { + testMetadataAutoTopicCreation("topic", enableAutoTopicCreation = false, + expectedError = Errors.UNKNOWN_TOPIC_OR_PARTITION) + } + + @Test + def testMetadataAutoCreationDisabledForNonInternal(): Unit = { + testMetadataAutoTopicCreation("topic", enableAutoTopicCreation = true, + expectedError = Errors.UNKNOWN_TOPIC_OR_PARTITION) + } + + private def testMetadataAutoTopicCreation(topicName: String, + enableAutoTopicCreation: Boolean, + expectedError: Errors): Unit = { + val authorizer: Authorizer = EasyMock.niceMock(classOf[Authorizer]) + + val requestHeader = new RequestHeader(ApiKeys.METADATA, ApiKeys.METADATA.latestVersion, + clientId, 0) + + val numBrokersNeeded = 3 + addTopicToMetadataCache("some-topic", 1, 3) + + authorizeResource(authorizer, AclOperation.DESCRIBE, ResourceType.TOPIC, + topicName, AuthorizationResult.ALLOWED) + + if (enableAutoTopicCreation) + authorizeResource(authorizer, AclOperation.CREATE, ResourceType.CLUSTER, + Resource.CLUSTER_NAME, AuthorizationResult.ALLOWED, logIfDenied = false) + + val topicConfigOverride = mutable.Map.empty[String, String] + val isInternal = + topicName match { + case Topic.GROUP_METADATA_TOPIC_NAME => + topicConfigOverride.put(KafkaConfig.OffsetsTopicPartitionsProp, numBrokersNeeded.toString) + topicConfigOverride.put(KafkaConfig.OffsetsTopicReplicationFactorProp, numBrokersNeeded.toString) + EasyMock.expect(groupCoordinator.offsetsTopicConfigs).andReturn(new Properties) + true + + case Topic.TRANSACTION_STATE_TOPIC_NAME => + topicConfigOverride.put(KafkaConfig.TransactionsTopicPartitionsProp, numBrokersNeeded.toString) + topicConfigOverride.put(KafkaConfig.TransactionsTopicReplicationFactorProp, numBrokersNeeded.toString) + EasyMock.expect(txnCoordinator.transactionTopicConfigs).andReturn(new Properties) + true + case _ => + topicConfigOverride.put(KafkaConfig.NumPartitionsProp, numBrokersNeeded.toString) + topicConfigOverride.put(KafkaConfig.DefaultReplicationFactorProp, numBrokersNeeded.toString) + false + } + + val metadataRequest = new MetadataRequest.Builder( + List(topicName).asJava, enableAutoTopicCreation + ).build(requestHeader.apiVersion) + val request = buildRequest(metadataRequest) + + val capturedResponse = expectNoThrottling() + + verifyTopicCreation(topicName, enableAutoTopicCreation, isInternal, request) + + EasyMock.replay(replicaManager, clientRequestQuotaManager, requestChannel, authorizer, + autoTopicCreationManager, forwardingManager, clientControllerQuotaManager, groupCoordinator, txnCoordinator) + + createKafkaApis(authorizer = Some(authorizer), enableForwarding = enableAutoTopicCreation, + overrideProperties = topicConfigOverride).handleTopicMetadataRequest(request) + + val response = readResponse(metadataRequest, capturedResponse) + .asInstanceOf[MetadataResponse] + + val expectedMetadataResponse = util.Collections.singletonList(new TopicMetadata( + expectedError, + topicName, + isInternal, + util.Collections.emptyList() + )) + + assertEquals(expectedMetadataResponse, response.topicMetadata()) + + verify(authorizer, autoTopicCreationManager) + } + + private def verifyTopicCreation(topicName: String, + enableAutoTopicCreation: Boolean, + isInternal: Boolean, + request: RequestChannel.Request) = { + if (enableAutoTopicCreation) { + EasyMock.expect(clientControllerQuotaManager.newPermissiveQuotaFor(EasyMock.eq(request))) + .andReturn(UnboundedControllerMutationQuota) + + EasyMock.expect(autoTopicCreationManager.createTopics( + EasyMock.eq(Set(topicName)), + EasyMock.eq(UnboundedControllerMutationQuota))).andReturn( + Seq(new MetadataResponseTopic() + .setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) + .setIsInternal(isInternal) + .setName(topicName)) + ).once() + } + } + + private def setupBrokerMetadata(hasEnoughLiveBrokers: Boolean, numBrokersNeeded: Int): Unit = { + addTopicToMetadataCache("some-topic", 1, + if (hasEnoughLiveBrokers) + numBrokersNeeded + else + numBrokersNeeded - 1) + } + @Test def testOffsetCommitWithInvalidPartition(): Unit = { val topic = "topic" @@ -1827,7 +2081,7 @@ class KafkaApisTest { def getAllTopicMetadataShouldNotCreateTopicOrReturnUnknownTopicPartition(): Unit = { // Setup: authorizer authorizes 2 topics, but one got deleted in metadata cache metadataCache = - EasyMock.partialMockBuilder(classOf[MetadataCache]) + EasyMock.partialMockBuilder(classOf[ZkMetadataCache]) .withConstructor(classOf[Int]) .withArgs(Int.box(brokerId)) // Need to box it for Scala 2.12 and before .addMockedMethod("getAllTopics") @@ -1875,7 +2129,8 @@ class KafkaApisTest { val response = sendMetadataRequestWithInconsistentListeners(requestListener) assertFalse(createTopicIsCalled) - assertEquals(List("remaining-topic"), response.topicMetadata().asScala.map { metadata => metadata.topic() }) + val responseTopics = response.topicMetadata().asScala.map { metadata => metadata.topic() } + assertEquals(List("remaining-topic"), responseTopics) assertTrue(response.topicsByError(Errors.UNKNOWN_TOPIC_OR_PARTITION).isEmpty) } @@ -2006,7 +2261,7 @@ class KafkaApisTest { } def testJoinGroupWhenAnErrorOccurs(version: Short): Unit = { - EasyMock.reset(groupCoordinator, clientRequestQuotaManager, requestChannel) + EasyMock.reset(groupCoordinator, clientRequestQuotaManager, requestChannel, replicaManager) val capturedResponse = expectNoThrottling() @@ -2043,7 +2298,7 @@ class KafkaApisTest { val requestChannelRequest = buildRequest(joinGroupRequest) - EasyMock.replay(groupCoordinator, clientRequestQuotaManager, requestChannel) + EasyMock.replay(groupCoordinator, clientRequestQuotaManager, requestChannel, replicaManager) createKafkaApis().handleJoinGroupRequest(requestChannelRequest) @@ -2078,7 +2333,7 @@ class KafkaApisTest { } def testJoinGroupProtocolType(version: Short): Unit = { - EasyMock.reset(groupCoordinator, clientRequestQuotaManager, requestChannel) + EasyMock.reset(groupCoordinator, clientRequestQuotaManager, requestChannel, replicaManager) val capturedResponse = expectNoThrottling() @@ -2116,7 +2371,7 @@ class KafkaApisTest { val requestChannelRequest = buildRequest(joinGroupRequest) - EasyMock.replay(groupCoordinator, clientRequestQuotaManager, requestChannel) + EasyMock.replay(groupCoordinator, clientRequestQuotaManager, requestChannel, replicaManager) createKafkaApis().handleJoinGroupRequest(requestChannelRequest) @@ -2159,7 +2414,7 @@ class KafkaApisTest { } def testSyncGroupProtocolTypeAndName(version: Short): Unit = { - EasyMock.reset(groupCoordinator, clientRequestQuotaManager, requestChannel) + EasyMock.reset(groupCoordinator, clientRequestQuotaManager, requestChannel, replicaManager) val capturedResponse = expectNoThrottling() @@ -2192,7 +2447,7 @@ class KafkaApisTest { val requestChannelRequest = buildRequest(syncGroupRequest) - EasyMock.replay(groupCoordinator, clientRequestQuotaManager, requestChannel) + EasyMock.replay(groupCoordinator, clientRequestQuotaManager, requestChannel, replicaManager) createKafkaApis().handleSyncGroupRequest(requestChannelRequest) @@ -2228,7 +2483,7 @@ class KafkaApisTest { } def testSyncGroupProtocolTypeAndNameAreMandatorySinceV5(version: Short): Unit = { - EasyMock.reset(groupCoordinator, clientRequestQuotaManager, requestChannel) + EasyMock.reset(groupCoordinator, clientRequestQuotaManager, requestChannel, replicaManager) val capturedResponse = expectNoThrottling() @@ -2261,7 +2516,7 @@ class KafkaApisTest { val requestChannelRequest = buildRequest(syncGroupRequest) - EasyMock.replay(groupCoordinator, clientRequestQuotaManager, requestChannel) + EasyMock.replay(groupCoordinator, clientRequestQuotaManager, requestChannel, replicaManager) createKafkaApis().handleSyncGroupRequest(requestChannelRequest) @@ -2572,7 +2827,7 @@ class KafkaApisTest { } def testUpdateMetadataRequest(currentBrokerEpoch: Long, brokerEpochInRequest: Long, expectedError: Errors): Unit = { - val updateMetadataRequest = createBasicMetadataRequest("topicA", 1, brokerEpochInRequest) + val updateMetadataRequest = createBasicMetadataRequest("topicA", 1, brokerEpochInRequest, 1) val request = buildRequest(updateMetadataRequest) val capturedResponse: Capture[RequestChannel.Response] = EasyMock.newCapture() @@ -2813,7 +3068,7 @@ class KafkaApisTest { assertEquals(metadataCache.getControllerId.get, describeClusterResponse.data.controllerId) assertEquals(clusterId, describeClusterResponse.data.clusterId) assertEquals(8096, describeClusterResponse.data.clusterAuthorizedOperations) - assertEquals(metadataCache.getAliveBrokers.map(_.node(plaintextListener)).toSet, + assertEquals(metadataCache.getAliveBrokers.map(_.endpoints(plaintextListener.value())).toSet, describeClusterResponse.nodes.asScala.values.toSet) } @@ -2981,7 +3236,10 @@ class KafkaApisTest { capturedResponse } - private def createBasicMetadataRequest(topic: String, numPartitions: Int, brokerEpoch: Long): UpdateMetadataRequest = { + private def createBasicMetadataRequest(topic: String, + numPartitions: Int, + brokerEpoch: Long, + numBrokers: Int): UpdateMetadataRequest = { val replicas = List(0.asInstanceOf[Integer]).asJava def createPartitionState(partition: Int) = new UpdateMetadataPartitionState() @@ -2995,24 +3253,30 @@ class KafkaApisTest { .setIsr(replicas) val plaintextListener = ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT) - val broker = new UpdateMetadataBroker() - .setId(0) - .setRack("rack") - .setEndpoints(Seq(new UpdateMetadataEndpoint() - .setHost("broker0") - .setPort(9092) - .setSecurityProtocol(SecurityProtocol.PLAINTEXT.id) - .setListener(plaintextListener.value)).asJava) val partitionStates = (0 until numPartitions).map(createPartitionState) + val liveBrokers = (0 until numBrokers).map( + brokerId => createMetadataBroker(brokerId, plaintextListener)) new UpdateMetadataRequest.Builder(ApiKeys.UPDATE_METADATA.latestVersion, 0, - 0, brokerEpoch, partitionStates.asJava, Seq(broker).asJava, Collections.emptyMap()).build() + 0, brokerEpoch, partitionStates.asJava, liveBrokers.asJava, Collections.emptyMap()).build() } - private def addTopicToMetadataCache(topic: String, numPartitions: Int): Unit = { - val updateMetadataRequest = createBasicMetadataRequest(topic, numPartitions, 0) + private def addTopicToMetadataCache(topic: String, numPartitions: Int, numBrokers: Int = 1): Unit = { + val updateMetadataRequest = createBasicMetadataRequest(topic, numPartitions, 0, numBrokers) metadataCache.updateMetadata(correlationId = 0, updateMetadataRequest) } + private def createMetadataBroker(brokerId: Int, + listener: ListenerName): UpdateMetadataBroker = { + new UpdateMetadataBroker() + .setId(brokerId) + .setRack("rack") + .setEndpoints(Seq(new UpdateMetadataEndpoint() + .setHost("broker" + brokerId) + .setPort(9092) + .setSecurityProtocol(SecurityProtocol.PLAINTEXT.id) + .setListener(listener.value)).asJava) + } + @Test def testAlterReplicaLogDirs(): Unit = { val data = new AlterReplicaLogDirsRequestData() @@ -3173,4 +3437,124 @@ class KafkaApisTest { } + private def createMockRequest(): RequestChannel.Request = { + val request: RequestChannel.Request = EasyMock.createNiceMock(classOf[RequestChannel.Request]) + val requestHeader: RequestHeader = EasyMock.createNiceMock(classOf[RequestHeader]) + expect(request.header).andReturn(requestHeader).anyTimes() + expect(requestHeader.apiKey()).andReturn(ApiKeys.values().head).anyTimes() + EasyMock.replay(request, requestHeader) + request + } + + private def verifyShouldNeverHandle(handler: RequestChannel.Request => Unit): Unit = { + val request = createMockRequest() + val e = assertThrows(classOf[UnsupportedVersionException], () => handler(request)) + assertEquals(KafkaApis.shouldNeverReceive(request).getMessage, e.getMessage) + } + + private def verifyShouldAlwaysForward(handler: RequestChannel.Request => Unit): Unit = { + val request = createMockRequest() + val e = assertThrows(classOf[UnsupportedVersionException], () => handler(request)) + assertEquals(KafkaApis.shouldAlwaysForward(request).getMessage, e.getMessage) + } + + @Test + def testRaftShouldNeverHandleLeaderAndIsrRequest(): Unit = { + verifyShouldNeverHandle(createKafkaApis(raftSupport = true).handleLeaderAndIsrRequest) + } + + @Test + def testRaftShouldNeverHandleStopReplicaRequest(): Unit = { + verifyShouldNeverHandle(createKafkaApis(raftSupport = true).handleStopReplicaRequest) + } + + @Test + def testRaftShouldNeverHandleUpdateMetadataRequest(): Unit = { + verifyShouldNeverHandle(createKafkaApis(raftSupport = true).handleUpdateMetadataRequest) + } + + @Test + def testRaftShouldNeverHandleControlledShutdownRequest(): Unit = { + verifyShouldNeverHandle(createKafkaApis(raftSupport = true).handleControlledShutdownRequest) + } + + @Test + def testRaftShouldNeverHandleAlterIsrRequest(): Unit = { + verifyShouldNeverHandle(createKafkaApis(raftSupport = true).handleAlterIsrRequest) + } + + @Test + def testRaftShouldNeverHandleEnvelope(): Unit = { + verifyShouldNeverHandle(createKafkaApis(raftSupport = true).handleEnvelope) + } + + @Test + def testRaftShouldAlwaysForwardCreateTopicsRequest(): Unit = { + verifyShouldAlwaysForward(createKafkaApis(raftSupport = true).handleCreateTopicsRequest) + } + + @Test + def testRaftShouldAlwaysForwardCreatePartitionsRequest(): Unit = { + verifyShouldAlwaysForward(createKafkaApis(raftSupport = true).handleCreatePartitionsRequest) + } + + @Test + def testRaftShouldAlwaysForwardDeleteTopicsRequest(): Unit = { + verifyShouldAlwaysForward(createKafkaApis(raftSupport = true).handleDeleteTopicsRequest) + } + + @Test + def testRaftShouldAlwaysForwardCreateAcls(): Unit = { + verifyShouldAlwaysForward(createKafkaApis(raftSupport = true).handleCreateAcls) + } + + @Test + def testRaftShouldAlwaysForwardDeleteAcls(): Unit = { + verifyShouldAlwaysForward(createKafkaApis(raftSupport = true).handleDeleteAcls) + } + + @Test + def testRaftShouldAlwaysForwardAlterConfigsRequest(): Unit = { + verifyShouldAlwaysForward(createKafkaApis(raftSupport = true).handleAlterConfigsRequest) + } + + @Test + def testRaftShouldAlwaysForwardAlterPartitionReassignmentsRequest(): Unit = { + verifyShouldAlwaysForward(createKafkaApis(raftSupport = true).handleAlterPartitionReassignmentsRequest) + } + + @Test + def testRaftShouldAlwaysForwardIncrementalAlterConfigsRequest(): Unit = { + verifyShouldAlwaysForward(createKafkaApis(raftSupport = true).handleIncrementalAlterConfigsRequest) + } + + @Test + def testRaftShouldAlwaysForwardCreateTokenRequest(): Unit = { + verifyShouldAlwaysForward(createKafkaApis(raftSupport = true).handleCreateTokenRequest) + } + + @Test + def testRaftShouldAlwaysForwardRenewTokenRequest(): Unit = { + verifyShouldAlwaysForward(createKafkaApis(raftSupport = true).handleRenewTokenRequest) + } + + @Test + def testRaftShouldAlwaysForwardExpireTokenRequest(): Unit = { + verifyShouldAlwaysForward(createKafkaApis(raftSupport = true).handleExpireTokenRequest) + } + + @Test + def testRaftShouldAlwaysForwardAlterClientQuotasRequest(): Unit = { + verifyShouldAlwaysForward(createKafkaApis(raftSupport = true).handleAlterClientQuotasRequest) + } + + @Test + def testRaftShouldAlwaysForwardAlterUserScramCredentialsRequest(): Unit = { + verifyShouldAlwaysForward(createKafkaApis(raftSupport = true).handleAlterUserScramCredentialsRequest) + } + + @Test + def testRaftShouldAlwaysForwardUpdateFeatures(): Unit = { + verifyShouldAlwaysForward(createKafkaApis(raftSupport = true).handleUpdateFeatures) + } } diff --git a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala index 58325e185d..1ce8006c63 100755 --- a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala @@ -163,7 +163,7 @@ class LogOffsetTest extends BaseRequestTest { createTopic(topic, 3, 1) val logManager = server.getLogManager - val log = logManager.getOrCreateLog(topicPartition, () => logManager.initialDefaultConfig) + val log = logManager.getOrCreateLog(topicPartition) for (_ <- 0 until 20) log.appendAsLeader(TestUtils.singletonRecords(value = Integer.toString(42).getBytes()), leaderEpoch = 0) @@ -192,7 +192,7 @@ class LogOffsetTest extends BaseRequestTest { createTopic(topic, 3, 1) val logManager = server.getLogManager - val log = logManager.getOrCreateLog(topicPartition, () => logManager.initialDefaultConfig) + val log = logManager.getOrCreateLog(topicPartition) for (_ <- 0 until 20) log.appendAsLeader(TestUtils.singletonRecords(value = Integer.toString(42).getBytes()), leaderEpoch = 0) log.flush() diff --git a/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala b/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala index 60600f02bb..00a42ef965 100644 --- a/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala +++ b/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala @@ -16,40 +16,49 @@ */ package kafka.server +import org.apache.kafka.common.{Node, TopicPartition, Uuid} + import java.util -import java.util.Collections import util.Arrays.asList - -import org.apache.kafka.common.{Node, TopicPartition, Uuid} import org.apache.kafka.common.message.UpdateMetadataRequestData.{UpdateMetadataBroker, UpdateMetadataEndpoint, UpdateMetadataPartitionState} import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.record.RecordBatch import org.apache.kafka.common.requests.UpdateMetadataRequest import org.apache.kafka.common.security.auth.SecurityProtocol -import org.junit.jupiter.api.Test import org.junit.jupiter.api.Assertions._ +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.MethodSource +import java.util.Collections import scala.jdk.CollectionConverters._ +object MetadataCacheTest { + def cacheProvider(): util.stream.Stream[MetadataCache] = { + util.stream.Stream.of( + MetadataCache.zkMetadataCache(1), + MetadataCache.raftMetadataCache(1) + ) + } +} + class MetadataCacheTest { val brokerEpoch = 0L - @Test - def getTopicMetadataNonExistingTopics(): Unit = { + @ParameterizedTest + @MethodSource(Array("cacheProvider")) + def getTopicMetadataNonExistingTopics(cache: MetadataCache): Unit = { val topic = "topic" - val cache = new MetadataCache(1) val topicMetadata = cache.getTopicMetadata(Set(topic), ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT)) assertTrue(topicMetadata.isEmpty) } - @Test - def getTopicMetadata(): Unit = { + @ParameterizedTest + @MethodSource(Array("cacheProvider")) + def getTopicMetadata(cache: MetadataCache): Unit = { val topic0 = "topic-0" val topic1 = "topic-1" - val cache = new MetadataCache(1) - val zkVersion = 3 val controllerId = 2 val controllerEpoch = 1 @@ -135,7 +144,7 @@ class MetadataCacheTest { assertEquals(Errors.NONE.code, partitionMetadata.errorCode) assertEquals(partitionId, partitionMetadata.partitionIndex) val partitionState = topicPartitionStates.find(_.partitionIndex == partitionId).getOrElse( - throw new AssertionError(s"Unable to find partition state for partition $partitionId")) + fail(s"Unable to find partition state for partition $partitionId")) assertEquals(partitionState.leader, partitionMetadata.leaderId) assertEquals(partitionState.leaderEpoch, partitionMetadata.leaderEpoch) assertEquals(partitionState.isr, partitionMetadata.isrNodes) @@ -149,8 +158,9 @@ class MetadataCacheTest { } - @Test - def getTopicMetadataPartitionLeaderNotAvailable(): Unit = { + @ParameterizedTest + @MethodSource(Array("cacheProvider")) + def getTopicMetadataPartitionLeaderNotAvailable(cache: MetadataCache): Unit = { val securityProtocol = SecurityProtocol.PLAINTEXT val listenerName = ListenerName.forSecurityProtocol(securityProtocol) val brokers = Seq(new UpdateMetadataBroker() @@ -162,14 +172,15 @@ class MetadataCacheTest { .setListener(listenerName.value)).asJava)) val metadataCacheBrokerId = 0 // leader is not available. expect LEADER_NOT_AVAILABLE for any metadata version. - verifyTopicMetadataPartitionLeaderOrEndpointNotAvailable(metadataCacheBrokerId, brokers, listenerName, + verifyTopicMetadataPartitionLeaderOrEndpointNotAvailable(cache, metadataCacheBrokerId, brokers, listenerName, leader = 1, Errors.LEADER_NOT_AVAILABLE, errorUnavailableListeners = false) - verifyTopicMetadataPartitionLeaderOrEndpointNotAvailable(metadataCacheBrokerId, brokers, listenerName, + verifyTopicMetadataPartitionLeaderOrEndpointNotAvailable(cache, metadataCacheBrokerId, brokers, listenerName, leader = 1, Errors.LEADER_NOT_AVAILABLE, errorUnavailableListeners = true) } - @Test - def getTopicMetadataPartitionListenerNotAvailableOnLeader(): Unit = { + @ParameterizedTest + @MethodSource(Array("cacheProvider")) + def getTopicMetadataPartitionListenerNotAvailableOnLeader(cache: MetadataCache): Unit = { // when listener name is not present in the metadata cache for a broker, getTopicMetadata should // return LEADER_NOT_AVAILABLE or LISTENER_NOT_FOUND errors for old and new versions respectively. val plaintextListenerName = ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT) @@ -199,14 +210,15 @@ class MetadataCacheTest { .setEndpoints(broker1Endpoints.asJava)) val metadataCacheBrokerId = 0 // leader available in cache but listener name not present. expect LISTENER_NOT_FOUND error for new metadata version - verifyTopicMetadataPartitionLeaderOrEndpointNotAvailable(metadataCacheBrokerId, brokers, sslListenerName, + verifyTopicMetadataPartitionLeaderOrEndpointNotAvailable(cache, metadataCacheBrokerId, brokers, sslListenerName, leader = 1, Errors.LISTENER_NOT_FOUND, errorUnavailableListeners = true) // leader available in cache but listener name not present. expect LEADER_NOT_AVAILABLE error for old metadata version - verifyTopicMetadataPartitionLeaderOrEndpointNotAvailable(metadataCacheBrokerId, brokers, sslListenerName, + verifyTopicMetadataPartitionLeaderOrEndpointNotAvailable(cache, metadataCacheBrokerId, brokers, sslListenerName, leader = 1, Errors.LEADER_NOT_AVAILABLE, errorUnavailableListeners = false) } - private def verifyTopicMetadataPartitionLeaderOrEndpointNotAvailable(metadataCacheBrokerId: Int, + private def verifyTopicMetadataPartitionLeaderOrEndpointNotAvailable(cache: MetadataCache, + metadataCacheBrokerId: Int, brokers: Seq[UpdateMetadataBroker], listenerName: ListenerName, leader: Int, @@ -214,8 +226,6 @@ class MetadataCacheTest { errorUnavailableListeners: Boolean): Unit = { val topic = "topic" - val cache = new MetadataCache(metadataCacheBrokerId) - val zkVersion = 3 val controllerId = 2 val controllerEpoch = 1 @@ -233,7 +243,7 @@ class MetadataCacheTest { val version = ApiKeys.UPDATE_METADATA.latestVersion val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch, brokerEpoch, - partitionStates.asJava, brokers.asJava, Collections.emptyMap()).build() + partitionStates.asJava, brokers.asJava, util.Collections.emptyMap()).build() cache.updateMetadata(15, updateMetadataRequest) val topicMetadatas = cache.getTopicMetadata(Set(topic), listenerName, errorUnavailableListeners = errorUnavailableListeners) @@ -252,12 +262,11 @@ class MetadataCacheTest { assertEquals(List(0), partitionMetadata.replicaNodes.asScala) } - @Test - def getTopicMetadataReplicaNotAvailable(): Unit = { + @ParameterizedTest + @MethodSource(Array("cacheProvider")) + def getTopicMetadataReplicaNotAvailable(cache: MetadataCache): Unit = { val topic = "topic" - val cache = new MetadataCache(1) - val zkVersion = 3 val controllerId = 2 val controllerEpoch = 1 @@ -290,7 +299,7 @@ class MetadataCacheTest { val version = ApiKeys.UPDATE_METADATA.latestVersion val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch, brokerEpoch, - partitionStates.asJava, brokers.asJava, Collections.emptyMap()).build() + partitionStates.asJava, brokers.asJava, util.Collections.emptyMap()).build() cache.updateMetadata(15, updateMetadataRequest) // Validate errorUnavailableEndpoints = false @@ -326,12 +335,11 @@ class MetadataCacheTest { assertEquals(Set(0), partitionMetadataWithError.isrNodes.asScala.toSet) } - @Test - def getTopicMetadataIsrNotAvailable(): Unit = { + @ParameterizedTest + @MethodSource(Array("cacheProvider")) + def getTopicMetadataIsrNotAvailable(cache: MetadataCache): Unit = { val topic = "topic" - val cache = new MetadataCache(1) - val zkVersion = 3 val controllerId = 2 val controllerEpoch = 1 @@ -364,7 +372,7 @@ class MetadataCacheTest { val version = ApiKeys.UPDATE_METADATA.latestVersion val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch, brokerEpoch, - partitionStates.asJava, brokers.asJava, Collections.emptyMap()).build() + partitionStates.asJava, brokers.asJava, util.Collections.emptyMap()).build() cache.updateMetadata(15, updateMetadataRequest) // Validate errorUnavailableEndpoints = false @@ -400,10 +408,10 @@ class MetadataCacheTest { assertEquals(Set(0), partitionMetadataWithError.isrNodes.asScala.toSet) } - @Test - def getTopicMetadataWithNonSupportedSecurityProtocol(): Unit = { + @ParameterizedTest + @MethodSource(Array("cacheProvider")) + def getTopicMetadataWithNonSupportedSecurityProtocol(cache: MetadataCache): Unit = { val topic = "topic" - val cache = new MetadataCache(1) val securityProtocol = SecurityProtocol.PLAINTEXT val brokers = Seq(new UpdateMetadataBroker() .setId(0) @@ -416,7 +424,7 @@ class MetadataCacheTest { val controllerEpoch = 1 val leader = 0 val leaderEpoch = 0 - val replicas = asList[Integer](0, 1) + val replicas = asList[Integer](0) val isr = asList[Integer](0, 1) val partitionStates = Seq(new UpdateMetadataPartitionState() .setTopicName(topic) @@ -429,7 +437,7 @@ class MetadataCacheTest { .setReplicas(replicas)) val version = ApiKeys.UPDATE_METADATA.latestVersion val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, 2, controllerEpoch, brokerEpoch, partitionStates.asJava, - brokers.asJava, Collections.emptyMap()).build() + brokers.asJava, util.Collections.emptyMap()).build() cache.updateMetadata(15, updateMetadataRequest) val topicMetadata = cache.getTopicMetadata(Set(topic), ListenerName.forSecurityProtocol(SecurityProtocol.SSL)) @@ -438,10 +446,10 @@ class MetadataCacheTest { assertEquals(RecordBatch.NO_PARTITION_LEADER_EPOCH, topicMetadata.head.partitions.get(0).leaderId) } - @Test - def getAliveBrokersShouldNotBeMutatedByUpdateCache(): Unit = { + @ParameterizedTest + @MethodSource(Array("cacheProvider")) + def getAliveBrokersShouldNotBeMutatedByUpdateCache(cache: MetadataCache): Unit = { val topic = "topic" - val cache = new MetadataCache(1) def updateCache(brokerIds: Seq[Int]): Unit = { val brokers = brokerIds.map { brokerId => @@ -458,7 +466,7 @@ class MetadataCacheTest { val controllerEpoch = 1 val leader = 0 val leaderEpoch = 0 - val replicas = asList[Integer](0, 1) + val replicas = asList[Integer](0) val isr = asList[Integer](0, 1) val partitionStates = Seq(new UpdateMetadataPartitionState() .setTopicName(topic) @@ -471,7 +479,7 @@ class MetadataCacheTest { .setReplicas(replicas)) val version = ApiKeys.UPDATE_METADATA.latestVersion val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, 2, controllerEpoch, brokerEpoch, partitionStates.asJava, - brokers.asJava, Collections.emptyMap()).build() + brokers.asJava, util.Collections.emptyMap()).build() cache.updateMetadata(15, updateMetadataRequest) } @@ -483,9 +491,9 @@ class MetadataCacheTest { assertEquals(initialBrokerIds.toSet, aliveBrokersFromCache.map(_.id).toSet) } - @Test - def testGetClusterMetadataWithOfflineReplicas(): Unit = { - val cache = new MetadataCache(1) + @ParameterizedTest + @MethodSource(Array("cacheProvider")) + def testGetClusterMetadataWithOfflineReplicas(cache: MetadataCache): Unit = { val topic = "topic" val topicPartition = new TopicPartition(topic, 0) val securityProtocol = SecurityProtocol.PLAINTEXT @@ -539,5 +547,4 @@ class MetadataCacheTest { assertEquals(Seq(expectedNode0, expectedNode1), partitionInfo.inSyncReplicas.toSeq) assertEquals(Seq(expectedNode1), partitionInfo.offlineReplicas.toSeq) } - } diff --git a/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala b/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala index 7b7078288e..01daf02245 100644 --- a/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala @@ -17,14 +17,12 @@ package kafka.server -import java.util.{Optional, Properties} +import java.util.Optional -import kafka.network.SocketServer import kafka.utils.TestUtils import org.apache.kafka.common.Uuid import org.apache.kafka.common.errors.UnsupportedVersionException import org.apache.kafka.common.internals.Topic -import org.apache.kafka.common.message.MetadataRequestData import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{MetadataRequest, MetadataResponse} import org.apache.kafka.metadata.BrokerState @@ -32,16 +30,10 @@ import org.apache.kafka.test.TestUtils.isValidClusterId import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{BeforeEach, Test} -import scala.jdk.CollectionConverters._ import scala.collection.Seq +import scala.jdk.CollectionConverters._ -class MetadataRequestTest extends BaseRequestTest { - - override def brokerPropertyOverrides(properties: Properties): Unit = { - properties.setProperty(KafkaConfig.OffsetsTopicPartitionsProp, "1") - properties.setProperty(KafkaConfig.DefaultReplicationFactorProp, "2") - properties.setProperty(KafkaConfig.RackProp, s"rack/${properties.getProperty(KafkaConfig.BrokerIdProp)}") - } +class MetadataRequestTest extends AbstractMetadataRequestTest { @BeforeEach override def setUp(): Unit = { @@ -128,19 +120,12 @@ class MetadataRequestTest extends BaseRequestTest { @Test def testAutoTopicCreation(): Unit = { - def checkAutoCreatedTopic(autoCreatedTopic: String, response: MetadataResponse): Unit = { - assertEquals(Errors.LEADER_NOT_AVAILABLE, response.errors.get(autoCreatedTopic)) - assertEquals(Some(servers.head.config.numPartitions), zkClient.getTopicPartitionCount(autoCreatedTopic)) - for (i <- 0 until servers.head.config.numPartitions) - TestUtils.waitForPartitionMetadata(servers, autoCreatedTopic, i) - } - val topic1 = "t1" val topic2 = "t2" val topic3 = "t3" val topic4 = "t4" val topic5 = "t5" - createTopic(topic1, numPartitions = 1, replicationFactor = 1) + createTopic(topic1) val response1 = sendMetadataRequest(new MetadataRequest.Builder(Seq(topic1, topic2).asJava, true).build()) assertNull(response1.errors.get(topic1)) @@ -178,23 +163,24 @@ class MetadataRequestTest extends BaseRequestTest { @Test def testAutoCreateOfCollidingTopics(): Unit = { - val topic1 = "testAutoCreate_Topic" - val topic2 = "testAutoCreate.Topic" + val topic1 = "testAutoCreate.Topic" + val topic2 = "testAutoCreate_Topic" val response1 = sendMetadataRequest(new MetadataRequest.Builder(Seq(topic1, topic2).asJava, true).build) assertEquals(2, response1.topicMetadata.size) - var topicMetadata1 = response1.topicMetadata.asScala.head - val topicMetadata2 = response1.topicMetadata.asScala.toSeq(1) - assertEquals(Errors.LEADER_NOT_AVAILABLE, topicMetadata1.error) - assertEquals(topic1, topicMetadata1.topic) - assertEquals(Errors.INVALID_TOPIC_EXCEPTION, topicMetadata2.error) - assertEquals(topic2, topicMetadata2.topic) - TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic1, 0) - TestUtils.waitForPartitionMetadata(servers, topic1, 0) + val responseMap = response1.topicMetadata.asScala.map(metadata => (metadata.topic(), metadata.error)).toMap + + assertEquals(Set(topic1, topic2), responseMap.keySet) + // The topic creation will be delayed, and the name collision error will be swallowed. + assertEquals(Set(Errors.LEADER_NOT_AVAILABLE, Errors.INVALID_TOPIC_EXCEPTION), responseMap.values.toSet) + + val topicCreated = responseMap.head._1 + TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topicCreated, 0) + TestUtils.waitForPartitionMetadata(servers, topicCreated, 0) // retry the metadata for the first auto created topic - val response2 = sendMetadataRequest(new MetadataRequest.Builder(Seq(topic1).asJava, true).build) - topicMetadata1 = response2.topicMetadata.asScala.head + val response2 = sendMetadataRequest(new MetadataRequest.Builder(Seq(topicCreated).asJava, true).build) + val topicMetadata1 = response2.topicMetadata.asScala.head assertEquals(Errors.NONE, topicMetadata1.error) assertEquals(Seq(Errors.NONE), topicMetadata1.partitionMetadata.asScala.map(_.error)) assertEquals(1, topicMetadata1.partitionMetadata.size) @@ -275,15 +261,6 @@ class MetadataRequestTest extends BaseRequestTest { } } - def requestData(topics: List[String], allowAutoTopicCreation: Boolean): MetadataRequestData = { - val data = new MetadataRequestData - if (topics == null) data.setTopics(null) - else topics.foreach(topic => data.topics.add(new MetadataRequestData.MetadataRequestTopic().setName(topic))) - - data.setAllowAutoTopicCreation(allowAutoTopicCreation) - data - } - @Test def testReplicaDownResponse(): Unit = { val replicaDownTopic = "replicaDown" @@ -397,9 +374,4 @@ class MetadataRequestTest extends BaseRequestTest { serverToShutdown.startup() checkMetadata(servers, servers.size) } - - private def sendMetadataRequest(request: MetadataRequest, destination: Option[SocketServer] = None): MetadataResponse = { - connectAndReceive[MetadataResponse](request, destination = destination.getOrElse(anySocketServer)) - } - } diff --git a/core/src/test/scala/unit/kafka/server/MetadataRequestWithForwardingTest.scala b/core/src/test/scala/unit/kafka/server/MetadataRequestWithForwardingTest.scala new file mode 100644 index 0000000000..8409c968f0 --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/MetadataRequestWithForwardingTest.scala @@ -0,0 +1,111 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server + +import kafka.utils.TestUtils +import org.apache.kafka.common.errors.UnsupportedVersionException +import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.requests.MetadataRequest +import org.junit.jupiter.api.Assertions.{assertEquals, assertNull, assertThrows, assertTrue} +import org.junit.jupiter.api.{BeforeEach, Test} + +import scala.collection.Seq +import scala.jdk.CollectionConverters._ + +class MetadataRequestWithForwardingTest extends AbstractMetadataRequestTest { + + @BeforeEach + override def setUp(): Unit = { + doSetup(createOffsetsTopic = false) + } + + override def enableForwarding: Boolean = true + + @Test + def testAutoTopicCreation(): Unit = { + val topic1 = "t1" + val topic2 = "t2" + val topic3 = "t3" + val topic4 = "t4" + val topic5 = "t5" + createTopic(topic1) + + val response1 = sendMetadataRequest(new MetadataRequest.Builder(Seq(topic1, topic2).asJava, true).build()) + assertNull(response1.errors.get(topic1)) + checkAutoCreatedTopic(topic2, response1) + + // The default behavior in old versions of the metadata API is to allow topic creation, so + // protocol downgrades should happen gracefully when auto-creation is explicitly requested. + val response2 = sendMetadataRequest(new MetadataRequest.Builder(Seq(topic3).asJava, true).build(1)) + checkAutoCreatedTopic(topic3, response2) + + // V3 doesn't support a configurable allowAutoTopicCreation, so disabling auto-creation is not supported + assertThrows(classOf[UnsupportedVersionException], () => sendMetadataRequest(new MetadataRequest(requestData(List(topic4), false), 3.toShort))) + + // V4 and higher support a configurable allowAutoTopicCreation + val response3 = sendMetadataRequest(new MetadataRequest.Builder(Seq(topic4, topic5).asJava, false, 4.toShort).build) + assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION, response3.errors.get(topic4)) + assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION, response3.errors.get(topic5)) + assertEquals(None, zkClient.getTopicPartitionCount(topic5)) + } + + @Test + def testAutoCreateTopicWithInvalidReplicationFactor(): Unit = { + // Shutdown all but one broker so that the number of brokers is less than the default replication factor + servers.tail.foreach(_.shutdown()) + servers.tail.foreach(_.awaitShutdown()) + + val topic1 = "testAutoCreateTopic" + val response1 = sendMetadataRequest(new MetadataRequest.Builder(Seq(topic1).asJava, true).build) + assertEquals(1, response1.topicMetadata.size) + val topicMetadata = response1.topicMetadata.asScala.head + assertEquals(Errors.INVALID_REPLICATION_FACTOR, topicMetadata.error) + assertEquals(topic1, topicMetadata.topic) + assertEquals(0, topicMetadata.partitionMetadata.size) + } + + @Test + def testAutoCreateOfCollidingTopics(): Unit = { + val topic1 = "testAutoCreate.Topic" + val topic2 = "testAutoCreate_Topic" + val response1 = sendMetadataRequest(new MetadataRequest.Builder(Seq(topic1, topic2).asJava, true).build) + assertEquals(2, response1.topicMetadata.size) + + val responseMap = response1.topicMetadata.asScala.map(metadata => (metadata.topic(), metadata.error)).toMap + + assertEquals(Set(topic1, topic2), responseMap.keySet) + // The topic creation will be delayed, and the name collision error will be swallowed. + assertEquals(Set(Errors.LEADER_NOT_AVAILABLE, Errors.INVALID_TOPIC_EXCEPTION), responseMap.values.toSet) + + val topicCreated = responseMap.head._1 + TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topicCreated, 0) + TestUtils.waitForPartitionMetadata(servers, topicCreated, 0) + + // retry the metadata for the first auto created topic + val response2 = sendMetadataRequest(new MetadataRequest.Builder(Seq(topicCreated).asJava, true).build) + val topicMetadata1 = response2.topicMetadata.asScala.head + assertEquals(Errors.NONE, topicMetadata1.error) + assertEquals(Seq(Errors.NONE), topicMetadata1.partitionMetadata.asScala.map(_.error)) + assertEquals(1, topicMetadata1.partitionMetadata.size) + val partitionMetadata = topicMetadata1.partitionMetadata.asScala.head + assertEquals(0, partitionMetadata.partition) + assertEquals(2, partitionMetadata.replicaIds.size) + assertTrue(partitionMetadata.leaderId.isPresent) + assertTrue(partitionMetadata.leaderId.get >= 0) + } +} diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala index b14dac4531..9649c48d82 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala @@ -23,7 +23,6 @@ import java.util.concurrent.atomic.AtomicBoolean import kafka.cluster.Partition import kafka.log.{Log, LogManager, LogOffsetSnapshot} import kafka.utils._ -import kafka.zk.KafkaZkClient import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRecord} @@ -31,6 +30,7 @@ import org.apache.kafka.common.requests.FetchRequest.PartitionData import org.easymock.EasyMock import EasyMock._ import kafka.server.QuotaFactory.QuotaManagers +import kafka.server.metadata.CachedConfigRepository import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, Test} @@ -199,7 +199,7 @@ class ReplicaManagerQuotasTest { def setUpMocks(fetchInfo: Seq[(TopicPartition, PartitionData)], record: SimpleRecord = this.record, bothReplicasInSync: Boolean = false): Unit = { - val zkClient: KafkaZkClient = EasyMock.createMock(classOf[KafkaZkClient]) + val configRepository = new CachedConfigRepository() val scheduler: KafkaScheduler = createNiceMock(classOf[KafkaScheduler]) //Create log which handles both a regular read and a 0 bytes read @@ -243,9 +243,9 @@ class ReplicaManagerQuotasTest { val leaderBrokerId = configs.head.brokerId quotaManager = QuotaFactory.instantiate(configs.head, metrics, time, "") - replicaManager = new ReplicaManager(configs.head, metrics, time, zkClient, scheduler, logManager, + replicaManager = new ReplicaManager(configs.head, metrics, time, None, scheduler, logManager, new AtomicBoolean(false), quotaManager, - new BrokerTopicStats, new MetadataCache(leaderBrokerId), new LogDirFailureChannel(configs.head.logDirs.size), alterIsrManager) + new BrokerTopicStats, MetadataCache.zkMetadataCache(leaderBrokerId), new LogDirFailureChannel(configs.head.logDirs.size), alterIsrManager, configRepository) //create the two replicas for ((p, _) <- fetchInfo) { diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 724b77b9e7..a21a016bd8 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -31,10 +31,10 @@ import kafka.server.QuotaFactory.{QuotaManagers, UnboundedQuota} import kafka.server.checkpoints.LazyOffsetCheckpoints import kafka.server.checkpoints.OffsetCheckpointFile import kafka.server.epoch.util.ReplicaFetcherMockBlockingSend +import kafka.server.metadata.CachedConfigRepository import kafka.utils.TestUtils.createBroker import kafka.utils.timer.MockTimer import kafka.utils.{MockScheduler, MockTime, TestUtils} -import kafka.zk.KafkaZkClient import org.apache.kafka.common.message.LeaderAndIsrRequestData import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset @@ -67,7 +67,7 @@ class ReplicaManagerTest { val time = new MockTime val scheduler = new MockScheduler(time) val metrics = new Metrics - var kafkaZkClient: KafkaZkClient = _ + val configRepository = new CachedConfigRepository() var alterIsrManager: AlterIsrManager = _ var config: KafkaConfig = _ var quotaManager: QuotaManagers = _ @@ -80,10 +80,6 @@ class ReplicaManagerTest { @BeforeEach def setUp(): Unit = { - kafkaZkClient = EasyMock.createMock(classOf[KafkaZkClient]) - EasyMock.expect(kafkaZkClient.getEntityConfigs(EasyMock.anyString(), EasyMock.anyString())).andReturn(new Properties()).anyTimes() - EasyMock.replay(kafkaZkClient) - val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect) config = KafkaConfig.fromProps(props) alterIsrManager = EasyMock.createMock(classOf[AlterIsrManager]) @@ -100,9 +96,9 @@ class ReplicaManagerTest { @Test def testHighWaterMarkDirectoryMapping(): Unit = { val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_))) - val rm = new ReplicaManager(config, metrics, time, kafkaZkClient, new MockScheduler(time), mockLogMgr, + val rm = new ReplicaManager(config, metrics, time, None, new MockScheduler(time), mockLogMgr, new AtomicBoolean(false), quotaManager, new BrokerTopicStats, - new MetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size), alterIsrManager) + MetadataCache.zkMetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size), alterIsrManager, configRepository) try { val partition = rm.createPartition(new TopicPartition(topic, 1)) partition.createLogIfNotExists(isNew = false, isFutureReplica = false, @@ -120,9 +116,9 @@ class ReplicaManagerTest { props.put("log.dir", TestUtils.tempRelativeDir("data").getAbsolutePath) val config = KafkaConfig.fromProps(props) val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_))) - val rm = new ReplicaManager(config, metrics, time, kafkaZkClient, new MockScheduler(time), mockLogMgr, + val rm = new ReplicaManager(config, metrics, time, None, new MockScheduler(time), mockLogMgr, new AtomicBoolean(false), quotaManager, new BrokerTopicStats, - new MetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size), alterIsrManager) + MetadataCache.zkMetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size), alterIsrManager, configRepository) try { val partition = rm.createPartition(new TopicPartition(topic, 1)) partition.createLogIfNotExists(isNew = false, isFutureReplica = false, @@ -137,9 +133,9 @@ class ReplicaManagerTest { @Test def testIllegalRequiredAcks(): Unit = { val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_))) - val rm = new ReplicaManager(config, metrics, time, kafkaZkClient, new MockScheduler(time), mockLogMgr, + val rm = new ReplicaManager(config, metrics, time, None, new MockScheduler(time), mockLogMgr, new AtomicBoolean(false), quotaManager, new BrokerTopicStats, - new MetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size), alterIsrManager, Option(this.getClass.getName)) + MetadataCache.zkMetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size), alterIsrManager, configRepository, Option(this.getClass.getName)) try { def callback(responseStatus: Map[TopicPartition, PartitionResponse]) = { assert(responseStatus.values.head.error == Errors.INVALID_REQUIRED_ACKS) @@ -170,9 +166,9 @@ class ReplicaManagerTest { val metadataCache: MetadataCache = EasyMock.createMock(classOf[MetadataCache]) EasyMock.expect(metadataCache.getAliveBrokers).andReturn(aliveBrokers).anyTimes() EasyMock.replay(metadataCache) - val rm = new ReplicaManager(config, metrics, time, kafkaZkClient, new MockScheduler(time), mockLogMgr, + val rm = new ReplicaManager(config, metrics, time, None, new MockScheduler(time), mockLogMgr, new AtomicBoolean(false), quotaManager, new BrokerTopicStats, - metadataCache, new LogDirFailureChannel(config.logDirs.size), alterIsrManager) + metadataCache, new LogDirFailureChannel(config.logDirs.size), alterIsrManager, configRepository) try { val brokerList = Seq[Integer](0, 1).asJava @@ -1509,8 +1505,7 @@ class ReplicaManagerTest { val mockLogMgr: LogManager = EasyMock.createMock(classOf[LogManager]) EasyMock.expect(mockLogMgr.liveLogDirs).andReturn(config.logDirs.map(new File(_).getAbsoluteFile)).anyTimes EasyMock.expect(mockLogMgr.getOrCreateLog(EasyMock.eq(topicPartitionObj), - EasyMock.anyObject[() => LogConfig](), isNew = EasyMock.eq(false), - isFuture = EasyMock.eq(false))).andReturn(mockLog).anyTimes + isNew = EasyMock.eq(false), isFuture = EasyMock.eq(false))).andReturn(mockLog).anyTimes if (expectTruncation) { EasyMock.expect(mockLogMgr.truncateTo(Map(topicPartitionObj -> offsetFromLeader), isFuture = false)).once @@ -1519,7 +1514,7 @@ class ReplicaManagerTest { EasyMock.expect(mockLogMgr.getLog(topicPartitionObj, isFuture = true)).andReturn(None) EasyMock.expect(mockLogMgr.finishedInitializingLog( - EasyMock.eq(topicPartitionObj), EasyMock.anyObject(), EasyMock.anyObject())).anyTimes + EasyMock.eq(topicPartitionObj), EasyMock.anyObject())).anyTimes EasyMock.replay(mockLogMgr) @@ -1560,10 +1555,11 @@ class ReplicaManagerTest { .setLeaderEpoch(leaderEpochFromLeader) .setEndOffset(offsetFromLeader)).asJava, BrokerEndPoint(1, "host1" ,1), time) - val replicaManager = new ReplicaManager(config, metrics, time, kafkaZkClient, mockScheduler, mockLogMgr, + val replicaManager = new ReplicaManager(config, metrics, time, None, mockScheduler, mockLogMgr, new AtomicBoolean(false), quotaManager, mockBrokerTopicStats, metadataCache, mockLogDirFailureChannel, mockProducePurgatory, mockFetchPurgatory, - mockDeleteRecordsPurgatory, mockElectLeaderPurgatory, Option(this.getClass.getName), alterIsrManager) { + mockDeleteRecordsPurgatory, mockElectLeaderPurgatory, Option(this.getClass.getName), + configRepository, alterIsrManager) { override protected def createReplicaFetcherManager(metrics: Metrics, time: Time, @@ -1742,10 +1738,11 @@ class ReplicaManagerTest { val mockDelayedElectLeaderPurgatory = new DelayedOperationPurgatory[DelayedElectLeader]( purgatoryName = "DelayedElectLeader", timer, reaperEnabled = false) - new ReplicaManager(config, metrics, time, kafkaZkClient, scheduler, mockLogMgr, + new ReplicaManager(config, metrics, time, None, scheduler, mockLogMgr, new AtomicBoolean(false), quotaManager, new BrokerTopicStats, metadataCache, new LogDirFailureChannel(config.logDirs.size), mockProducePurgatory, mockFetchPurgatory, - mockDeleteRecordsPurgatory, mockDelayedElectLeaderPurgatory, Option(this.getClass.getName), alterIsrManager) + mockDeleteRecordsPurgatory, mockDelayedElectLeaderPurgatory, Option(this.getClass.getName), + configRepository, alterIsrManager) } @Test @@ -1956,12 +1953,12 @@ class ReplicaManagerTest { EasyMock.replay(metadataCache1) // each replica manager is for a broker - val rm0 = new ReplicaManager(config0, metrics, time, kafkaZkClient, new MockScheduler(time), mockLogMgr0, + val rm0 = new ReplicaManager(config0, metrics, time, None, new MockScheduler(time), mockLogMgr0, new AtomicBoolean(false), quotaManager, - brokerTopicStats1, metadataCache0, new LogDirFailureChannel(config0.logDirs.size), alterIsrManager) - val rm1 = new ReplicaManager(config1, metrics, time, kafkaZkClient, new MockScheduler(time), mockLogMgr1, + brokerTopicStats1, metadataCache0, new LogDirFailureChannel(config0.logDirs.size), alterIsrManager, configRepository) + val rm1 = new ReplicaManager(config1, metrics, time, None, new MockScheduler(time), mockLogMgr1, new AtomicBoolean(false), quotaManager, - brokerTopicStats2, metadataCache1, new LogDirFailureChannel(config1.logDirs.size), alterIsrManager) + brokerTopicStats2, metadataCache1, new LogDirFailureChannel(config1.logDirs.size), alterIsrManager, configRepository) (rm0, rm1) } @@ -2042,7 +2039,7 @@ class ReplicaManagerTest { val replicaManager = setupReplicaManagerWithMockedPurgatories(mockTimer, aliveBrokerIds = Seq(0, 1)) val tp0 = new TopicPartition(topic, 0) - val log = replicaManager.logManager.getOrCreateLog(tp0, () => LogConfig(), true) + val log = replicaManager.logManager.getOrCreateLog(tp0, true) if (throwIOException) { // Delete the underlying directory to trigger an KafkaStorageException @@ -2202,9 +2199,9 @@ class ReplicaManagerTest { val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect) val config = KafkaConfig.fromProps(props) val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_))) - new ReplicaManager(config, metrics, time, kafkaZkClient, new MockScheduler(time), mockLogMgr, + new ReplicaManager(config, metrics, time, None, new MockScheduler(time), mockLogMgr, new AtomicBoolean(false), quotaManager, new BrokerTopicStats, - new MetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size), alterIsrManager) { + MetadataCache.zkMetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size), alterIsrManager, configRepository) { override def getPartitionOrException(topicPartition: TopicPartition): Partition = { throw Errors.NOT_LEADER_OR_FOLLOWER.exception() } diff --git a/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala b/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala index 0444264edd..bbc71cad26 100644 --- a/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala @@ -16,6 +16,8 @@ */ package kafka.server +import integration.kafka.server.IntegrationTestUtils + import java.net.Socket import java.util.Collections import kafka.api.{KafkaSasl, SaslSetup} @@ -23,49 +25,53 @@ import kafka.utils.JaasTestUtils import org.apache.kafka.common.message.SaslHandshakeRequestData import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.{ApiVersionsRequest, ApiVersionsResponse, SaslHandshakeRequest, SaslHandshakeResponse} +import kafka.test.annotation.{ClusterTest, Type} +import kafka.test.junit.ClusterTestExtensions +import kafka.test.{ClusterConfig, ClusterInstance} import org.apache.kafka.common.security.auth.SecurityProtocol import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} +import org.junit.jupiter.api.{AfterEach, BeforeEach} +import org.junit.jupiter.api.extension.ExtendWith -class SaslApiVersionsRequestTest extends AbstractApiVersionsRequestTest with SaslSetup { - override protected def securityProtocol = SecurityProtocol.SASL_PLAINTEXT - private val kafkaClientSaslMechanism = "PLAIN" - private val kafkaServerSaslMechanisms = List("PLAIN") - protected override val serverSaslProperties = Some(kafkaServerSaslProperties(kafkaServerSaslMechanisms, kafkaClientSaslMechanism)) - protected override val clientSaslProperties = Some(kafkaClientSaslProperties(kafkaClientSaslMechanism)) - override def brokerCount = 1 +import scala.jdk.CollectionConverters._ - @BeforeEach - override def setUp(): Unit = { - startSasl(jaasSections(kafkaServerSaslMechanisms, Some(kafkaClientSaslMechanism), KafkaSasl, JaasTestUtils.KafkaServerContextName)) - super.setUp() - } - @AfterEach - override def tearDown(): Unit = { - super.tearDown() - closeSasl() +@ExtendWith(value = Array(classOf[ClusterTestExtensions])) +class SaslApiVersionsRequestTest(cluster: ClusterInstance) extends AbstractApiVersionsRequestTest(cluster) { + + val kafkaClientSaslMechanism = "PLAIN" + val kafkaServerSaslMechanisms = List("PLAIN") + + private var sasl: SaslSetup = _ + + @BeforeEach + def setupSasl(config: ClusterConfig): Unit = { + sasl = new SaslSetup() {} + sasl.startSasl(sasl.jaasSections(kafkaServerSaslMechanisms, Some(kafkaClientSaslMechanism), KafkaSasl, JaasTestUtils.KafkaServerContextName)) + config.saslServerProperties().putAll(sasl.kafkaServerSaslProperties(kafkaServerSaslMechanisms, kafkaClientSaslMechanism)) + config.saslClientProperties().putAll(sasl.kafkaClientSaslProperties(kafkaClientSaslMechanism)) + super.brokerPropertyOverrides(config.serverProperties()) } - @Test + @ClusterTest(securityProtocol = SecurityProtocol.SASL_PLAINTEXT, clusterType = Type.ZK) def testApiVersionsRequestBeforeSaslHandshakeRequest(): Unit = { - val socket = connect() + val socket = IntegrationTestUtils.connect(cluster.brokerSocketServers().asScala.head, cluster.clientListener()) try { - val apiVersionsResponse = sendAndReceive[ApiVersionsResponse]( + val apiVersionsResponse = IntegrationTestUtils.sendAndReceive[ApiVersionsResponse]( new ApiVersionsRequest.Builder().build(0), socket) - validateApiVersionsResponse(apiVersionsResponse) + validateApiVersionsResponse(apiVersionsResponse, cluster.clientListener()) sendSaslHandshakeRequestValidateResponse(socket) } finally { socket.close() } } - @Test + @ClusterTest(securityProtocol = SecurityProtocol.SASL_PLAINTEXT, clusterType = Type.ZK) def testApiVersionsRequestAfterSaslHandshakeRequest(): Unit = { - val socket = connect() + val socket = IntegrationTestUtils.connect(cluster.brokerSocketServers().asScala.head, cluster.clientListener()) try { sendSaslHandshakeRequestValidateResponse(socket) - val response = sendAndReceive[ApiVersionsResponse]( + val response = IntegrationTestUtils.sendAndReceive[ApiVersionsResponse]( new ApiVersionsRequest.Builder().build(0), socket) assertEquals(Errors.ILLEGAL_SASL_STATE.code, response.data.errorCode) } finally { @@ -73,26 +79,31 @@ class SaslApiVersionsRequestTest extends AbstractApiVersionsRequestTest with Sas } } - @Test + @ClusterTest(securityProtocol = SecurityProtocol.SASL_PLAINTEXT, clusterType = Type.ZK) def testApiVersionsRequestWithUnsupportedVersion(): Unit = { - val socket = connect() + val socket = IntegrationTestUtils.connect(cluster.brokerSocketServers().asScala.head, cluster.clientListener()) try { val apiVersionsRequest = new ApiVersionsRequest.Builder().build(0) val apiVersionsResponse = sendUnsupportedApiVersionRequest(apiVersionsRequest) assertEquals(Errors.UNSUPPORTED_VERSION.code, apiVersionsResponse.data.errorCode) - val apiVersionsResponse2 = sendAndReceive[ApiVersionsResponse]( + val apiVersionsResponse2 = IntegrationTestUtils.sendAndReceive[ApiVersionsResponse]( new ApiVersionsRequest.Builder().build(0), socket) - validateApiVersionsResponse(apiVersionsResponse2) + validateApiVersionsResponse(apiVersionsResponse2, cluster.clientListener()) sendSaslHandshakeRequestValidateResponse(socket) } finally { socket.close() } } + @AfterEach + def closeSasl(): Unit = { + sasl.closeSasl() + } + private def sendSaslHandshakeRequestValidateResponse(socket: Socket): Unit = { val request = new SaslHandshakeRequest(new SaslHandshakeRequestData().setMechanism("PLAIN"), ApiKeys.SASL_HANDSHAKE.latestVersion) - val response = sendAndReceive[SaslHandshakeResponse](request, socket) + val response = IntegrationTestUtils.sendAndReceive[SaslHandshakeResponse](request, socket) assertEquals(Errors.NONE, response.error) assertEquals(Collections.singletonList("PLAIN"), response.enabledMechanisms) } diff --git a/core/src/test/scala/unit/kafka/server/ZkAdminManagerTest.scala b/core/src/test/scala/unit/kafka/server/ZkAdminManagerTest.scala index 01c4474dd4..ae0ac79c41 100644 --- a/core/src/test/scala/unit/kafka/server/ZkAdminManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ZkAdminManagerTest.scala @@ -17,7 +17,7 @@ package kafka.server -import kafka.zk.KafkaZkClient +import kafka.zk.{AdminZkClient, KafkaZkClient} import org.apache.kafka.common.metrics.Metrics import org.easymock.EasyMock import kafka.utils.TestUtils @@ -25,7 +25,6 @@ import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.message.DescribeConfigsRequestData import org.apache.kafka.common.message.DescribeConfigsResponseData import org.apache.kafka.common.protocol.Errors - import org.junit.jupiter.api.{AfterEach, Test} import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.Assertions.assertFalse @@ -33,6 +32,8 @@ import org.junit.jupiter.api.Assertions.assertNotNull import org.junit.jupiter.api.Assertions.assertNotEquals import java.util.Properties +import kafka.server.metadata.ZkConfigRepository + class ZkAdminManagerTest { private val zkClient: KafkaZkClient = EasyMock.createNiceMock(classOf[KafkaZkClient]) @@ -46,9 +47,9 @@ class ZkAdminManagerTest { metrics.close() } - def createAdminManager(): ZkAdminManager = { + def createConfigHelper(metadataCache: MetadataCache, zkClient: KafkaZkClient): ConfigHelper = { val props = TestUtils.createBrokerConfig(brokerId, "zk") - new ZkAdminManager(KafkaConfig.fromProps(props), metrics, metadataCache, zkClient) + new ConfigHelper(metadataCache, KafkaConfig.fromProps(props), new ZkConfigRepository(new AdminZkClient(zkClient))) } @Test @@ -62,8 +63,8 @@ class ZkAdminManagerTest { .setResourceName(topic) .setResourceType(ConfigResource.Type.TOPIC.id) .setConfigurationKeys(null)) - val adminManager = createAdminManager() - val results: List[DescribeConfigsResponseData.DescribeConfigsResult] = adminManager.describeConfigs(resources, true, true) + val configHelper = createConfigHelper(metadataCache, zkClient) + val results: List[DescribeConfigsResponseData.DescribeConfigsResult] = configHelper.describeConfigs(resources, true, true) assertEquals(Errors.NONE.code, results.head.errorCode()) assertFalse(results.head.configs().isEmpty, "Should return configs") } @@ -78,8 +79,8 @@ class ZkAdminManagerTest { val resources = List(new DescribeConfigsRequestData.DescribeConfigsResource() .setResourceName(topic) .setResourceType(ConfigResource.Type.TOPIC.id)) - val adminManager = createAdminManager() - val results: List[DescribeConfigsResponseData.DescribeConfigsResult] = adminManager.describeConfigs(resources, true, true) + val configHelper = createConfigHelper(metadataCache, zkClient) + val results: List[DescribeConfigsResponseData.DescribeConfigsResult] = configHelper.describeConfigs(resources, true, true) assertEquals(Errors.NONE.code, results.head.errorCode()) assertFalse(results.head.configs().isEmpty, "Should return configs") } @@ -91,7 +92,7 @@ class ZkAdminManagerTest { EasyMock.expect(metadataCache.contains(topic)).andReturn(true) EasyMock.replay(zkClient, metadataCache) - val adminManager = createAdminManager() + val configHelper = createConfigHelper(metadataCache, zkClient) val resources = List( new DescribeConfigsRequestData.DescribeConfigsResource() @@ -101,7 +102,7 @@ class ZkAdminManagerTest { .setResourceName(brokerId.toString) .setResourceType(ConfigResource.Type.BROKER.id)) - val results: List[DescribeConfigsResponseData.DescribeConfigsResult] = adminManager.describeConfigs(resources, true, true) + val results: List[DescribeConfigsResponseData.DescribeConfigsResult] = configHelper.describeConfigs(resources, true, true) assertEquals(2, results.size) results.foreach(r => { assertEquals(Errors.NONE.code, r.errorCode) diff --git a/core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala b/core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala index aaf82cfd59..d95b15d73b 100644 --- a/core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala +++ b/core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala @@ -22,10 +22,11 @@ import java.util.concurrent.atomic.AtomicBoolean import kafka.log.{Log, LogManager} import kafka.server.QuotaFactory.QuotaManagers import kafka.server._ +import kafka.server.metadata.CachedConfigRepository import kafka.utils.{MockTime, TestUtils} import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.{OffsetForLeaderTopic, OffsetForLeaderPartition} -import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.{OffsetForLeaderTopicResult, EpochEndOffset} +import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.{OffsetForLeaderPartition, OffsetForLeaderTopic} +import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.{EpochEndOffset, OffsetForLeaderTopicResult} import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.record.RecordBatch @@ -41,6 +42,7 @@ class OffsetsForLeaderEpochTest { private val time = new MockTime private val metrics = new Metrics private val alterIsrManager = TestUtils.createAlterIsrManager() + private val configRepository = new CachedConfigRepository() private val tp = new TopicPartition("topic", 1) private var replicaManager: ReplicaManager = _ private var quotaManager: QuotaManagers = _ @@ -65,9 +67,9 @@ class OffsetsForLeaderEpochTest { replay(mockLog, logManager) // create a replica manager with 1 partition that has 1 replica - replicaManager = new ReplicaManager(config, metrics, time, null, null, logManager, new AtomicBoolean(false), + replicaManager = new ReplicaManager(config, metrics, time, None, null, logManager, new AtomicBoolean(false), quotaManager, new BrokerTopicStats, - new MetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size), alterIsrManager) + MetadataCache.zkMetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size), alterIsrManager, configRepository) val partition = replicaManager.createPartition(tp) partition.setLog(mockLog, isFutureLog = false) partition.leaderReplicaIdOpt = Some(config.brokerId) @@ -88,9 +90,9 @@ class OffsetsForLeaderEpochTest { replay(logManager) //create a replica manager with 1 partition that has 0 replica - replicaManager = new ReplicaManager(config, metrics, time, null, null, logManager, new AtomicBoolean(false), + replicaManager = new ReplicaManager(config, metrics, time, None, null, logManager, new AtomicBoolean(false), quotaManager, new BrokerTopicStats, - new MetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size), alterIsrManager) + MetadataCache.zkMetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size), alterIsrManager, configRepository) replicaManager.createPartition(tp) //Given @@ -113,9 +115,9 @@ class OffsetsForLeaderEpochTest { replay(logManager) //create a replica manager with 0 partition - replicaManager = new ReplicaManager(config, metrics, time, null, null, logManager, new AtomicBoolean(false), + replicaManager = new ReplicaManager(config, metrics, time, None, null, logManager, new AtomicBoolean(false), quotaManager, new BrokerTopicStats, - new MetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size), alterIsrManager) + MetadataCache.zkMetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size), alterIsrManager, configRepository) //Given val epochRequested: Integer = 5 diff --git a/core/src/test/scala/unit/kafka/server/metadata/CachedConfigRepositoryTest.scala b/core/src/test/scala/unit/kafka/server/metadata/CachedConfigRepositoryTest.scala new file mode 100644 index 0000000000..6ec2b0657e --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/metadata/CachedConfigRepositoryTest.scala @@ -0,0 +1,78 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server.metadata + +import java.util.Properties + +import org.junit.jupiter.api.Assertions.assertEquals +import org.junit.jupiter.api.Test + +class CachedConfigRepositoryTest { + @Test + def testEmptyRepository(): Unit = { + val repository = new CachedConfigRepository() + assertEquals(new Properties(), repository.brokerConfig(0)) + assertEquals(new Properties(), repository.topicConfig("foo")) + } + + @Test + def testSetBrokerConfig(): Unit = { + val repository = new CachedConfigRepository() + val brokerId0 = 0 + repository.setBrokerConfig(brokerId0, "foo", null) + assertEquals(new Properties(), repository.brokerConfig(0)) + + val brokerId1 = 1 + repository.setBrokerConfig(brokerId1, "foo", "bar") + val brokerProperties = new Properties() + brokerProperties.put("foo", "bar") + assertEquals(brokerProperties, repository.brokerConfig(brokerId1)) + + val brokerProperties2 = new Properties() + brokerProperties2.put("foo", "bar") + brokerProperties2.put("foo2", "baz") + repository.setBrokerConfig(brokerId1, "foo2", "baz") // add another prop + assertEquals(brokerProperties2, repository.brokerConfig(brokerId1)) // should get both props + + repository.setBrokerConfig(brokerId1, "foo2", null) + assertEquals(brokerProperties, repository.brokerConfig(brokerId1)) + } + + @Test + def testSetTopicConfig(): Unit = { + val repository = new CachedConfigRepository() + val topic0 = "topic0" + repository.setTopicConfig(topic0, "foo", null) + assertEquals(new Properties(), repository.brokerConfig(0)) + + val topic1 = "topic1" + repository.setTopicConfig(topic1, "foo", "bar") + val topicProperties = new Properties() + topicProperties.put("foo", "bar") + assertEquals(topicProperties, repository.topicConfig(topic1)) + + val topicProperties2 = new Properties() + topicProperties2.put("foo", "bar") + topicProperties2.put("foo2", "baz") + repository.setTopicConfig(topic1, "foo2", "baz") // add another prop + assertEquals(topicProperties2, repository.topicConfig(topic1)) // should get both props + + repository.setTopicConfig(topic1, "foo2", null) + assertEquals(topicProperties, repository.topicConfig(topic1)) + } +} diff --git a/core/src/test/scala/unit/kafka/server/metadata/ZkConfigRepositoryTest.scala b/core/src/test/scala/unit/kafka/server/metadata/ZkConfigRepositoryTest.scala new file mode 100644 index 0000000000..f8737751fa --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/metadata/ZkConfigRepositoryTest.scala @@ -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 kafka.server + +import java.util.Properties + +import kafka.server.metadata.ZkConfigRepository +import kafka.zk.KafkaZkClient +import org.apache.kafka.common.config.ConfigResource +import org.apache.kafka.common.config.ConfigResource.Type +import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows} +import org.junit.jupiter.api.Test +import org.mockito.Mockito.{mock, when} + +class ZkConfigRepositoryTest { + + @Test + def testZkConfigRepository(): Unit = { + val zkClient: KafkaZkClient = mock(classOf[KafkaZkClient]) + val zkConfigRepository = ZkConfigRepository(zkClient) + val brokerId = 1 + val topic = "topic" + val brokerProps = new Properties() + brokerProps.put("a", "b") + val topicProps = new Properties() + topicProps.put("c", "d") + when(zkClient.getEntityConfigs(ConfigType.Broker, brokerId.toString)).thenReturn(brokerProps) + when(zkClient.getEntityConfigs(ConfigType.Topic, topic)).thenReturn(topicProps) + assertEquals(brokerProps, zkConfigRepository.brokerConfig(brokerId)) + assertEquals(topicProps, zkConfigRepository.topicConfig(topic)) + } + + @Test + def testUnsupportedTypes(): Unit = { + val zkClient: KafkaZkClient = mock(classOf[KafkaZkClient]) + val zkConfigRepository = ZkConfigRepository(zkClient) + Type.values().foreach(value => if (value != Type.BROKER && value != Type.TOPIC) + assertThrows(classOf[IllegalArgumentException], () => zkConfigRepository.config(new ConfigResource(value, value.toString)))) + } +} diff --git a/core/src/test/scala/unit/kafka/tools/ClusterToolTest.scala b/core/src/test/scala/unit/kafka/tools/ClusterToolTest.scala new file mode 100644 index 0000000000..0ce100cf94 --- /dev/null +++ b/core/src/test/scala/unit/kafka/tools/ClusterToolTest.scala @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.tools + +import java.io.{ByteArrayOutputStream, PrintStream} +import org.apache.kafka.clients.admin.MockAdminClient +import org.junit.jupiter.api.Assertions._ +import org.junit.jupiter.api.{Test, Timeout} + +@Timeout(value = 60) +class ClusterToolTest { + @Test + def testPrintClusterId(): Unit = { + val adminClient = new MockAdminClient.Builder(). + clusterId("QtNwvtfVQ3GEFpzOmDEE-w"). + build() + val stream = new ByteArrayOutputStream() + ClusterTool.clusterIdCommand(new PrintStream(stream), adminClient) + assertEquals( + s"""Cluster ID: QtNwvtfVQ3GEFpzOmDEE-w +""", stream.toString()) + } + + @Test + def testClusterTooOldToHaveId(): Unit = { + val adminClient = new MockAdminClient.Builder(). + clusterId(null). + build() + val stream = new ByteArrayOutputStream() + ClusterTool.clusterIdCommand(new PrintStream(stream), adminClient) + assertEquals( + s"""No cluster ID found. The Kafka version is probably too old. +""", stream.toString()) + } + + @Test + def testDecommissionBroker(): Unit = { + val adminClient = new MockAdminClient.Builder().numBrokers(3). + usingRaftController(true). + build() + val stream = new ByteArrayOutputStream() + ClusterTool.decommissionCommand(new PrintStream(stream), adminClient, 0) + assertEquals( + s"""Broker 0 is no longer registered. Note that if the broker is still running, or is restarted, it will re-register. +""", stream.toString()) + } + + @Test + def testLegacyModeClusterCannotDecommissionBroker(): Unit = { + val adminClient = new MockAdminClient.Builder().numBrokers(3). + usingRaftController(false). + build() + val stream = new ByteArrayOutputStream() + ClusterTool.decommissionCommand(new PrintStream(stream), adminClient, 0) + assertEquals( + s"""The target cluster does not support broker decommissioning. +""", stream.toString()) + } +} diff --git a/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala b/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala new file mode 100644 index 0000000000..d601e36fcf --- /dev/null +++ b/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala @@ -0,0 +1,187 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.tools + +import java.io.{ByteArrayOutputStream, PrintStream} +import java.nio.charset.StandardCharsets +import java.nio.file.Files +import java.util +import java.util.Properties + +import kafka.server.{KafkaConfig, MetaProperties} +import kafka.utils.TestUtils +import org.apache.kafka.common.Uuid +import org.apache.kafka.common.utils.Utils +import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows} +import org.junit.jupiter.api.{Test, Timeout} + + +@Timeout(value = 40) +class StorageToolTest { + private def newKip500Properties() = { + val properties = new Properties() + properties.setProperty(KafkaConfig.LogDirsProp, "/tmp/foo,/tmp/bar") + properties.setProperty(KafkaConfig.ProcessRolesProp, "controller") + properties.setProperty(KafkaConfig.NodeIdProp, "2") + properties + } + + @Test + def testConfigToLogDirectories(): Unit = { + val config = new KafkaConfig(newKip500Properties()) + assertEquals(Seq("/tmp/bar", "/tmp/foo"), StorageTool.configToLogDirectories(config)) + } + + @Test + def testConfigToLogDirectoriesWithMetaLogDir(): Unit = { + val properties = newKip500Properties() + properties.setProperty(KafkaConfig.MetadataLogDirProp, "/tmp/baz") + val config = new KafkaConfig(properties) + assertEquals(Seq("/tmp/bar", "/tmp/baz", "/tmp/foo"), + StorageTool.configToLogDirectories(config)) + } + + @Test + def testInfoCommandOnEmptyDirectory(): Unit = { + val stream = new ByteArrayOutputStream() + val tempDir = TestUtils.tempDir() + try { + assertEquals(1, StorageTool. + infoCommand(new PrintStream(stream), true, Seq(tempDir.toString))) + assertEquals(s"""Found log directory: + ${tempDir.toString} + +Found problem: + ${tempDir.toString} is not formatted. + +""", stream.toString()) + } finally Utils.delete(tempDir) + } + + @Test + def testInfoCommandOnMissingDirectory(): Unit = { + val stream = new ByteArrayOutputStream() + val tempDir = TestUtils.tempDir() + tempDir.delete() + try { + assertEquals(1, StorageTool. + infoCommand(new PrintStream(stream), true, Seq(tempDir.toString))) + assertEquals(s"""Found problem: + ${tempDir.toString} does not exist + +""", stream.toString()) + } finally Utils.delete(tempDir) + } + + @Test + def testInfoCommandOnDirectoryAsFile(): Unit = { + val stream = new ByteArrayOutputStream() + val tempFile = TestUtils.tempFile() + try { + assertEquals(1, StorageTool. + infoCommand(new PrintStream(stream), true, Seq(tempFile.toString))) + assertEquals(s"""Found problem: + ${tempFile.toString} is not a directory + +""", stream.toString()) + } finally tempFile.delete() + } + + @Test + def testInfoWithMismatchedLegacyKafkaConfig(): Unit = { + val stream = new ByteArrayOutputStream() + val tempDir = TestUtils.tempDir() + try { + Files.write(tempDir.toPath.resolve("meta.properties"), + String.join("\n", util.Arrays.asList( + "version=1", + "cluster.id=XcZZOzUqS4yHOjhMQB6JLQ")). + getBytes(StandardCharsets.UTF_8)) + assertEquals(1, StorageTool. + infoCommand(new PrintStream(stream), false, Seq(tempDir.toString))) + assertEquals(s"""Found log directory: + ${tempDir.toString} + +Found metadata: {cluster.id=XcZZOzUqS4yHOjhMQB6JLQ, version=1} + +Found problem: + The kafka configuration file appears to be for a legacy cluster, but the directories are formatted for kip-500. + +""", stream.toString()) + } finally Utils.delete(tempDir) + } + + @Test + def testInfoWithMismatchedKip500KafkaConfig(): Unit = { + val stream = new ByteArrayOutputStream() + val tempDir = TestUtils.tempDir() + try { + Files.write(tempDir.toPath.resolve("meta.properties"), + String.join("\n", util.Arrays.asList( + "version=0", + "broker.id=1", + "cluster.id=26c36907-4158-4a35-919d-6534229f5241")). + getBytes(StandardCharsets.UTF_8)) + assertEquals(1, StorageTool. + infoCommand(new PrintStream(stream), true, Seq(tempDir.toString))) + assertEquals(s"""Found log directory: + ${tempDir.toString} + +Found metadata: {broker.id=1, cluster.id=26c36907-4158-4a35-919d-6534229f5241, version=0} + +Found problem: + The kafka configuration file appears to be for a kip-500 cluster, but the directories are formatted for legacy mode. + +""", stream.toString()) + } finally Utils.delete(tempDir) + } + + @Test + def testFormatEmptyDirectory(): Unit = { + val tempDir = TestUtils.tempDir() + try { + val metaProperties = MetaProperties( + clusterId = Uuid.fromString("XcZZOzUqS4yHOjhMQB6JLQ"), nodeId = 2) + val stream = new ByteArrayOutputStream() + assertEquals(0, StorageTool. + formatCommand(new PrintStream(stream), Seq(tempDir.toString), metaProperties, false)) + assertEquals("Formatting %s%n".format(tempDir), stream.toString()) + + try assertEquals(1, StorageTool. + formatCommand(new PrintStream(new ByteArrayOutputStream()), Seq(tempDir.toString), metaProperties, false)) catch { + case e: TerseFailure => assertEquals(s"Log directory ${tempDir} is already " + + "formatted. Use --ignore-formatted to ignore this directory and format the " + + "others.", e.getMessage) + } + + val stream2 = new ByteArrayOutputStream() + assertEquals(0, StorageTool. + formatCommand(new PrintStream(stream2), Seq(tempDir.toString), metaProperties, true)) + assertEquals("All of the log directories are already formatted.%n".format(), stream2.toString()) + } finally Utils.delete(tempDir) + } + + @Test + def testFormatWithInvalidClusterId(): Unit = { + val config = new KafkaConfig(newKip500Properties()) + assertEquals("Cluster ID string invalid does not appear to be a valid UUID: " + + "Input string `invalid` decoded as 5 bytes, which is not equal to the expected " + + "16 bytes of a base64-encoded UUID", assertThrows(classOf[TerseFailure], + () => StorageTool.buildMetadataProperties("invalid", config)).getMessage) + } +} diff --git a/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala b/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala index 6436c62dda..c62224c85d 100644 --- a/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala +++ b/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala @@ -17,15 +17,12 @@ package kafka.utils -import kafka.server.{KafkaConfig, ReplicaFetcherManager, ReplicaManager} import kafka.api.LeaderAndIsr import kafka.controller.LeaderIsrAndControllerEpoch -import kafka.log.{Log, LogManager} import kafka.zk._ import org.apache.kafka.common.TopicPartition import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{BeforeEach, Test} -import org.easymock.EasyMock class ReplicationUtilsTest extends ZooKeeperTestHarness { private val zkVersion = 1 @@ -48,23 +45,6 @@ class ReplicationUtilsTest extends ZooKeeperTestHarness { @Test def testUpdateLeaderAndIsr(): Unit = { - val configs = TestUtils.createBrokerConfigs(1, zkConnect).map(KafkaConfig.fromProps) - val log: Log = EasyMock.createMock(classOf[Log]) - EasyMock.expect(log.logEndOffset).andReturn(20).anyTimes() - EasyMock.expect(log) - EasyMock.replay(log) - - val logManager: LogManager = EasyMock.createMock(classOf[LogManager]) - EasyMock.expect(logManager.getLog(new TopicPartition(topic, partition), false)).andReturn(Some(log)).anyTimes() - EasyMock.replay(logManager) - - val replicaManager: ReplicaManager = EasyMock.createMock(classOf[ReplicaManager]) - EasyMock.expect(replicaManager.config).andReturn(configs.head) - EasyMock.expect(replicaManager.logManager).andReturn(logManager) - EasyMock.expect(replicaManager.replicaFetcherManager).andReturn(EasyMock.createMock(classOf[ReplicaFetcherManager])) - EasyMock.expect(replicaManager.zkClient).andReturn(zkClient) - EasyMock.replay(replicaManager) - zkClient.makeSurePersistentPathExists(IsrChangeNotificationZNode.path) val replicas = List(0, 1) diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 12cd15c24e..6a7db8092f 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -23,13 +23,12 @@ import java.nio.charset.{Charset, StandardCharsets} import java.nio.file.{Files, StandardOpenOption} import java.security.cert.X509Certificate import java.time.Duration -import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger, AtomicReference} +import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} import java.util.{Arrays, Collections, Properties} import java.util.concurrent.{Callable, ExecutionException, Executors, TimeUnit} - import javax.net.ssl.X509TrustManager import kafka.api._ -import kafka.cluster.{Broker, EndPoint, IsrChangeListener, TopicConfigFetcher} +import kafka.cluster.{Broker, EndPoint, IsrChangeListener} import kafka.log._ import kafka.security.auth.{Acl, Resource, Authorizer => LegacyAuthorizer} import kafka.server._ @@ -37,7 +36,7 @@ import kafka.server.checkpoints.OffsetCheckpointFile import com.yammer.metrics.core.Meter import kafka.controller.LeaderIsrAndControllerEpoch import kafka.metrics.KafkaYammerMetrics -import kafka.server.metadata.MetadataBroker +import kafka.server.metadata.{CachedConfigRepository, ConfigRepository, MetadataBroker} import kafka.utils.Implicits._ import kafka.zk._ import org.apache.kafka.clients.CommonClientConfigs @@ -61,7 +60,6 @@ import org.apache.kafka.common.serialization.{ByteArrayDeserializer, ByteArraySe import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.utils.Utils._ import org.apache.kafka.common.{KafkaFuture, Node, TopicPartition} -import org.apache.kafka.metadata.BrokerState import org.apache.kafka.server.authorizer.{Authorizer => JAuthorizer} import org.apache.kafka.test.{TestSslUtils, TestUtils => JTestUtils} import org.apache.zookeeper.KeeperException.SessionExpiredException @@ -172,8 +170,9 @@ object TestUtils extends Logging { def boundPort(server: KafkaServer, securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT): Int = server.boundPort(ListenerName.forSecurityProtocol(securityProtocol)) - def createBroker(id: Int, host: String, port: Int, securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT): Broker = - new Broker(id, host, port, ListenerName.forSecurityProtocol(securityProtocol), securityProtocol) + def createBroker(id: Int, host: String, port: Int, securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT): MetadataBroker = { + MetadataBroker(id, null, Map(securityProtocol.name -> new Node(id, host, port)), false) + } def createMetadataBroker(id: Int, host: String = "localhost", @@ -678,7 +677,7 @@ object TestUtils extends Logging { brokers } - def deleteBrokersInZk(zkClient: KafkaZkClient, ids: Seq[Int]): Seq[Broker] = { + def deleteBrokersInZk(zkClient: KafkaZkClient, ids: Seq[Int]): Seq[MetadataBroker] = { val brokers = ids.map(createBroker(_, "localhost", 6667, SecurityProtocol.PLAINTEXT)) ids.foreach(b => zkClient.deletePath(BrokerIdsZNode.path + "/" + b)) brokers @@ -1073,11 +1072,12 @@ object TestUtils extends Logging { */ def createLogManager(logDirs: Seq[File] = Seq.empty[File], defaultConfig: LogConfig = LogConfig(), + configRepository: ConfigRepository = new CachedConfigRepository, cleanerConfig: CleanerConfig = CleanerConfig(enableCleaner = false), time: MockTime = new MockTime()): LogManager = { new LogManager(logDirs = logDirs.map(_.getAbsoluteFile), initialOfflineDirs = Array.empty[File], - topicConfigs = Map(), + configRepository = configRepository, initialDefaultConfig = defaultConfig, cleanerConfig = cleanerConfig, recoveryThreadsPerDataDir = 4, @@ -1088,7 +1088,6 @@ object TestUtils extends Logging { maxPidExpirationMs = 60 * 60 * 1000, scheduler = time.scheduler, time = time, - brokerState = new AtomicReference[BrokerState](BrokerState.NOT_RUNNING), brokerTopicStats = new BrokerTopicStats, logDirFailureChannel = new LogDirFailureChannel(logDirs.size)) } @@ -1155,12 +1154,10 @@ object TestUtils extends Logging { new MockIsrChangeListener() } - class MockTopicConfigFetcher(var props: Properties) extends TopicConfigFetcher { - override def fetch(): Properties = props - } - - def createTopicConfigProvider(props: Properties): MockTopicConfigFetcher = { - new MockTopicConfigFetcher(props) + def createConfigRepository(topic: String, props: Properties): CachedConfigRepository = { + val configRepository = new CachedConfigRepository() + props.entrySet().forEach(e => configRepository.setTopicConfig(topic, e.getKey.toString, e.getValue.toString)) + configRepository } def produceMessages(servers: Seq[KafkaServer], diff --git a/docs/js/templateData.js b/docs/js/templateData.js index 378b8bd5a6..64fe9d8b07 100644 --- a/docs/js/templateData.js +++ b/docs/js/templateData.js @@ -17,8 +17,8 @@ limitations under the License. // Define variables for doc templates var context={ - "version": "28", - "dotVersion": "2.8", - "fullDotVersion": "2.8.0", + "version": "29", + "dotVersion": "2.9", + "fullDotVersion": "2.9.0", "scalaVersion": "2.13" }; diff --git a/docs/security.html b/docs/security.html index b22a165770..b90fbbb4ba 100644 --- a/docs/security.html +++ b/docs/security.html @@ -47,7 +47,7 @@

keytool -keystore {keystorefile} -alias localhost -validity {validity} -genkey -keyalg RSA -storetype pkcs12 +
keytool -keystore {keystorefile} -alias localhost -validity {validity} -genkey -keyalg RSA -storetype pkcs12
You need to specify two parameters in the above command:
  1. keystorefile: the keystore file that stores the keys (and later the certificate) for this broker. The keystore file contains the private @@ -63,7 +63,7 @@

    To generate certificate signing requests run the following command for all server keystores created so far. -
                    keytool -keystore server.keystore.jks -alias localhost -validity {validity} -genkey -keyalg RSA -destkeystoretype pkcs12 -ext SAN=DNS:{FQDN},IP:{IPADDRESS1}
    +
    keytool -keystore server.keystore.jks -alias localhost -validity {validity} -genkey -keyalg RSA -destkeystoretype pkcs12 -ext SAN=DNS:{FQDN},IP:{IPADDRESS1}
    This command assumes that you want to add hostname information to the certificate, if this is not the case, you can omit the extension parameter -ext SAN=DNS:{FQDN},IP:{IPADDRESS1}. Please see below for more information on this.
    Host Name Verification
    @@ -76,7 +76,7 @@
    Host Name Verification
    Server host name verification may be disabled by setting ssl.endpoint.identification.algorithm to an empty string.
    For dynamically configured broker listeners, hostname verification may be disabled using kafka-configs.sh:
    -
                    bin/kafka-configs.sh --bootstrap-server localhost:9093 --entity-type brokers --entity-name 0 --alter --add-config "listener.name.internal.ssl.endpoint.identification.algorithm="
    +
    bin/kafka-configs.sh --bootstrap-server localhost:9093 --entity-type brokers --entity-name 0 --alter --add-config "listener.name.internal.ssl.endpoint.identification.algorithm="

    Note:

    Normally there is no good reason to disable hostname verification apart from being the quickest way to "just get it to work" followed @@ -99,7 +99,7 @@
    Host Name Verification
    To add a SAN field append the following argument -ext SAN=DNS:{FQDN},IP:{IPADDRESS} to the keytool command: -
                    keytool -keystore server.keystore.jks -alias localhost -validity {validity} -genkey -keyalg RSA -destkeystoretype pkcs12 -ext SAN=DNS:{FQDN},IP:{IPADDRESS1}
    +
    keytool -keystore server.keystore.jks -alias localhost -validity {validity} -genkey -keyalg RSA -destkeystoretype pkcs12 -ext SAN=DNS:{FQDN},IP:{IPADDRESS1}

  2. Creating your own CA

    @@ -208,25 +208,25 @@
    Host Name Verification
    Then create a database and serial number file, these will be used to keep track of which certificates were signed with this CA. Both of these are simply text files that reside in the same directory as your CA keys. -
                    echo 01 > serial.txt
    -                touch index.txt
    +
    echo 01 > serial.txt
    +touch index.txt
    With these steps done you are now ready to generate your CA that will be used to sign certificates later. -
                openssl req -x509 -config openssl-ca.cnf -newkey rsa:4096 -sha256 -nodes -out cacert.pem -outform PEM
    +
    openssl req -x509 -config openssl-ca.cnf -newkey rsa:4096 -sha256 -nodes -out cacert.pem -outform PEM
    The CA is simply a public/private key pair and certificate that is signed by itself, and is only intended to sign other certificates.
    This keypair should be kept very safe, if someone gains access to it, they can create and sign certificates that will be trusted by your infrastructure, which means they will be able to impersonate anybody when connecting to any service that trusts this CA.
    The next step is to add the generated CA to the **clients' truststore** so that the clients can trust this CA: -
                    keytool -keystore client.truststore.jks -alias CARoot -import -file ca-cert
    +
    keytool -keystore client.truststore.jks -alias CARoot -import -file ca-cert
    Note: If you configure the Kafka brokers to require client authentication by setting ssl.client.auth to be "requested" or "required" in the Kafka brokers config then you must provide a truststore for the Kafka brokers as well and it should have all the CA certificates that clients' keys were signed by. -
                    keytool -keystore server.truststore.jks -alias CARoot -import -file ca-cert
    +
    keytool -keystore server.truststore.jks -alias CARoot -import -file ca-cert
    In contrast to the keystore in step 1 that stores each machine's own identity, the truststore of a client stores all the certificates that the client should trust. Importing a certificate into one's truststore also means trusting all certificates that are signed by that @@ -237,11 +237,11 @@
    Host Name Verification
  3. Signing the certificate

    Then sign it with the CA: -
                    openssl ca -config openssl-ca.cnf -policy signing_policy -extensions signing_req -out {server certificate} -infiles {certificate signing request}
    +
    openssl ca -config openssl-ca.cnf -policy signing_policy -extensions signing_req -out {server certificate} -infiles {certificate signing request}
    Finally, you need to import both the certificate of the CA and the signed certificate into the keystore: -
                    keytool -keystore {keystore} -alias CARoot -import -file {CA certificate}
    -                keytool -keystore {keystore} -alias localhost -import -file cert-signed
    +
    keytool -keystore {keystore} -alias CARoot -import -file {CA certificate}
    +keytool -keystore {keystore} -alias localhost -import -file cert-signed
    The definitions of the parameters are the following:
      @@ -310,7 +310,7 @@
      SSL key and certificates in PEM format
      harder for a malicious party to obtain certificates with potentially misleading or fraudulent values. It is adviseable to double check signed certificates, whether these contain all requested SAN fields to enable proper hostname verification. The following command can be used to print certificate details to the console, which should be compared with what was originally requested: -
                              openssl x509 -in certificate.crt -text -noout
      +
      openssl x509 -in certificate.crt -text -noout
  4. @@ -320,14 +320,14 @@
    SSL key and certificates in PEM format
    listeners
    If SSL is not enabled for inter-broker communication (see below for how to enable it), both PLAINTEXT and SSL ports will be necessary. -
                listeners=PLAINTEXT://host.name:port,SSL://host.name:port
    +
    listeners=PLAINTEXT://host.name:port,SSL://host.name:port
    Following SSL configs are needed on the broker side -
                ssl.keystore.location=/var/private/ssl/server.keystore.jks
    -            ssl.keystore.password=test1234
    -            ssl.key.password=test1234
    -            ssl.truststore.location=/var/private/ssl/server.truststore.jks
    -            ssl.truststore.password=test1234
    +
    ssl.keystore.location=/var/private/ssl/server.keystore.jks
    +ssl.keystore.password=test1234
    +ssl.key.password=test1234
    +ssl.truststore.location=/var/private/ssl/server.truststore.jks
    +ssl.truststore.password=test1234
    Note: ssl.truststore.password is technically optional but highly recommended. If a password is not set access to the truststore is still available, but integrity checking is disabled. @@ -341,7 +341,7 @@
    SSL key and certificates in PEM format
  5. ssl.secure.random.implementation=SHA1PRNG
If you want to enable SSL for inter-broker communication, add the following to the server.properties file (it defaults to PLAINTEXT) -
            security.inter.broker.protocol=SSL
+
security.inter.broker.protocol=SSL

Due to import regulations in some countries, the Oracle implementation limits the strength of cryptographic algorithms available by default. If stronger algorithms are needed (for example, AES with 256-bit keys), the JCE Unlimited Strength Jurisdiction Policy Files must be obtained and installed in the JDK/JRE. See the @@ -357,31 +357,31 @@

SSL key and certificates in PEM format

Once you start the broker you should be able to see in the server.log -
            with addresses: PLAINTEXT -> EndPoint(192.168.64.1,9092,PLAINTEXT),SSL -> EndPoint(192.168.64.1,9093,SSL)
+
with addresses: PLAINTEXT -> EndPoint(192.168.64.1,9092,PLAINTEXT),SSL -> EndPoint(192.168.64.1,9093,SSL)
To check quickly if the server keystore and truststore are setup properly you can run the following command
openssl s_client -debug -connect localhost:9093 -tls1
(Note: TLSv1 should be listed under ssl.enabled.protocols)
In the output of this command you should see server's certificate: -
            -----BEGIN CERTIFICATE-----
-            {variable sized random bytes}
-            -----END CERTIFICATE-----
-            subject=/C=US/ST=CA/L=Santa Clara/O=org/OU=org/CN=Sriharsha Chintalapani
-            issuer=/C=US/ST=CA/L=Santa Clara/O=org/OU=org/CN=kafka/emailAddress=test@test.com
+
-----BEGIN CERTIFICATE-----
+{variable sized random bytes}
+-----END CERTIFICATE-----
+subject=/C=US/ST=CA/L=Santa Clara/O=org/OU=org/CN=Sriharsha Chintalapani
+issuer=/C=US/ST=CA/L=Santa Clara/O=org/OU=org/CN=kafka/emailAddress=test@test.com
If the certificate does not show up or if there are any other error messages then your keystore is not setup properly.
  • Configuring Kafka Clients

    SSL is supported only for the new Kafka Producer and Consumer, the older API is not supported. The configs for SSL will be the same for both producer and consumer.
    If client authentication is not required in the broker, then the following is a minimal configuration example: -
                security.protocol=SSL
    -            ssl.truststore.location=/var/private/ssl/client.truststore.jks
    -            ssl.truststore.password=test1234
    +
    security.protocol=SSL
    +ssl.truststore.location=/var/private/ssl/client.truststore.jks
    +ssl.truststore.password=test1234
    Note: ssl.truststore.password is technically optional but highly recommended. If a password is not set access to the truststore is still available, but integrity checking is disabled. If client authentication is required, then a keystore must be created like in step 1 and the following must also be configured: -
                ssl.keystore.location=/var/private/ssl/client.keystore.jks
    -            ssl.keystore.password=test1234
    -            ssl.key.password=test1234
    +
    ssl.keystore.location=/var/private/ssl/client.keystore.jks
    +ssl.keystore.password=test1234
    +ssl.key.password=test1234
    Other configuration settings that may also be needed depending on our requirements and the broker configuration:
      @@ -393,8 +393,8 @@
      SSL key and certificates in PEM format

    Examples using console-producer and console-consumer: -
                kafka-console-producer.sh --bootstrap-server localhost:9093 --topic test --producer.config client-ssl.properties
    -            kafka-console-consumer.sh --bootstrap-server localhost:9093 --topic test --consumer.config client-ssl.properties
    +
    kafka-console-producer.sh --bootstrap-server localhost:9093 --topic test --producer.config client-ssl.properties
    +kafka-console-consumer.sh --bootstrap-server localhost:9093 --topic test --consumer.config client-ssl.properties
  • 7.3 Authentication using SASL

    @@ -434,14 +434,14 @@

    listener.name.sasl_ssl.scram-sha-256.sasl.jaas.config=org.apache.kafka.common.security.scram.ScramLoginModule required \ - username="admin" \ - password="admin-secret"; - listener.name.sasl_ssl.plain.sasl.jaas.config=org.apache.kafka.common.security.plain.PlainLoginModule required \ - username="admin" \ - password="admin-secret" \ - user_admin="admin-secret" \ - user_alice="alice-secret"; +
    listener.name.sasl_ssl.scram-sha-256.sasl.jaas.config=org.apache.kafka.common.security.scram.ScramLoginModule required \
    +    username="admin" \
    +    password="admin-secret";
    +listener.name.sasl_ssl.plain.sasl.jaas.config=org.apache.kafka.common.security.plain.PlainLoginModule required \
    +    username="admin" \
    +    password="admin-secret" \
    +    user_admin="admin-secret" \
    +    user_alice="alice-secret";
    If JAAS configuration is defined at different levels, the order of precedence used is:
    @@ -1929,43 +1929,43 @@

    When performing an incremental bounce stop the brokers cleanly via a SIGTERM. It's also good practice to wait for restarted replicas to return to the ISR list before moving onto the next node.

    As an example, say we wish to encrypt both broker-client and broker-broker communication with SSL. In the first incremental bounce, an SSL port is opened on each node: -
                listeners=PLAINTEXT://broker1:9091,SSL://broker1:9092
    +
    listeners=PLAINTEXT://broker1:9091,SSL://broker1:9092
    We then restart the clients, changing their config to point at the newly opened, secured port: -
                bootstrap.servers = [broker1:9092,...]
    -            security.protocol = SSL
    -            ...etc
    +
    bootstrap.servers = [broker1:9092,...]
    +security.protocol = SSL
    +...etc
    In the second incremental server bounce we instruct Kafka to use SSL as the broker-broker protocol (which will use the same SSL port): -
                listeners=PLAINTEXT://broker1:9091,SSL://broker1:9092
    -            security.inter.broker.protocol=SSL
    +
    listeners=PLAINTEXT://broker1:9091,SSL://broker1:9092
    +security.inter.broker.protocol=SSL
    In the final bounce we secure the cluster by closing the PLAINTEXT port: -
                listeners=SSL://broker1:9092
    -            security.inter.broker.protocol=SSL
    +
    listeners=SSL://broker1:9092
    +security.inter.broker.protocol=SSL
    Alternatively we might choose to open multiple ports so that different protocols can be used for broker-broker and broker-client communication. Say we wished to use SSL encryption throughout (i.e. for broker-broker and broker-client communication) but we'd like to add SASL authentication to the broker-client connection also. We would achieve this by opening two additional ports during the first bounce: -
                listeners=PLAINTEXT://broker1:9091,SSL://broker1:9092,SASL_SSL://broker1:9093
    +
    listeners=PLAINTEXT://broker1:9091,SSL://broker1:9092,SASL_SSL://broker1:9093
    We would then restart the clients, changing their config to point at the newly opened, SASL & SSL secured port: -
                bootstrap.servers = [broker1:9093,...]
    -            security.protocol = SASL_SSL
    -            ...etc
    +
    bootstrap.servers = [broker1:9093,...]
    +security.protocol = SASL_SSL
    +...etc
    The second server bounce would switch the cluster to use encrypted broker-broker communication via the SSL port we previously opened on port 9092: -
                listeners=PLAINTEXT://broker1:9091,SSL://broker1:9092,SASL_SSL://broker1:9093
    -            security.inter.broker.protocol=SSL
    +
    listeners=PLAINTEXT://broker1:9091,SSL://broker1:9092,SASL_SSL://broker1:9093
    +security.inter.broker.protocol=SSL
    The final bounce secures the cluster by closing the PLAINTEXT port. -
            listeners=SSL://broker1:9092,SASL_SSL://broker1:9093
    -        security.inter.broker.protocol=SSL
    +
    listeners=SSL://broker1:9092,SASL_SSL://broker1:9093
    +security.inter.broker.protocol=SSL
    ZooKeeper can be secured independently of the Kafka cluster. The steps for doing this are covered in section
    7.6.2. @@ -2028,13 +2028,13 @@

    ZooKeeper Admin Guide. -
            secureClientPort=2182
    -        serverCnxnFactory=org.apache.zookeeper.server.NettyServerCnxnFactory
    -        authProvider.x509=org.apache.zookeeper.server.auth.X509AuthenticationProvider
    -        ssl.keyStore.location=/path/to/zk/keystore.jks
    -        ssl.keyStore.password=zk-ks-passwd
    -        ssl.trustStore.location=/path/to/zk/truststore.jks
    -        ssl.trustStore.password=zk-ts-passwd
    +
    secureClientPort=2182
    +serverCnxnFactory=org.apache.zookeeper.server.NettyServerCnxnFactory
    +authProvider.x509=org.apache.zookeeper.server.auth.X509AuthenticationProvider
    +ssl.keyStore.location=/path/to/zk/keystore.jks
    +ssl.keyStore.password=zk-ks-passwd
    +ssl.trustStore.location=/path/to/zk/truststore.jks
    +ssl.trustStore.password=zk-ts-passwd
    IMPORTANT: ZooKeeper does not support setting the key password in the ZooKeeper server keystore to a value different from the keystore password itself. Be sure to set the key password to be the same as the keystore password. @@ -2042,19 +2042,19 @@
    Here is a sample (partial) Kafka Broker configuration for connecting to ZooKeeper with mTLS authentication. These configurations are described above in Broker Configs.

    -
            # connect to the ZooKeeper port configured for TLS
    -        zookeeper.connect=zk1:2182,zk2:2182,zk3:2182
    -        # required to use TLS to ZooKeeper (default is false)
    -        zookeeper.ssl.client.enable=true
    -        # required to use TLS to ZooKeeper
    -        zookeeper.clientCnxnSocket=org.apache.zookeeper.ClientCnxnSocketNetty
    -        # define key/trust stores to use TLS to ZooKeeper; ignored unless zookeeper.ssl.client.enable=true
    -        zookeeper.ssl.keystore.location=/path/to/kafka/keystore.jks
    -        zookeeper.ssl.keystore.password=kafka-ks-passwd
    -        zookeeper.ssl.truststore.location=/path/to/kafka/truststore.jks
    -        zookeeper.ssl.truststore.password=kafka-ts-passwd
    -        # tell broker to create ACLs on znodes
    -        zookeeper.set.acl=true
    +
    # connect to the ZooKeeper port configured for TLS
    +zookeeper.connect=zk1:2182,zk2:2182,zk3:2182
    +# required to use TLS to ZooKeeper (default is false)
    +zookeeper.ssl.client.enable=true
    +# required to use TLS to ZooKeeper
    +zookeeper.clientCnxnSocket=org.apache.zookeeper.ClientCnxnSocketNetty
    +# define key/trust stores to use TLS to ZooKeeper; ignored unless zookeeper.ssl.client.enable=true
    +zookeeper.ssl.keystore.location=/path/to/kafka/keystore.jks
    +zookeeper.ssl.keystore.password=kafka-ks-passwd
    +zookeeper.ssl.truststore.location=/path/to/kafka/truststore.jks
    +zookeeper.ssl.truststore.password=kafka-ts-passwd
    +# tell broker to create ACLs on znodes
    +zookeeper.set.acl=true
    IMPORTANT: ZooKeeper does not support setting the key password in the ZooKeeper client (i.e. broker) keystore to a value different from the keystore password itself. Be sure to set the key password to be the same as the keystore password. @@ -2063,14 +2063,14 @@

  • Enable SASL and/or mTLS authentication on ZooKeeper. If enabling mTLS, you would now have both a non-TLS port and a TLS port, like this: -
        clientPort=2181
    -    secureClientPort=2182
    -    serverCnxnFactory=org.apache.zookeeper.server.NettyServerCnxnFactory
    -    authProvider.x509=org.apache.zookeeper.server.auth.X509AuthenticationProvider
    -    ssl.keyStore.location=/path/to/zk/keystore.jks
    -    ssl.keyStore.password=zk-ks-passwd
    -    ssl.trustStore.location=/path/to/zk/truststore.jks
    -    ssl.trustStore.password=zk-ts-passwd
    +
    clientPort=2181
    +secureClientPort=2182
    +serverCnxnFactory=org.apache.zookeeper.server.NettyServerCnxnFactory
    +authProvider.x509=org.apache.zookeeper.server.auth.X509AuthenticationProvider
    +ssl.keyStore.location=/path/to/zk/keystore.jks
    +ssl.keyStore.password=zk-ks-passwd
    +ssl.trustStore.location=/path/to/zk/truststore.jks
    +ssl.trustStore.password=zk-ts-passwd
  • Perform a rolling restart of brokers setting the JAAS login file and/or defining ZooKeeper mutual TLS configurations (including connecting to the TLS-enabled ZooKeeper port) as required, which enables brokers to authenticate to ZooKeeper. At the end of the rolling restart, brokers are able to manipulate znodes with strict ACLs, but they will not create znodes with those ACLs
  • If you enabled mTLS, disable the non-TLS port in ZooKeeper
  • @@ -2086,9 +2086,9 @@

    If you are disabling mTLS, disable the TLS port in ZooKeeper Here is an example of how to run the migration tool: -
        bin/zookeeper-security-migration.sh --zookeeper.acl=secure --zookeeper.connect=localhost:2181
    +
    bin/zookeeper-security-migration.sh --zookeeper.acl=secure --zookeeper.connect=localhost:2181

    Run this to see the full list of parameters:

    -
        bin/zookeeper-security-migration.sh --help
    +
    bin/zookeeper-security-migration.sh --help

    7.6.3 Migrating the ZooKeeper ensemble

    It is also necessary to enable SASL and/or mTLS authentication on the ZooKeeper ensemble. To do it, we need to perform a rolling restart of the server and set a few properties. See above for mTLS information. Please refer to the ZooKeeper documentation for more detail:
      @@ -2106,18 +2106,18 @@

      none in ZooKeeper allows clients to connect via a TLS-encrypted connection without presenting their own certificate. Here is a sample (partial) Kafka Broker configuration for connecting to ZooKeeper with just TLS encryption. These configurations are described above in Broker Configs. -
              # connect to the ZooKeeper port configured for TLS
      -        zookeeper.connect=zk1:2182,zk2:2182,zk3:2182
      -        # required to use TLS to ZooKeeper (default is false)
      -        zookeeper.ssl.client.enable=true
      -        # required to use TLS to ZooKeeper
      -        zookeeper.clientCnxnSocket=org.apache.zookeeper.ClientCnxnSocketNetty
      -        # define trust stores to use TLS to ZooKeeper; ignored unless zookeeper.ssl.client.enable=true
      -        # no need to set keystore information assuming ssl.clientAuth=none on ZooKeeper
      -        zookeeper.ssl.truststore.location=/path/to/kafka/truststore.jks
      -        zookeeper.ssl.truststore.password=kafka-ts-passwd
      -        # tell broker to create ACLs on znodes (if using SASL authentication, otherwise do not set this)
      -        zookeeper.set.acl=true
      +
      # connect to the ZooKeeper port configured for TLS
      +zookeeper.connect=zk1:2182,zk2:2182,zk3:2182
      +# required to use TLS to ZooKeeper (default is false)
      +zookeeper.ssl.client.enable=true
      +# required to use TLS to ZooKeeper
      +zookeeper.clientCnxnSocket=org.apache.zookeeper.ClientCnxnSocketNetty
      +# define trust stores to use TLS to ZooKeeper; ignored unless zookeeper.ssl.client.enable=true
      +# no need to set keystore information assuming ssl.clientAuth=none on ZooKeeper
      +zookeeper.ssl.truststore.location=/path/to/kafka/truststore.jks
      +zookeeper.ssl.truststore.password=kafka-ts-passwd
      +# tell broker to create ACLs on znodes (if using SASL authentication, otherwise do not set this)
      +zookeeper.set.acl=true
      diff --git a/docs/streams/developer-guide/dsl-api.html b/docs/streams/developer-guide/dsl-api.html index cd734dea55..c3c4b9302b 100644 --- a/docs/streams/developer-guide/dsl-api.html +++ b/docs/streams/developer-guide/dsl-api.html @@ -334,7 +334,7 @@

      filter and map on a KStream will generate another KStream - - branch on KStream can generate multiple KStreams

      + - split on KStream can generate multiple KStreams

      Some others may generate a KTable object, for example an aggregation of a KStream also yields a KTable. This allows Kafka Streams to continuously update the computed value upon arrivals of out-of-order records after it has already been produced to the downstream transformation operators.

      All KTable transformation operations can only generate another KTable. However, the Kafka Streams DSL does provide a special function @@ -366,24 +366,28 @@

      Branch (or split) a KStream based on the supplied predicates into one or more KStream instances. - (details)

      + (details)

      Predicates are evaluated in order. A record is placed to one and only one output stream on the first match: if the n-th predicate evaluates to true, the record is placed to n-th stream. If no predicate matches, the the record is dropped.

      Branching is useful, for example, to route records to different downstream topics.

      -
      KStream<String, Long> stream = ...;
      -KStream<String, Long>[] branches = stream.branch(
      -    (key, value) -> key.startsWith("A"), /* first predicate  */
      -    (key, value) -> key.startsWith("B"), /* second predicate */
      -    (key, value) -> true                 /* third predicate  */
      -  );
      -
      -// KStream branches[0] contains all records whose keys start with "A"
      -// KStream branches[1] contains all records whose keys start with "B"
      -// KStream branches[2] contains all other records
      -
      -// Java 7 example: cf. `filter` for how to create `Predicate` instances
      -
      +
      +KStream stream = ...;
      +Map> branches =
      +    stream.split(Named.as("Branch-"))
      +        .branch((key, value) -> key.startsWith("A"),  /* first predicate  */
      +             Branched.as("A"))
      +        .branch((key, value) -> key.startsWith("B"),  /* second predicate */
      +             Branched.as("B"))
      +.defaultBranch(Branched.as("C"))
      +);
      +
      +// KStream branches.get("Branch-A") contains all records whose keys start with "A"
      +// KStream branches.get("Branch-B") contains all records whose keys start with "B"
      +// KStream branches.get("Branch-C") contains all other records
      +
      +// Java 7 example: cf. `filter` for how to create `Predicate` instances     
      +                            

      Filter

      diff --git a/docs/streams/developer-guide/memory-mgmt.html b/docs/streams/developer-guide/memory-mgmt.html index 0070705a96..9aa382e089 100644 --- a/docs/streams/developer-guide/memory-mgmt.html +++ b/docs/streams/developer-guide/memory-mgmt.html @@ -168,7 +168,15 @@

      RocksDB

      Each instance of RocksDB allocates off-heap memory for a block cache, index and filter blocks, and memtable (write buffer). Critical configs (for RocksDB version 4.1.0) include block_cache_size, write_buffer_size and max_write_buffer_number. These can be specified through the - rocksdb.config.setter configuration. + rocksdb.config.setter configuration.

      +

      Also, we recommend changing RocksDB's default memory allocator, because the default allocator may lead to increased memory consumption. + To change the memory allocator to jemalloc, you need to set the environment variable LD_PRELOADbefore you start your Kafka Streams application:

      +
      +# example: install jemalloc (on Debian)
      +$ apt install -y libjemalloc-dev
      +# set LD_PRELOAD before you start your Kafka Streams application
      +$ export LD_PRELOAD="/usr/lib/x86_64-linux-gnu/libjemalloc.so”
      +      

      As of 2.3.0 the memory usage across all instances can be bounded, limiting the total off-heap memory of your Kafka Streams application. To do so you must configure RocksDB to cache the index and filter blocks in the block cache, limit the memtable memory through a shared WriteBufferManager and count its memory against the block cache, and then pass the same Cache object to each instance. See RocksDB Memory Usage for details. An example RocksDBConfigSetter implementing this is shown below:

      - + - -