You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2020/06/10 00:52:07 UTC

[GitHub] [kafka] kkonstantine commented on a change in pull request #8069: KAFKA-9374: Make connector interactions asynchronous

kkonstantine commented on a change in pull request #8069:
URL: https://github.com/apache/kafka/pull/8069#discussion_r437229477



##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
##########
@@ -356,40 +390,96 @@ public boolean isSinkConnector(String connName) {
         return result;
     }
 
-    private void stopConnectors() {
-        // Herder is responsible for stopping connectors. This is an internal method to sequentially
-        // stop connectors that have not explicitly been stopped.
-        for (String connector: connectors.keySet())
-            stopConnector(connector);
-    }
-
     /**
      * Stop a connector managed by this worker.
      *
      * @param connName the connector name.
-     * @return true if the connector belonged to this worker and was successfully stopped.
      */
-    public boolean stopConnector(String connName) {
+    private void stopConnector(String connName) {
         try (LoggingContext loggingContext = LoggingContext.forConnector(connName)) {
-            log.info("Stopping connector {}", connName);
-
-            WorkerConnector workerConnector = connectors.remove(connName);
+            WorkerConnector workerConnector = connectors.get(connName);
             if (workerConnector == null) {
                 log.warn("Ignoring stop request for unowned connector {}", connName);
-                return false;
+                return;
             }
 
+            log.info("Stopping connector {}", connName);

Review comment:
       not sure what's the advantage of bringing this log message further down here. Searching the logs for `Stopping connector` will miss the cases of "unowned" connectors, making debugging potentially more challenging. 

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/CloseableConnectorContext.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 org.apache.kafka.connect.runtime;
+
+import org.apache.kafka.connect.connector.ConnectorContext;
+import org.apache.kafka.connect.errors.ConnectException;
+
+import java.io.Closeable;
+
+public interface CloseableConnectorContext extends ConnectorContext, Closeable {

Review comment:
       Any reason not to use `AutoCloseable` instead?

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConnector.java
##########
@@ -266,31 +422,51 @@ public void close() {
         @Override
         public void onStartup(String connector) {
             state = AbstractStatus.State.RUNNING;
-            delegate.onStartup(connector);
+            synchronized (this) {

Review comment:
       Thanks for checking the underlying implementation @C0urante . 
   
   That takes us to my earlier concern about this operation potentially blocking for too long to be in a `synchronized` block. And the potential of blocking does not have to do with acknowledging that the record was written only. The producer call has a metadata update call too. 
   
   Going over the uses of `KafkaBasedLog` in Connect, I didn't find an example where we have `KafkaBasedLog#send` running  in mutual exclusion. Contrary, similar concerns are probably the reason why we call `OffsetStorageWriter#doFlush` outside the synchronized block in `WorkerSourceTask`. 
   
   I think we might be able to live with a rare race condition as the one you described, in order to avoid introducing unintended side-effects due to locking. 




----------------------------------------------------------------
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:
users@infra.apache.org