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

ASF GitHub Bot commented on KAFKA-6148:
---------------------------------------

hachikuji closed pull request #4457: KAFKA-6148: ClassCastException in 
connectors that include kafka-clients
URL: https://github.com/apache/kafka/pull/4457
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/PluginUtils.java
 
b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/PluginUtils.java
index 9ef7c3a40e3..c2611c625b3 100644
--- 
a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/PluginUtils.java
+++ 
b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/PluginUtils.java
@@ -117,8 +117,7 @@
             + "|org\\.omg\\.stub\\.java\\.rmi"
             + "|org\\.w3c\\.dom"
             + "|org\\.xml\\.sax"
-            + "|org\\.apache\\.kafka\\.common"
-            + "|org\\.apache\\.kafka\\.connect"
+            + "|org\\.apache\\.kafka"
             + "|org\\.slf4j"
             + ")\\..*$";
 
diff --git 
a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/PluginUtilsTest.java
 
b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/PluginUtilsTest.java
index f9532a6d8cb..7b258a592ef 100644
--- 
a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/PluginUtilsTest.java
+++ 
b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/PluginUtilsTest.java
@@ -104,6 +104,15 @@ public void testConnectFrameworkClasses() throws Exception 
{
         assertFalse(PluginUtils.shouldLoadInIsolation(
                 "org.apache.kafka.connect.storage.OffsetBackingStore")
         );
+        assertFalse(PluginUtils.shouldLoadInIsolation(
+                "org.apache.kafka.clients.producer.ProducerConfig")
+        );
+        assertFalse(PluginUtils.shouldLoadInIsolation(
+                "org.apache.kafka.clients.consumer.ConsumerConfig")
+        );
+        assertFalse(PluginUtils.shouldLoadInIsolation(
+                "org.apache.kafka.clients.admin.KafkaAdminClient")
+        );
     }
 
     @Test


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> ClassCastException in BigQuery connector
> ----------------------------------------
>
>                 Key: KAFKA-6148
>                 URL: https://issues.apache.org/jira/browse/KAFKA-6148
>             Project: Kafka
>          Issue Type: Bug
>          Components: KafkaConnect
>    Affects Versions: 1.0.0, 0.11.0.2
>            Reporter: Eugene Burd
>            Assignee: Konstantine Karantasis
>            Priority: Major
>             Fix For: 0.11.0.3, 1.0.2
>
>
> 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
(v7.6.3#76005)

Reply via email to