You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by rh...@apache.org on 2019/05/23 19:14:21 UTC

[kafka] branch trunk updated: KAFKA-8415: Interface ConnectorClientConfigOverridePolicy needs to be excluded from class loading isolation (#6796)

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

rhauch 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 68a3eb3  KAFKA-8415: Interface ConnectorClientConfigOverridePolicy needs to be excluded from class loading isolation (#6796)
68a3eb3 is described below

commit 68a3eb373b5e176468ba5b9c045f327eb75f5f19
Author: Konstantine Karantasis <ko...@confluent.io>
AuthorDate: Thu May 23 12:14:09 2019 -0700

    KAFKA-8415: Interface ConnectorClientConfigOverridePolicy needs to be excluded from class loading isolation (#6796)
    
    Author: Konstantine Karantasis <ko...@confluent.io>
    Reviewer: Randall Hauch <rh...@gmail.com>
---
 .../kafka/connect/runtime/isolation/PluginUtils.java  |  6 +++++-
 .../connect/runtime/isolation/PluginUtilsTest.java    | 19 +++++++++++++++++++
 2 files changed, 24 insertions(+), 1 deletion(-)

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 c029802..b9d5470 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
@@ -123,6 +123,10 @@ public class PluginUtils {
             + "|org\\.slf4j"
             + ")\\..*$");
 
+    // If the base interface or class that will be used to identify Connect plugins resides within
+    // the same java package as the plugins that need to be loaded in isolation (and thus are
+    // added to the WHITELIST), then this base interface or class needs to be excluded in the
+    // regular expression pattern
     private static final Pattern WHITELIST = Pattern.compile("^org\\.apache\\.kafka\\.(?:connect\\.(?:"
             + "transforms\\.(?!Transformation$).*"
             + "|json\\..*"
@@ -131,7 +135,7 @@ public class PluginUtils {
             + "|storage\\.StringConverter"
             + "|storage\\.SimpleHeaderConverter"
             + "|rest\\.basic\\.auth\\.extension\\.BasicAuthSecurityRestExtension"
-            + "|connector\\.policy\\..*"
+            + "|connector\\.policy\\.(?!ConnectorClientConfigOverridePolicy$).*"
             + ")"
             + "|common\\.config\\.provider\\.(?!ConfigProvider$).*"
             + ")$");
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 f9a2d8f..bf441ff 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
@@ -186,6 +186,25 @@ public class PluginUtilsTest {
     }
 
     @Test
+    public void testConnectorClientConfigOverridePolicy() {
+        assertFalse(PluginUtils.shouldLoadInIsolation(
+                "org.apache.kafka.connect.connector.policy.ConnectorClientConfigOverridePolicy")
+        );
+        assertTrue(PluginUtils.shouldLoadInIsolation(
+                "org.apache.kafka.connect.connector.policy.AbstractConnectorClientConfigOverridePolicy")
+        );
+        assertTrue(PluginUtils.shouldLoadInIsolation(
+                "org.apache.kafka.connect.connector.policy.AllConnectorClientConfigOverridePolicy")
+        );
+        assertTrue(PluginUtils.shouldLoadInIsolation(
+                "org.apache.kafka.connect.connector.policy.NoneConnectorClientConfigOverridePolicy")
+        );
+        assertTrue(PluginUtils.shouldLoadInIsolation(
+                "org.apache.kafka.connect.connector.policy.PrincipalConnectorClientConfigOverridePolicy")
+        );
+    }
+
+    @Test
     public void testEmptyPluginUrls() throws Exception {
         assertEquals(Collections.<Path>emptyList(), PluginUtils.pluginUrls(pluginPath));
     }