You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "mimaison (via GitHub)" <gi...@apache.org> on 2023/02/08 17:34:00 UTC

[GitHub] [kafka] mimaison commented on a diff in pull request #13137: KAFKA-15086: Intra-cluster communication for Mirror Maker 2

mimaison commented on code in PR #13137:
URL: https://github.com/apache/kafka/pull/13137#discussion_r1100386611


##########
connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMakerConfig.java:
##########
@@ -80,18 +81,25 @@ public class MirrorMakerConfig extends AbstractConfig {
     static final String TARGET_CLUSTER_PREFIX = "target.cluster.";
     static final String SOURCE_PREFIX = "source.";
     static final String TARGET_PREFIX = "target.";
+    static final String ENABLE_INTERNAL_REST_CONFIG = "dedicated.mode.enable.internal.rest";
+    private static final String ENABLE_INTERNAL_REST_DOC =
+            "Whether to bring up an internal-only REST server that allows multi-node clusters to operate correctly";

Review Comment:
   Nit: We usually have a period at the end of descriptions.



##########
connect/mirror/src/main/java/org/apache/kafka/connect/mirror/rest/resources/InternalMirrorResource.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.connect.mirror.rest.resources;
+
+import org.apache.kafka.connect.mirror.SourceAndTarget;
+import org.apache.kafka.connect.runtime.Herder;
+import org.apache.kafka.connect.runtime.rest.RestClient;
+import org.apache.kafka.connect.runtime.rest.resources.InternalClusterResource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.ws.rs.NotFoundException;
+import javax.ws.rs.Path;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.UriInfo;
+import java.util.List;
+import java.util.Map;
+
+@Path("/{source}/{target}/connectors")
+public class InternalMirrorResource extends InternalClusterResource {
+
+    @Context
+    private UriInfo uriInfo;
+
+    private static final Logger log = LoggerFactory.getLogger(InternalMirrorResource.class);
+
+    private final Map<SourceAndTarget, Herder> herders;
+
+    public InternalMirrorResource(Map<SourceAndTarget, Herder> herders, RestClient restClient) {
+        super(restClient);
+        this.herders = herders;
+    }
+
+    @Override
+    protected Herder herderForRequest() {
+        String source = pathParam("source");
+        String target = pathParam("target");
+        Herder result = herders.get(new SourceAndTarget(source, target));
+        if (result == null) {
+            log.debug("Failed to find herder for source '{}' and target '{}'", source, target);
+            throw new NotFoundException("No replication flow found for source '" + source + "' and target '" + target + "'");
+        }
+        return result;
+    }
+
+    private String pathParam(String name) {
+        List<String> result = uriInfo.getPathParameters().get(name);
+        if (result == null || result.isEmpty())
+            throw new NotFoundException();

Review Comment:
   Should we add a message here?



##########
connect/mirror/src/main/java/org/apache/kafka/connect/mirror/rest/resources/InternalMirrorResource.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.connect.mirror.rest.resources;
+
+import org.apache.kafka.connect.mirror.SourceAndTarget;
+import org.apache.kafka.connect.runtime.Herder;
+import org.apache.kafka.connect.runtime.rest.RestClient;
+import org.apache.kafka.connect.runtime.rest.resources.InternalClusterResource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.ws.rs.NotFoundException;
+import javax.ws.rs.Path;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.UriInfo;
+import java.util.List;
+import java.util.Map;
+
+@Path("/{source}/{target}/connectors")
+public class InternalMirrorResource extends InternalClusterResource {
+
+    @Context
+    private UriInfo uriInfo;
+
+    private static final Logger log = LoggerFactory.getLogger(InternalMirrorResource.class);
+
+    private final Map<SourceAndTarget, Herder> herders;
+
+    public InternalMirrorResource(Map<SourceAndTarget, Herder> herders, RestClient restClient) {
+        super(restClient);
+        this.herders = herders;
+    }
+
+    @Override
+    protected Herder herderForRequest() {
+        String source = pathParam("source");
+        String target = pathParam("target");
+        Herder result = herders.get(new SourceAndTarget(source, target));
+        if (result == null) {
+            log.debug("Failed to find herder for source '{}' and target '{}'", source, target);
+            throw new NotFoundException("No replication flow found for source '" + source + "' and target '" + target + "'");
+        }
+        return result;
+    }
+
+    private String pathParam(String name) {
+        List<String> result = uriInfo.getPathParameters().get(name);

Review Comment:
   Could we use `getFirst()` and not have to check whether result is empty or do `get(0)`.



##########
connect/mirror/src/main/java/org/apache/kafka/connect/mirror/rest/resources/InternalMirrorResource.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.connect.mirror.rest.resources;
+
+import org.apache.kafka.connect.mirror.SourceAndTarget;
+import org.apache.kafka.connect.runtime.Herder;
+import org.apache.kafka.connect.runtime.rest.RestClient;
+import org.apache.kafka.connect.runtime.rest.resources.InternalClusterResource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.ws.rs.NotFoundException;
+import javax.ws.rs.Path;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.UriInfo;
+import java.util.List;
+import java.util.Map;
+
+@Path("/{source}/{target}/connectors")
+public class InternalMirrorResource extends InternalClusterResource {
+
+    @Context
+    private UriInfo uriInfo;
+
+    private static final Logger log = LoggerFactory.getLogger(InternalMirrorResource.class);
+
+    private final Map<SourceAndTarget, Herder> herders;
+
+    public InternalMirrorResource(Map<SourceAndTarget, Herder> herders, RestClient restClient) {
+        super(restClient);
+        this.herders = herders;
+    }
+
+    @Override
+    protected Herder herderForRequest() {
+        String source = pathParam("source");
+        String target = pathParam("target");
+        Herder result = herders.get(new SourceAndTarget(source, target));
+        if (result == null) {
+            log.debug("Failed to find herder for source '{}' and target '{}'", source, target);

Review Comment:
   Since we throw below, do we need this debug line?



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java:
##########
@@ -210,44 +200,47 @@ public void initializeServer() {
         }
 
         log.info("REST server listening at " + jettyServer.getURI() + ", advertising URL " + advertisedUrl());
-        log.info("REST admin endpoints at " + adminUrl());
+        URI adminUrl = adminUrl();
+        if (adminUrl != null)
+            log.info("REST admin endpoints at " + adminUrl);
     }
 
-    public void initializeResources(Herder herder) {
+    protected final void initializeResources() {
         log.info("Initializing REST resources");
+        this.resources = new ArrayList<>();

Review Comment:
   nit: slightly more readable without `this.` here



-- 
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-unsubscribe@kafka.apache.org

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