You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "Ewen Cheslack-Postava (JIRA)" <ji...@apache.org> on 2017/11/03 23:45:00 UTC

[jira] [Commented] (KAFKA-6148) ClassCastException in BigQuery connector

    [ https://issues.apache.org/jira/browse/KAFKA-6148?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16238589#comment-16238589 ] 

Ewen Cheslack-Postava commented on KAFKA-6148:
----------------------------------------------

I think this is due to the new classpath isolation. I see from https://github.com/apache/kafka/blob/trunk/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/PluginUtils.java#L120-L121 that we covered excluding org.apache.kafka.common and org.apache.kafka.connect and thought that covered the classes we needed to ensure were loaded from the system classloader, but SinkTask also uses a class from org.apache.kafka.clients.

[~kkonstantine] seem like an accurate assessment? If so, any connector that overrides flush() or precommit() could potentially run into issues (though I'm not sure if the issue is ultimately caused by having a separate method call there that uses the type). I think the code in the BigQuery connector is actually unnecessary -- just flushing the data is sufficient since the same offsets will be used and do not need to be set on the context object explicitly -- but this is also a bug in Connect itself. Looks like the fix is easy and should probably be backported back to the first release branch with classloader isolation.

> ClassCastException in BigQuery connector
> ----------------------------------------
>
>                 Key: KAFKA-6148
>                 URL: https://issues.apache.org/jira/browse/KAFKA-6148
>             Project: Kafka
>          Issue Type: Bug
>          Components: KafkaConnect
>            Reporter: Eugene Burd
>            Priority: Major
>
> I am trying to run a com.wepay.kafka.connect.bigquery.BigQuerySinkConnector connector, but getting the following exception.  
> [2017-10-30 21:48:49,007] ERROR WorkerSinkTask{id=bigquery-connector-log-0} Offset commit failed, rewinding to last committed offsets (org.apache.kafka.connect.runtime.WorkerSinkTask:311)
> java.lang.ClassCastException: org.apache.kafka.clients.consumer.OffsetAndMetadata cannot be cast to org.apache.kafka.clients.consumer.OffsetAndMetadata
> 	at com.wepay.kafka.connect.bigquery.BigQuerySinkTask.updateOffsets(BigQuerySinkTask.java:107)
> 	at com.wepay.kafka.connect.bigquery.BigQuerySinkTask.flush(BigQuerySinkTask.java:96)
> 	at org.apache.kafka.connect.sink.SinkTask.preCommit(SinkTask.java:117)
> 	at org.apache.kafka.connect.runtime.WorkerSinkTask.commitOffsets(WorkerSinkTask.java:305)
> 	at org.apache.kafka.connect.runtime.WorkerSinkTask.iteration(WorkerSinkTask.java:164)
> 	at org.apache.kafka.connect.runtime.WorkerSinkTask.execute(WorkerSinkTask.java:148)
> 	at org.apache.kafka.connect.runtime.WorkerTask.doRun(WorkerTask.java:146)
> 	at org.apache.kafka.connect.runtime.WorkerTask.run(WorkerTask.java:190)
> 	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:1142)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> 	at java.lang.Thread.run(Thread.java:748)
> [2017-10-30 21:48:49,012] ERROR Commit of WorkerSinkTask{id=bigquery-connector-log-0} offsets threw an unexpected exception:  (org.apache.kafka.connect.runtime.WorkerSinkTask:205)
> java.lang.ClassCastException: org.apache.kafka.clients.consumer.OffsetAndMetadata cannot be cast to org.apache.kafka.clients.consumer.OffsetAndMetadata
> 	at com.wepay.kafka.connect.bigquery.BigQuerySinkTask.updateOffsets(BigQuerySinkTask.java:107)
> 	at com.wepay.kafka.connect.bigquery.BigQuerySinkTask.flush(BigQuerySinkTask.java:96)
> 	at org.apache.kafka.connect.sink.SinkTask.preCommit(SinkTask.java:117)
> 	at org.apache.kafka.connect.runtime.WorkerSinkTask.commitOffsets(WorkerSinkTask.java:305)
> 	at org.apache.kafka.connect.runtime.WorkerSinkTask.iteration(WorkerSinkTask.java:164)
> 	at org.apache.kafka.connect.runtime.WorkerSinkTask.execute(WorkerSinkTask.java:148)
> 	at org.apache.kafka.connect.runtime.WorkerTask.doRun(WorkerTask.java:146)
> 	at org.apache.kafka.connect.runtime.WorkerTask.run(WorkerTask.java:190)
> 	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:1142)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> 	at java.lang.Thread.run(Thread.java:748)
> I have checked the version number of kafka client in the plug in and kafka connect itself and they are the same.  
> - kafka-clients-0.11.0.0.jar matches
> I am still suspecting a type of versioning issue.  Do you have any advice? 
> Thanks. 



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)