tombentley commented on code in PR #12577:
URL: https://github.com/apache/kafka/pull/12577#discussion_r991043265


##########
clients/src/main/java/org/apache/kafka/clients/admin/ForwardingAdmin.java:
##########
@@ -0,0 +1,277 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.clients.admin;
+
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.ElectionType;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.TopicCollection;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionReplica;
+import org.apache.kafka.common.acl.AclBinding;
+import org.apache.kafka.common.acl.AclBindingFilter;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.quota.ClientQuotaAlteration;
+import org.apache.kafka.common.quota.ClientQuotaFilter;
+
+import java.time.Duration;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+public class ForwardingAdmin implements Admin {

Review Comment:
   This class needs some javadoc to explain that this is the default 
`forwarding.admin.class` used by MM2, and that it is intended that users 
wanting customized behaviour extend this class and override the methods they 
need. 



##########
clients/src/main/java/org/apache/kafka/clients/admin/ForwardingAdmin.java:
##########
@@ -0,0 +1,277 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.clients.admin;
+
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.ElectionType;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.TopicCollection;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionReplica;
+import org.apache.kafka.common.acl.AclBinding;
+import org.apache.kafka.common.acl.AclBindingFilter;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.quota.ClientQuotaAlteration;
+import org.apache.kafka.common.quota.ClientQuotaFilter;
+
+import java.time.Duration;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+public class ForwardingAdmin implements Admin {
+    private final Admin delegate;
+
+    public ForwardingAdmin(Map<String, Object> configs) {
+        this.delegate = AdminClient.create(configs);

Review Comment:
   I guess this could be `Admin.create`, not need for explicit dependency on 
`AdminClient`.



##########
connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorConnectorConfig.java:
##########
@@ -436,6 +441,17 @@ ReplicationPolicy replicationPolicy() {
         return getConfiguredInstance(REPLICATION_POLICY_CLASS, 
ReplicationPolicy.class);
     }
 
+    @SuppressWarnings("unchecked")
+    ForwardingAdmin getForwardingAdmin(Map<String, Object> config) {
+        try {
+            return Utils.newParameterizedInstance(
+                    getClass(FORWARDING_ADMIN_CLASS).getName(), 
(Class<Map<String, Object>>) (Class) Map.class, config

Review Comment:
   I think you'll need to suppress warnings for raw types too.



##########
connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java:
##########
@@ -56,7 +56,7 @@
 public class MirrorClient implements AutoCloseable {
     private static final Logger log = 
LoggerFactory.getLogger(MirrorClient.class);
 
-    private final AdminClient adminClient;
+    private final ForwardingAdmin adminClient;

Review Comment:
   Is this change really necessary? The callers of Admin methods don't need to 
care about `ForwardingAdmin`, they only depend on `Admin` AFAICS.



##########
connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointConnector.java:
##########
@@ -47,7 +47,7 @@ public class MirrorCheckpointConnector extends 
SourceConnector {
     private Scheduler scheduler;
     private MirrorConnectorConfig config;
     private GroupFilter groupFilter;
-    private AdminClient sourceAdminClient;
+    private ForwardingAdmin sourceAdminClient;

Review Comment:
   Again, I think it should keep this as Admin (until it turns out that the 
call sites care).



##########
connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java:
##########
@@ -79,8 +80,8 @@ public class MirrorSourceConnector extends SourceConnector {
     private List<TopicPartition> knownTargetTopicPartitions = 
Collections.emptyList();
     private ReplicationPolicy replicationPolicy;
     private int replicationFactor;
-    private AdminClient sourceAdminClient;
-    private AdminClient targetAdminClient;
+    private ForwardingAdmin sourceAdminClient;
+    private ForwardingAdmin targetAdminClient;

Review Comment:
   Again, there no need for this because the call sites don't care.



-- 
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.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to