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

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


The following commit(s) were added to refs/heads/1.0 by this push:
     new 48d7c72  KAFKA-6148; ClassCastException in connectors that include 
kafka-clients packages (#4457)
48d7c72 is described below

commit 48d7c72b61aff622070bd25e37ea8c05c428ef9f
Author: Konstantine Karantasis <[email protected]>
AuthorDate: Tue Jan 30 08:45:07 2018 -0800

    KAFKA-6148; ClassCastException in connectors that include kafka-clients 
packages (#4457)
    
    Exclusion for packages that need not be loaded in isolation needs to be 
extended to all the `org.apache.kafka` packages (that do not belong to 
transforms and the other whitelisted packages). Most notably, this refers to 
any classes in `kafka-clients` package.
    
    Reviewers: Randall Hauch <[email protected]>, Jason Gustafson 
<[email protected]>
---
 .../org/apache/kafka/connect/runtime/isolation/PluginUtils.java  | 3 +--
 .../apache/kafka/connect/runtime/isolation/PluginUtilsTest.java  | 9 +++++++++
 2 files changed, 10 insertions(+), 2 deletions(-)

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 c1774e9..d85986e 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
@@ -118,8 +118,7 @@ public class PluginUtils {
             + "|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 61ece1c..4bc6e15 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
@@ -105,6 +105,15 @@ public class PluginUtilsTest {
         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

-- 
To stop receiving notification emails like this one, please contact
[email protected].

Reply via email to