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 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,
                              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<TopicPartition> 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<TopicPartition> 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<TopicPartition> 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<TopicPartition> 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<TopicPartition, Long> offsets = 
admin.endOffsets(Collections.emptySet());
+            assertTrue(offsets.isEmpty());
+        }
+    }
+
+    @Test
+    public void endOffsetsShouldReturnOffsetsForOnePartition() {
+        String topicName = "myTopic";
+        TopicPartition tp1 = new TopicPartition(topicName, 0);
+        Set<TopicPartition> 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<TopicPartition, Long> 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<TopicPartition> 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<TopicPartition, Long> 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<TopicPartition> 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];

Review comment:
       Yes, these are just the ISRs for the one partition that we set up the 
cluster with.
   
   The utility method did allow multiple nodes, but we don't really use that 
much in this class. I think this changed because we now have to define the 
`PartitionInfo` instances rather than an empty map. Not sure why that's now 
different, but supplying the empty infos definitely caused problems in these 
new tests.

##########
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<String, Object> 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.

##########
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)}.
+ *
+ * <p>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.
+ *
+ * <p>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<TopicAdmin> {
+
+    // Visible for testing
+    static final Duration DEFAULT_CLOSE_DURATION = 
Duration.ofMillis(Long.MAX_VALUE);
+
+    private final Map<String, Object> adminProps;
+    private final AtomicReference<TopicAdmin> admin = new AtomicReference<>();
+    private final AtomicBoolean closed = new AtomicBoolean(false);
+    private final Function<Map<String, Object>, TopicAdmin> factory;
+
+    public SharedTopicAdmin(Map<String, Object> adminProps) {
+        this(adminProps, TopicAdmin::new);
+    }
+
+    // Visible for testing
+    SharedTopicAdmin(Map<String, Object> adminProps, Function<Map<String, 
Object>, 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


Reply via email to