From 22356d55ef232871f144be750087f71c5ed1e9a9 Mon Sep 17 00:00:00 2001 From: Randall Hauch Date: Tue, 12 Jun 2018 21:35:45 -0700 Subject: [PATCH] KAFKA-7043: Modified plugin isolation whitelist with recently added converters (KIP-305) Several recently-added converters are included in the plugin isolation whitelist, similarly to the `StringConverter`. This is a change in the implementation, and does not affect the approved KIP. Several unit tests were added to verify they are being loaded in isolation, again similarly to `StringConverter`. These changes should be applied only to `trunk` and `2.0`, since these converters were added as part of KIP-305 for AK 2.0. Author: Randall Hauch Reviewers: Magesh Nandakumar , Konstantine Karantasis , Ewen Cheslack-Postava Closes #5198 from rhauch/kafka-7043 --- .../connect/runtime/isolation/PluginUtils.java | 5 +++++ .../runtime/isolation/PluginUtilsTest.java | 15 +++++++++++++++ 2 files changed, 20 insertions(+) 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 9c4151bdd1a..74682206bd3 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 @@ -127,6 +127,11 @@ public class PluginUtils { + "|json\\..*" + "|file\\..*" + "|converters\\..*" + + "|storage\\.DoubleConverter" + + "|storage\\.FloatConverter" + + "|storage\\.IntegerConverter" + + "|storage\\.LongConverter" + + "|storage\\.ShortConverter" + "|storage\\.StringConverter" + "|rest\\.basic\\.auth\\.extension\\.BasicAuthSecurityRestExtension" + ")" 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 71c93637beb..a17520fb064 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 @@ -146,6 +146,21 @@ public class PluginUtilsTest { assertTrue(PluginUtils.shouldLoadInIsolation( "org.apache.kafka.connect.converters.ByteArrayConverter") ); + assertTrue(PluginUtils.shouldLoadInIsolation( + "org.apache.kafka.connect.storage.DoubleConverter") + ); + assertTrue(PluginUtils.shouldLoadInIsolation( + "org.apache.kafka.connect.storage.FloatConverter") + ); + assertTrue(PluginUtils.shouldLoadInIsolation( + "org.apache.kafka.connect.storage.IntegerConverter") + ); + assertTrue(PluginUtils.shouldLoadInIsolation( + "org.apache.kafka.connect.storage.LongConverter") + ); + assertTrue(PluginUtils.shouldLoadInIsolation( + "org.apache.kafka.connect.storage.ShortConverter") + ); assertTrue(PluginUtils.shouldLoadInIsolation( "org.apache.kafka.connect.storage.StringConverter") );