[GitHub] [kafka] rhauch commented on a change in pull request #9780: KAFKA-10021: Changed Kafka backing stores to use shared admin client to get end offsets and create topics
rhauch commented on a change in pull request #9780: URL: https://github.com/apache/kafka/pull/9780#discussion_r572421771 ## File path: connect/runtime/src/test/java/org/apache/kafka/connect/util/SharedTopicAdminTest.java ## @@ -0,0 +1,118 @@ +/* + * 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 org.apache.kafka.connect.errors.ConnectException; +import org.easymock.EasyMock; +import org.easymock.Mock; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.easymock.PowerMock; +import org.powermock.core.classloader.annotations.PowerMockIgnore; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertThrows; + +@RunWith(PowerMockRunner.class) +@PrepareForTest(KafkaBasedLog.class) +@PowerMockIgnore("javax.management.*") +public class SharedTopicAdminTest { + +private static final Map CONFIG = Collections.emptyMap(); + +@Mock private TopicAdmin mockTopicAdmin; +private SharedTopicAdmin sharedAdmin; +private int created = 0; Review comment: Okay, Mockito FTW! I've rewritten the `SharedTopicAdminTest` class to use Mockito instead of PowerMock and EasyMock, and was able to use mocks to assert the correct number of times an admin instance was created and closed. This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [kafka] rhauch commented on a change in pull request #9780: KAFKA-10021: Changed Kafka backing stores to use shared admin client to get end offsets and create topics
rhauch commented on a change in pull request #9780: URL: https://github.com/apache/kafka/pull/9780#discussion_r572256392 ## File path: connect/runtime/src/test/java/org/apache/kafka/connect/util/SharedTopicAdminTest.java ## @@ -0,0 +1,118 @@ +/* + * 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 org.apache.kafka.connect.errors.ConnectException; +import org.easymock.EasyMock; +import org.easymock.Mock; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.easymock.PowerMock; +import org.powermock.core.classloader.annotations.PowerMockIgnore; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertThrows; + +@RunWith(PowerMockRunner.class) +@PrepareForTest(KafkaBasedLog.class) +@PowerMockIgnore("javax.management.*") +public class SharedTopicAdminTest { + +private static final Map CONFIG = Collections.emptyMap(); + +@Mock private TopicAdmin mockTopicAdmin; +private SharedTopicAdmin sharedAdmin; +private int created = 0; Review comment: I vaguely remember struggling with getting EasyMock to work properly when I originally created this PR, but I tried it again just to be sure. Unfortunately, the `SharedTopicAdmin` constructor takes a _method reference_, and apparently EasyMock provides no way to effectively mock the behavior of a method passed via a method reference. See [this still-open EasyMock issue](https://github.com/easymock/easymock/issues/213) for details. So, while I agree using mocks to verify the # of times the factor is invoked, because EasyMock can't really mock method references means we're stuck with the current code using the counter. This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [kafka] rhauch commented on a change in pull request #9780: KAFKA-10021: Changed Kafka backing stores to use shared admin client to get end offsets and create topics
rhauch commented on a change in pull request #9780: URL: https://github.com/apache/kafka/pull/9780#discussion_r570648556 ## File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java ## @@ -185,16 +188,33 @@ 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 restUrlthe 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, String kafkaClusterId, StatusBackingStore statusBackingStore, ConfigBackingStore configBackingStore, String restUrl, - ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy) { + ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy, + AutoCloseable... uponShutdown) { Review comment: The reason I used a variadic array here was to avoid having to create a new connector when no `AutoCloseable` instances are supplied. If we use a List, then we can change the usage in Connect runtime and in MirrorMaker 2, but anywhere else will break without keeping the old signature. WDYT? ## File path: connect/runtime/src/test/java/org/apache/kafka/connect/util/TopicAdminTest.java ## @@ -457,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
[GitHub] [kafka] rhauch commented on a change in pull request #9780: KAFKA-10021: Changed Kafka backing stores to use shared admin client to get end offsets and create topics
rhauch commented on a change in pull request #9780: URL: https://github.com/apache/kafka/pull/9780#discussion_r570657028 ## File path: 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); Review comment: I'm not sure there is much advantage either way, considering these methods are not called frequently and `synchronized` would indeed work. I personally like the simplicity of using `AtomicReference`, which to me seemed natural and straightforward, avoided having to synchronize the entire methods, and needed no if-checks in this method. This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [kafka] rhauch commented on a change in pull request #9780: KAFKA-10021: Changed Kafka backing stores to use shared admin client to get end offsets and create topics
rhauch commented on a change in pull request #9780: URL: https://github.com/apache/kafka/pull/9780#discussion_r570655008 ## File path: connect/runtime/src/test/java/org/apache/kafka/connect/util/SharedTopicAdminTest.java ## @@ -0,0 +1,118 @@ +/* + * 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 org.apache.kafka.connect.errors.ConnectException; +import org.easymock.EasyMock; +import org.easymock.Mock; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.easymock.PowerMock; +import org.powermock.core.classloader.annotations.PowerMockIgnore; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertThrows; + +@RunWith(PowerMockRunner.class) +@PrepareForTest(KafkaBasedLog.class) +@PowerMockIgnore("javax.management.*") +public class SharedTopicAdminTest { + +private static final Map CONFIG = Collections.emptyMap(); + +@Mock private TopicAdmin mockTopicAdmin; +private SharedTopicAdmin sharedAdmin; +private int created = 0; Review comment: Really I'm just using that to be able to test that the new `topicAdmin()` method is returning the correct instance, even after repeated calls. It was an easy way to verify that the `TopicAdmin` matches what the factory function returned. This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [kafka] rhauch commented on a change in pull request #9780: KAFKA-10021: Changed Kafka backing stores to use shared admin client to get end offsets and create topics
rhauch commented on a change in pull request #9780: URL: https://github.com/apache/kafka/pull/9780#discussion_r570652791 ## File path: connect/runtime/src/test/java/org/apache/kafka/connect/util/TopicAdminTest.java ## @@ -457,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()); +
[GitHub] [kafka] rhauch commented on a change in pull request #9780: KAFKA-10021: Changed Kafka backing stores to use shared admin client to get end offsets and create topics
rhauch commented on a change in pull request #9780: URL: https://github.com/apache/kafka/pull/9780#discussion_r570648556 ## File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java ## @@ -185,16 +188,33 @@ 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 restUrlthe 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, String kafkaClusterId, StatusBackingStore statusBackingStore, ConfigBackingStore configBackingStore, String restUrl, - ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy) { + ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy, + AutoCloseable... uponShutdown) { Review comment: The reason I used a variadic array here was to avoid having to create a new connector when no `AutoCloseable` instances are supplied. If we use a List, then we can change the usage in Connect runtime and in MirrorMaker 2, but anywhere else will break without keeping the old signature. WDYT? This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org