You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2020/06/02 07:24:27 UTC

[GitHub] [kafka] kkonstantine commented on a change in pull request #8630: KAFKA-9969: Exclude ConnectorClientConfigRequest from class loading isolation

kkonstantine commented on a change in pull request #8630:
URL: https://github.com/apache/kafka/pull/8630#discussion_r433640144



##########
File path: connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/PluginUtilsTest.java
##########
@@ -114,68 +90,264 @@ public void testConnectFrameworkClasses() {
         assertFalse(PluginUtils.shouldLoadInIsolation(
                 "org.apache.kafka.clients.admin.KafkaAdminClient")
         );
-        assertFalse(PluginUtils.shouldLoadInIsolation(
-                "org.apache.kafka.connect.rest.ConnectRestExtension")
-        );
     }
 
     @Test
-    public void testAllowedConnectFrameworkClasses() {
-        assertTrue(PluginUtils.shouldLoadInIsolation("org.apache.kafka.connect.transforms."));
-        assertTrue(PluginUtils.shouldLoadInIsolation(
-                "org.apache.kafka.connect.transforms.ExtractField")
-        );
-        assertTrue(PluginUtils.shouldLoadInIsolation(
-                "org.apache.kafka.connect.transforms.ExtractField$Key")
-        );
-        assertTrue(PluginUtils.shouldLoadInIsolation("org.apache.kafka.connect.json."));
-        assertTrue(PluginUtils.shouldLoadInIsolation(
-                "org.apache.kafka.connect.json.JsonConverter")
-        );
-        assertTrue(PluginUtils.shouldLoadInIsolation(
-                "org.apache.kafka.connect.json.JsonConverter$21")
-        );
-        assertTrue(PluginUtils.shouldLoadInIsolation("org.apache.kafka.connect.file."));
-        assertTrue(PluginUtils.shouldLoadInIsolation(
-                "org.apache.kafka.connect.file.FileStreamSourceTask")
-        );
-        assertTrue(PluginUtils.shouldLoadInIsolation(
-                "org.apache.kafka.connect.file.FileStreamSinkConnector")
-        );
+    public void testConnectApiClasses() {
+        String[] apiClasses = new String[] {

Review comment:
       nit: do you mind using `List<String>` and `Arrays.asList(...)`? I don't think array declaration is better if the result is not going to be used as an array. Also, won't work if you try to reinitialize a declared variable. 

##########
File path: connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/PluginUtilsTest.java
##########
@@ -114,68 +90,264 @@ public void testConnectFrameworkClasses() {
         assertFalse(PluginUtils.shouldLoadInIsolation(
                 "org.apache.kafka.clients.admin.KafkaAdminClient")
         );
-        assertFalse(PluginUtils.shouldLoadInIsolation(
-                "org.apache.kafka.connect.rest.ConnectRestExtension")
-        );
     }
 
     @Test
-    public void testAllowedConnectFrameworkClasses() {
-        assertTrue(PluginUtils.shouldLoadInIsolation("org.apache.kafka.connect.transforms."));
-        assertTrue(PluginUtils.shouldLoadInIsolation(
-                "org.apache.kafka.connect.transforms.ExtractField")
-        );
-        assertTrue(PluginUtils.shouldLoadInIsolation(
-                "org.apache.kafka.connect.transforms.ExtractField$Key")
-        );
-        assertTrue(PluginUtils.shouldLoadInIsolation("org.apache.kafka.connect.json."));
-        assertTrue(PluginUtils.shouldLoadInIsolation(
-                "org.apache.kafka.connect.json.JsonConverter")
-        );
-        assertTrue(PluginUtils.shouldLoadInIsolation(
-                "org.apache.kafka.connect.json.JsonConverter$21")
-        );
-        assertTrue(PluginUtils.shouldLoadInIsolation("org.apache.kafka.connect.file."));
-        assertTrue(PluginUtils.shouldLoadInIsolation(
-                "org.apache.kafka.connect.file.FileStreamSourceTask")
-        );
-        assertTrue(PluginUtils.shouldLoadInIsolation(
-                "org.apache.kafka.connect.file.FileStreamSinkConnector")
-        );
+    public void testConnectApiClasses() {
+        String[] apiClasses = new String[] {
+            // Enumerate all packages and classes
+            "org.apache.kafka.connect.",
+            "org.apache.kafka.connect.components.",
+            "org.apache.kafka.connect.components.Versioned",
+            //"org.apache.kafka.connect.connector.policy.", isolated by default
+            "org.apache.kafka.connect.connector.policy.ConnectorClientConfigOverridePolicy",
+            "org.apache.kafka.connect.connector.policy.ConnectorClientConfigRequest",
+            "org.apache.kafka.connect.connector.",
+            "org.apache.kafka.connect.connector.Connector",
+            "org.apache.kafka.connect.connector.ConnectorContext",
+            "org.apache.kafka.connect.connector.ConnectRecord",
+            "org.apache.kafka.connect.connector.Task",
+            "org.apache.kafka.connect.data.",
+            "org.apache.kafka.connect.data.ConnectSchema",
+            "org.apache.kafka.connect.data.Date",
+            "org.apache.kafka.connect.data.Decimal",
+            "org.apache.kafka.connect.data.Field",
+            "org.apache.kafka.connect.data.Schema",
+            "org.apache.kafka.connect.data.SchemaAndValue",
+            "org.apache.kafka.connect.data.SchemaBuilder",
+            "org.apache.kafka.connect.data.SchemaProjector",
+            "org.apache.kafka.connect.data.Struct",
+            "org.apache.kafka.connect.data.Time",
+            "org.apache.kafka.connect.data.Timestamp",
+            "org.apache.kafka.connect.data.Values",
+            "org.apache.kafka.connect.errors.",
+            "org.apache.kafka.connect.errors.AlreadyExistsException",
+            "org.apache.kafka.connect.errors.ConnectException",
+            "org.apache.kafka.connect.errors.DataException",
+            "org.apache.kafka.connect.errors.IllegalWorkerStateException",
+            "org.apache.kafka.connect.errors.NotFoundException",
+            "org.apache.kafka.connect.errors.RetriableException",
+            "org.apache.kafka.connect.errors.SchemaBuilderException",
+            "org.apache.kafka.connect.errors.SchemaProjectorException",
+            "org.apache.kafka.connect.header.",
+            "org.apache.kafka.connect.header.ConnectHeader",
+            "org.apache.kafka.connect.header.ConnectHeaders",
+            "org.apache.kafka.connect.header.Header",
+            "org.apache.kafka.connect.header.Headers",
+            "org.apache.kafka.connect.health.",
+            "org.apache.kafka.connect.health.AbstractState",
+            "org.apache.kafka.connect.health.ConnectClusterDetails",
+            "org.apache.kafka.connect.health.ConnectClusterState",
+            "org.apache.kafka.connect.health.ConnectorHealth",
+            "org.apache.kafka.connect.health.ConnectorState",
+            "org.apache.kafka.connect.health.ConnectorType",
+            "org.apache.kafka.connect.health.TaskState",
+            "org.apache.kafka.connect.rest.",
+            "org.apache.kafka.connect.rest.ConnectRestExtension",
+            "org.apache.kafka.connect.rest.ConnectRestExtensionContext",
+            "org.apache.kafka.connect.sink.",
+            "org.apache.kafka.connect.sink.SinkConnector",
+            "org.apache.kafka.connect.sink.SinkRecord",
+            "org.apache.kafka.connect.sink.SinkTask",
+            "org.apache.kafka.connect.sink.SinkTaskContext",
+            "org.apache.kafka.connect.source.",
+            "org.apache.kafka.connect.source.SourceConnector",
+            "org.apache.kafka.connect.source.SourceRecord",
+            "org.apache.kafka.connect.source.SourceTask",
+            "org.apache.kafka.connect.source.SourceTaskContext",
+            "org.apache.kafka.connect.storage.",
+            "org.apache.kafka.connect.storage.Converter",
+            "org.apache.kafka.connect.storage.ConverterConfig",
+            "org.apache.kafka.connect.storage.ConverterType",
+            "org.apache.kafka.connect.storage.HeaderConverter",
+            "org.apache.kafka.connect.storage.OffsetStorageReader",
+            //"org.apache.kafka.connect.storage.SimpleHeaderConverter", explicitly isolated
+            //"org.apache.kafka.connect.storage.StringConverter", explicitly isolated
+            "org.apache.kafka.connect.storage.StringConverterConfig",
+            //"org.apache.kafka.connect.transforms.", isolated by default
+            "org.apache.kafka.connect.transforms.Transformation",
+            "org.apache.kafka.connect.util.",
+            "org.apache.kafka.connect.util.ConnectorUtils"
+        };
+        // Classes in the API should never be loaded in isolation.
+        for (String clazz : apiClasses) {
+            assertFalse(
+                clazz + " from 'api' is loaded in isolation but should not be",
+                PluginUtils.shouldLoadInIsolation(clazz)
+            );
+        }
+    }
+
+    @Test
+    public void testConnectRuntimeClasses() {
+        // Only list packages, because there are too many classes.
+        String[] runtimeClasses = new String[]{

Review comment:
       nit: same as above. 

##########
File path: connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/PluginUtilsTest.java
##########
@@ -114,68 +90,264 @@ public void testConnectFrameworkClasses() {
         assertFalse(PluginUtils.shouldLoadInIsolation(
                 "org.apache.kafka.clients.admin.KafkaAdminClient")
         );
-        assertFalse(PluginUtils.shouldLoadInIsolation(
-                "org.apache.kafka.connect.rest.ConnectRestExtension")
-        );
     }
 
     @Test
-    public void testAllowedConnectFrameworkClasses() {
-        assertTrue(PluginUtils.shouldLoadInIsolation("org.apache.kafka.connect.transforms."));
-        assertTrue(PluginUtils.shouldLoadInIsolation(
-                "org.apache.kafka.connect.transforms.ExtractField")
-        );
-        assertTrue(PluginUtils.shouldLoadInIsolation(
-                "org.apache.kafka.connect.transforms.ExtractField$Key")
-        );
-        assertTrue(PluginUtils.shouldLoadInIsolation("org.apache.kafka.connect.json."));
-        assertTrue(PluginUtils.shouldLoadInIsolation(
-                "org.apache.kafka.connect.json.JsonConverter")
-        );
-        assertTrue(PluginUtils.shouldLoadInIsolation(
-                "org.apache.kafka.connect.json.JsonConverter$21")
-        );
-        assertTrue(PluginUtils.shouldLoadInIsolation("org.apache.kafka.connect.file."));
-        assertTrue(PluginUtils.shouldLoadInIsolation(
-                "org.apache.kafka.connect.file.FileStreamSourceTask")
-        );
-        assertTrue(PluginUtils.shouldLoadInIsolation(
-                "org.apache.kafka.connect.file.FileStreamSinkConnector")
-        );
+    public void testConnectApiClasses() {
+        String[] apiClasses = new String[] {
+            // Enumerate all packages and classes
+            "org.apache.kafka.connect.",
+            "org.apache.kafka.connect.components.",
+            "org.apache.kafka.connect.components.Versioned",
+            //"org.apache.kafka.connect.connector.policy.", isolated by default
+            "org.apache.kafka.connect.connector.policy.ConnectorClientConfigOverridePolicy",
+            "org.apache.kafka.connect.connector.policy.ConnectorClientConfigRequest",
+            "org.apache.kafka.connect.connector.",
+            "org.apache.kafka.connect.connector.Connector",
+            "org.apache.kafka.connect.connector.ConnectorContext",
+            "org.apache.kafka.connect.connector.ConnectRecord",
+            "org.apache.kafka.connect.connector.Task",
+            "org.apache.kafka.connect.data.",
+            "org.apache.kafka.connect.data.ConnectSchema",
+            "org.apache.kafka.connect.data.Date",

Review comment:
       The exhaustive list of classes is unmaintainable and it's highly improbable that new classes will be removed or added. 
   
   Can we instead keep the existing testing based on packages, keep testing the exceptions and then test a few indicative classes that most probably won't be removed any time soon as a way to test the inclusion and exclusion from isolation?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org