morazow commented on code in PR #89:
URL: 
https://github.com/apache/flink-connector-kafka/pull/89#discussion_r1555308164


##########
flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSerializerWrapperTest.java:
##########
@@ -0,0 +1,63 @@
+package org.apache.flink.connector.kafka.sink;
+
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
+import org.apache.flink.util.FlinkUserCodeClassLoaders;
+import org.apache.flink.util.SimpleUserCodeClassLoader;
+import org.apache.flink.util.UserCodeClassLoader;
+
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.junit.Test;
+
+import java.net.URL;
+
+import static org.junit.Assert.assertEquals;
+
+/** Tests for {@link KafkaSerializerWrapper}. */
+public class KafkaSerializerWrapperTest {
+    @Test
+    public void testUserCodeClassLoaderIsUsed() throws Exception {
+        final KafkaSerializerWrapperCaptureForTest wrapper =
+                new KafkaSerializerWrapperCaptureForTest();
+        final ClassLoader classLoader =
+                FlinkUserCodeClassLoaders.childFirst(
+                        new URL[0],
+                        getClass().getClassLoader(),
+                        new String[0],
+                        throwable -> {},
+                        true);
+        wrapper.open(
+                new SerializationSchema.InitializationContext() {
+                    @Override
+                    public MetricGroup getMetricGroup() {
+                        return new UnregisteredMetricsGroup();
+                    }
+
+                    @Override
+                    public UserCodeClassLoader getUserCodeClassLoader() {
+                        return SimpleUserCodeClassLoader.create(classLoader);
+                    }
+                });
+
+        assertEquals(classLoader, wrapper.getClassLoaderUsed());
+    }

Review Comment:
   Could you please add another test case for previous classloader? With a 
descriptive test similar to yours, e.g, `testDefaultClassLoaderIsUsed`.
   
   In this case you can use the `DummyInitializationContext`.
   
   The test `KafkaRecordDeserializationSchemaTest` already tests this but the 
assertion is not on the classloader, so having default classloader assertion 
(e.g, `DummyInitializationContext.class.getClassLoader()`) make it clearer.
   



##########
flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSerializerWrapperTest.java:
##########
@@ -0,0 +1,63 @@
+package org.apache.flink.connector.kafka.sink;
+
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
+import org.apache.flink.util.FlinkUserCodeClassLoaders;
+import org.apache.flink.util.SimpleUserCodeClassLoader;
+import org.apache.flink.util.UserCodeClassLoader;
+
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.junit.Test;
+
+import java.net.URL;
+
+import static org.junit.Assert.assertEquals;
+
+/** Tests for {@link KafkaSerializerWrapper}. */
+public class KafkaSerializerWrapperTest {
+    @Test
+    public void testUserCodeClassLoaderIsUsed() throws Exception {
+        final KafkaSerializerWrapperCaptureForTest wrapper =
+                new KafkaSerializerWrapperCaptureForTest();
+        final ClassLoader classLoader =
+                FlinkUserCodeClassLoaders.childFirst(
+                        new URL[0],
+                        getClass().getClassLoader(),
+                        new String[0],
+                        throwable -> {},
+                        true);
+        wrapper.open(
+                new SerializationSchema.InitializationContext() {
+                    @Override
+                    public MetricGroup getMetricGroup() {
+                        return new UnregisteredMetricsGroup();
+                    }
+
+                    @Override
+                    public UserCodeClassLoader getUserCodeClassLoader() {
+                        return SimpleUserCodeClassLoader.create(classLoader);
+                    }
+                });
+
+        assertEquals(classLoader, wrapper.getClassLoaderUsed());
+    }

Review Comment:
   Same comment for the deserialization test below



##########
flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaValueOnlyDeserializerWrapper.java:
##########
@@ -55,17 +55,11 @@ class KafkaValueOnlyDeserializerWrapper<T> implements 
KafkaRecordDeserialization
     }
 
     @Override
-    @SuppressWarnings("unchecked")
     public void open(DeserializationSchema.InitializationContext context) 
throws Exception {
-        ClassLoader userCodeClassLoader = 
context.getUserCodeClassLoader().asClassLoader();
+        ClassLoader userCodeClassLoader = selectClassLoader(context);

Review Comment:
   Same here let's skip the method



##########
flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSerializerWrapper.java:
##########
@@ -61,17 +61,12 @@ class KafkaSerializerWrapper<IN> implements 
SerializationSchema<IN> {
         this(serializerClass, isKey, Collections.emptyMap(), topicSelector);
     }
 
-    @SuppressWarnings("unchecked")
     @Override
     public void open(InitializationContext context) throws Exception {
-        final ClassLoader userCodeClassLoader = 
context.getUserCodeClassLoader().asClassLoader();
+        final ClassLoader userCodeClassLoader = selectClassLoader(context);

Review Comment:
   Since the `selectClassLoader` is not used in the tests, maybe we don't have 
to extract it into a method



-- 
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.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to