StephanEwen commented on a change in pull request #14303:
URL: https://github.com/apache/flink/pull/14303#discussion_r535517311



##########
File path: 
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaDeserializationSchemaWrapper.java
##########
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.kafka.source.reader.deserializer;
+
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema;
+import org.apache.flink.util.Collector;
+
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+
+import java.io.IOException;
+
+/**
+ * A wrapper class that wraps a {@link 
org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema}
+ * to deserialize {@link ConsumerRecord ConsumerRecords}.
+ *
+ * @param <T> the type of the deserialized records.
+ */
+class KafkaDeserializationSchemaWrapper<T> implements 
KafkaRecordDeserializer<T> {
+       private static final long serialVersionUID = 3239435655135705790L;

Review comment:
       Per the style guide, please start the serialVersionUID at 1L.
   
https://flink.apache.org/contributing/code-style-and-quality-java.html#java-serialization

##########
File path: 
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaValueOnlyDeserializationSchemaWrapper.java
##########
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.kafka.source.reader.deserializer;
+
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.util.Collector;
+
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+
+import java.io.IOException;
+
+/**
+ * A class that wraps a {@link DeserializationSchema} as the value 
deserializer for a
+ * {@link ConsumerRecord}.
+ *
+ * @param <T> the return type of the deserialization.
+ */
+class KafkaValueOnlyDeserializationSchemaWrapper<T> implements 
KafkaRecordDeserializer<T> {
+       private static final long serialVersionUID = -3962448817248263667L;

Review comment:
       `serialVersionUID` should start at 1L, see above.

##########
File path: 
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaValueOnlyDeserializerWrapper.java
##########
@@ -18,45 +18,59 @@
 
 package org.apache.flink.connector.kafka.source.reader.deserializer;
 
+import org.apache.flink.api.common.serialization.DeserializationSchema;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.typeutils.TypeExtractor;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.InstantiationUtil;
+import org.apache.flink.util.TemporaryClassLoaderContext;
 
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.common.Configurable;
 import org.apache.kafka.common.serialization.Deserializer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
 import java.util.Map;
 
 /**
  * A package private class to wrap {@link Deserializer}.
  */
-class ValueDeserializerWrapper<T> implements KafkaRecordDeserializer<T> {
+class KafkaValueOnlyDeserializerWrapper<T> implements 
KafkaRecordDeserializer<T> {
        private static final long serialVersionUID = 5409547407386004054L;
-       private static final Logger LOG = 
LoggerFactory.getLogger(ValueDeserializerWrapper.class);
+       private static final Logger LOG = 
LoggerFactory.getLogger(KafkaValueOnlyDeserializerWrapper.class);
        private final String deserializerClass;
        private final Map<String, String> config;
+       private ClassLoader userCodeClassLoader;
 
        private transient Deserializer<T> deserializer;
 
-       ValueDeserializerWrapper(
+       KafkaValueOnlyDeserializerWrapper(
                        Class<? extends Deserializer<T>> deserializerClass,
                        Map<String, String> config) {
                this.deserializerClass = deserializerClass.getName();
                this.config = config;
        }
 
+       @Override
+       public void open(DeserializationSchema.InitializationContext context) 
throws Exception {
+               this.userCodeClassLoader = 
context.getUserCodeClassLoader().asClassLoader();
+       }
+
        @Override
        @SuppressWarnings("unchecked")
-       public void deserialize(ConsumerRecord<byte[], byte[]> record, 
Collector<T> collector) throws Exception {
+       public void deserialize(ConsumerRecord<byte[], byte[]> record, 
Collector<T> collector) throws IOException {
                if (deserializer == null) {
-                       deserializer = (Deserializer<T>) 
InstantiationUtil.instantiate(
+                       try (TemporaryClassLoaderContext ignored = 
TemporaryClassLoaderContext.of(userCodeClassLoader)) {

Review comment:
       Could this initialization be moved into the open method?
   If not, could be good to move the init logic to separate method, to keep the 
main method code paths small, for JIT friendlyness. 

##########
File path: 
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaValueOnlyDeserializerWrapper.java
##########
@@ -18,45 +18,59 @@
 
 package org.apache.flink.connector.kafka.source.reader.deserializer;
 
+import org.apache.flink.api.common.serialization.DeserializationSchema;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.typeutils.TypeExtractor;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.InstantiationUtil;
+import org.apache.flink.util.TemporaryClassLoaderContext;
 
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.common.Configurable;
 import org.apache.kafka.common.serialization.Deserializer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
 import java.util.Map;
 
 /**
  * A package private class to wrap {@link Deserializer}.
  */
-class ValueDeserializerWrapper<T> implements KafkaRecordDeserializer<T> {
+class KafkaValueOnlyDeserializerWrapper<T> implements 
KafkaRecordDeserializer<T> {
        private static final long serialVersionUID = 5409547407386004054L;

Review comment:
       `serialVersionUID` should start at 1L, see above.

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SourceOperator.java
##########
@@ -191,6 +194,11 @@ public void sendSplitRequest() {
                        public void sendSourceEventToCoordinator(SourceEvent 
event) {
                                operatorEventGateway.sendEventToCoordinator(new 
SourceEventWrapper(event));
                        }
+
+                       @Override
+                       public UserCodeClassLoader getUserCodeClassLoader() {
+                               return (UserCodeClassLoader) 
getRuntimeContext().getUserCodeClassLoader();

Review comment:
       You can probably reuse some parts from 
`RuntimeContextInitializationContextAdapters` here.

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SourceOperator.java
##########
@@ -191,6 +194,11 @@ public void sendSplitRequest() {
                        public void sendSourceEventToCoordinator(SourceEvent 
event) {
                                operatorEventGateway.sendEventToCoordinator(new 
SourceEventWrapper(event));
                        }
+
+                       @Override
+                       public UserCodeClassLoader getUserCodeClassLoader() {
+                               return (UserCodeClassLoader) 
getRuntimeContext().getUserCodeClassLoader();

Review comment:
       This looks like a cast between incompatible types.

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SourceOperator.java
##########
@@ -197,7 +195,20 @@ public void sendSourceEventToCoordinator(SourceEvent 
event) {
 
                        @Override
                        public UserCodeClassLoader getUserCodeClassLoader() {
-                               return (UserCodeClassLoader) 
getRuntimeContext().getUserCodeClassLoader();
+                               return new UserCodeClassLoader() {

Review comment:
       Is it possible to reuse code from 
`RuntimeContextInitializationContextAdapters` here?




----------------------------------------------------------------
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:
us...@infra.apache.org


Reply via email to