This is an automated email from the ASF dual-hosted git repository.

chia7712 pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
     new afa54233 MINOR: Fix connector startup error logging (#9784)
afa54233 is described below

commit afa5423356d3d2a2135a51200573b45d097f6d60
Author: Chris Egerton <[email protected]>
AuthorDate: Wed Dec 30 10:06:29 2020 -0500

    MINOR: Fix connector startup error logging (#9784)
    
    If a connector fails on startup, the original cause of the error gets 
discarded by the framework and the only message that gets logged looks like 
this:
    
    ```
    [2020-12-04 16:46:30,464] ERROR [Worker clientId=connect-1, 
groupId=connect-cluster] Failed to start connector 'conn-1' 
(org.apache.kafka.connect.runtime.distributed.DistributedHerder)
    org.apache.kafka.connect.errors.ConnectException: Failed to start 
connector: conn-1
            at 
org.apache.kafka.connect.runtime.distributed.DistributedHerder.lambda$startConnector$5(DistributedHerder.java:1297)
            at 
org.apache.kafka.connect.runtime.Worker.startConnector(Worker.java:258)
            at 
org.apache.kafka.connect.runtime.distributed.DistributedHerder.startConnector(DistributedHerder.java:1321)
            at 
org.apache.kafka.connect.runtime.distributed.DistributedHerder.access$1400(DistributedHerder.java:127)
            at 
org.apache.kafka.connect.runtime.distributed.DistributedHerder$13.call(DistributedHerder.java:1329)
            at 
org.apache.kafka.connect.runtime.distributed.DistributedHerder$13.call(DistributedHerder.java:1325)
            at 
java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
            at 
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
            at 
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
            at java.base/java.lang.Thread.run(Thread.java:834)
    ```
    
    The changes here should cause the original cause of the connector startup 
failure to be logged as well.
    
    ```
    [2020-12-30 09:56:35,481] ERROR [test-connector|worker] [Worker 
clientId=connect-1, groupId=connect-cluster] Failed to start connector 'conn-1' 
(org.apache.kafka.connect.runtime.distributed.DistributedHerder:599)
    org.apache.kafka.connect.errors.ConnectException: Failed to start 
connector: conn-1
        at 
org.apache.kafka.connect.runtime.distributed.DistributedHerder.lambda$startConnector$4(DistributedHerder.java:1298)
        at 
org.apache.kafka.connect.runtime.Worker.startConnector(Worker.java:294)
        at 
org.apache.kafka.connect.runtime.distributed.DistributedHerder.startConnector(DistributedHerder.java:1322)
        at 
org.apache.kafka.connect.runtime.distributed.DistributedHerder.processConnectorConfigUpdates(DistributedHerder.java:597)
        at 
org.apache.kafka.connect.runtime.distributed.DistributedHerder.tick(DistributedHerder.java:416)
        at 
org.apache.kafka.connect.runtime.distributed.DistributedHerder.run(DistributedHerder.java:294)
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
        at java.util.concurrent.FutureTask.run(FutureTask.java:266)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)
    Caused by: org.apache.kafka.connect.errors.ConnectException: Failed to find 
any class that implements
    ```
    
    Reviewers: Chia-Ping Tsai <[email protected]>
---
 .../org/apache/kafka/connect/runtime/distributed/DistributedHerder.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git 
a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java
 
b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java
index 6299eb6..e41009e 100644
--- 
a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java
+++ 
b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java
@@ -1295,7 +1295,7 @@ public class DistributedHerder extends AbstractHerder 
implements Runnable {
         final TargetState initialState = 
configState.targetState(connectorName);
         final Callback<TargetState> onInitialStateChange = (error, newState) 
-> {
             if (error != null) {
-                callback.onCompletion(new ConnectException("Failed to start 
connector: " + connectorName), null);
+                callback.onCompletion(new ConnectException("Failed to start 
connector: " + connectorName, error), null);
                 return;
             }
 

Reply via email to