-
Notifications
You must be signed in to change notification settings - Fork 159
[FLINK-34996][Connectors/Kafka] Use UserCodeCL to instantiate Deserializer #89
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Open
hugogu
wants to merge
4
commits into
apache:main
Choose a base branch
from
hugogu:deserializer-fix
base: main
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Open
Changes from all commits
Commits
Show all changes
4 commits
Select commit
Hold shift + click to select a range
b496437
[FLINK-34996][Connectors/Kafka] Use UserCodeClassLoader to instantiat…
hugogu b4e0478
[FLINK-34996][Connectors/Kafka] Test proper class loader is used and …
hugogu 6a5b919
[FLINK-34996][Connectors/Kafka] Allow custom Serializer/Deserializer …
hugogu b288080
[FLINK-34996][Connectors/Kafka] Simplify code and add more tests on s…
hugogu File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
74 changes: 74 additions & 0 deletions
74
...kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSerializerWrapperTest.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,74 @@ | ||
package org.apache.flink.connector.kafka.sink; | ||
|
||
import org.apache.flink.api.common.serialization.SerializationSchema; | ||
import org.apache.flink.connector.testutils.formats.DummyInitializationContext; | ||
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()); | ||
} | ||
|
||
@Test | ||
public void testDefaultClassLoaderIsUsed() throws Exception { | ||
final KafkaSerializerWrapperCaptureForTest wrapper = | ||
new KafkaSerializerWrapperCaptureForTest(); | ||
wrapper.open(new DummyInitializationContext()); | ||
|
||
assertEquals( | ||
DummyInitializationContext.class.getClassLoader(), wrapper.getClassLoaderUsed()); | ||
} | ||
|
||
static class KafkaSerializerWrapperCaptureForTest extends KafkaSerializerWrapper<String> { | ||
private ClassLoader classLoaderUsed; | ||
|
||
KafkaSerializerWrapperCaptureForTest() { | ||
super(StringSerializer.class, true, (value) -> "topic"); | ||
} | ||
|
||
public ClassLoader getClassLoaderUsed() { | ||
return classLoaderUsed; | ||
} | ||
|
||
@Override | ||
protected void initializeSerializer(ClassLoader classLoader) throws Exception { | ||
classLoaderUsed = classLoader; | ||
super.initializeSerializer(classLoader); | ||
} | ||
} | ||
} |
76 changes: 76 additions & 0 deletions
76
...ink/connector/kafka/source/reader/deserializer/KafkaValueOnlyDeserializerWrapperTest.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,76 @@ | ||
package org.apache.flink.connector.kafka.source.reader.deserializer; | ||
|
||
import org.apache.flink.api.common.serialization.DeserializationSchema; | ||
import org.apache.flink.connector.testutils.formats.DummyInitializationContext; | ||
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.StringDeserializer; | ||
import org.junit.Test; | ||
|
||
import java.net.URL; | ||
import java.util.HashMap; | ||
|
||
import static org.junit.Assert.assertEquals; | ||
|
||
/** Tests for {@link KafkaValueOnlyDeserializerWrapper}. */ | ||
public class KafkaValueOnlyDeserializerWrapperTest { | ||
@Test | ||
public void testUserCodeClassLoaderIsUsed() throws Exception { | ||
final KafkaValueOnlyDeserializerWrapperCaptureForTest wrapper = | ||
new KafkaValueOnlyDeserializerWrapperCaptureForTest(); | ||
final ClassLoader classLoader = | ||
FlinkUserCodeClassLoaders.childFirst( | ||
new URL[0], | ||
getClass().getClassLoader(), | ||
new String[0], | ||
throwable -> {}, | ||
true); | ||
wrapper.open( | ||
new DeserializationSchema.InitializationContext() { | ||
@Override | ||
public MetricGroup getMetricGroup() { | ||
return new UnregisteredMetricsGroup(); | ||
} | ||
|
||
@Override | ||
public UserCodeClassLoader getUserCodeClassLoader() { | ||
return SimpleUserCodeClassLoader.create(classLoader); | ||
} | ||
}); | ||
|
||
assertEquals(classLoader, wrapper.getClassLoaderUsed()); | ||
} | ||
|
||
@Test | ||
public void testDefaultClassLoaderIsUsed() throws Exception { | ||
final KafkaValueOnlyDeserializerWrapperCaptureForTest wrapper = | ||
new KafkaValueOnlyDeserializerWrapperCaptureForTest(); | ||
wrapper.open(new DummyInitializationContext()); | ||
|
||
assertEquals( | ||
DummyInitializationContext.class.getClassLoader(), wrapper.getClassLoaderUsed()); | ||
} | ||
|
||
static class KafkaValueOnlyDeserializerWrapperCaptureForTest | ||
extends KafkaValueOnlyDeserializerWrapper<String> { | ||
private ClassLoader classLoaderUsed; | ||
|
||
KafkaValueOnlyDeserializerWrapperCaptureForTest() { | ||
super(StringDeserializer.class, new HashMap<>()); | ||
} | ||
|
||
public ClassLoader getClassLoaderUsed() { | ||
return classLoaderUsed; | ||
} | ||
|
||
@Override | ||
protected void initializeDeserializer(ClassLoader classLoader) throws Exception { | ||
classLoaderUsed = classLoader; | ||
super.initializeDeserializer(classLoader); | ||
} | ||
} | ||
} |
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.