-
Notifications
You must be signed in to change notification settings - Fork 140
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-34996][Connectors/Kafka] Allow custom Serializer/Deserializer …
…initialization and remove mockito.
- Loading branch information
Showing
6 changed files
with
131 additions
and
108 deletions.
There are no files selected for viewing
This file contains 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 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
62 changes: 46 additions & 16 deletions
62
...kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSerializerWrapperTest.java
This file contains 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 |
---|---|---|
@@ -1,28 +1,58 @@ | ||
package org.apache.flink.connector.kafka.sink; | ||
|
||
import org.apache.flink.streaming.connectors.kafka.testutils.SerializationTestBase; | ||
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 org.junit.runner.RunWith; | ||
import org.mockito.junit.MockitoJUnitRunner; | ||
|
||
import static org.mockito.Mockito.when; | ||
import java.net.URL; | ||
|
||
@RunWith(MockitoJUnitRunner.class) | ||
public class KafkaSerializerWrapperTest extends SerializationTestBase { | ||
@Override | ||
protected void setupContext() { | ||
when(serializationContext.getUserCodeClassLoader()).thenReturn(userCodeClassLoader); | ||
} | ||
import static org.junit.Assert.assertEquals; | ||
|
||
/** | ||
* Tests for {@link KafkaSerializerWrapper}. | ||
*/ | ||
public class KafkaSerializerWrapperTest { | ||
@Test | ||
public void testUserCodeClassLoaderIsUsed() throws Exception { | ||
final KafkaSerializerWrapper<String> wrapper = | ||
new KafkaSerializerWrapper<>(StringSerializer.class, true, (value) -> "topic"); | ||
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()); | ||
} | ||
|
||
static class KafkaSerializerWrapperCaptureForTest extends KafkaSerializerWrapper<String> { | ||
private ClassLoader classLoaderUsed; | ||
|
||
KafkaSerializerWrapperCaptureForTest() { | ||
super(StringSerializer.class, true, (value) -> "topic"); | ||
} | ||
|
||
public ClassLoader getClassLoaderUsed() { | ||
return classLoaderUsed; | ||
} | ||
|
||
testUserClassLoaderIsUsedWhen(() -> { | ||
wrapper.open(serializationContext); | ||
return null; | ||
}, new StringSerializer()); | ||
@Override | ||
protected void initializeSerializer(ClassLoader classLoader) throws Exception { | ||
classLoaderUsed = classLoader; | ||
super.initializeSerializer(classLoader); | ||
} | ||
} | ||
} |
68 changes: 48 additions & 20 deletions
68
...ink/connector/kafka/source/reader/deserializer/KafkaValueOnlyDeserializerWrapperTest.java
This file contains 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 |
---|---|---|
@@ -1,32 +1,60 @@ | ||
package org.apache.flink.connector.kafka.source.reader.deserializer; | ||
|
||
import org.apache.flink.streaming.connectors.kafka.testutils.SerializationTestBase; | ||
import org.apache.flink.api.common.serialization.DeserializationSchema; | ||
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 org.junit.runner.RunWith; | ||
import org.mockito.junit.MockitoJUnitRunner; | ||
|
||
import java.net.URL; | ||
import java.util.HashMap; | ||
import java.util.Map; | ||
|
||
import static org.mockito.Mockito.when; | ||
|
||
@RunWith(MockitoJUnitRunner.class) | ||
public class KafkaValueOnlyDeserializerWrapperTest extends SerializationTestBase { | ||
@Override | ||
protected void setupContext() { | ||
when(deserializationContext.getUserCodeClassLoader()).thenReturn(userCodeClassLoader); | ||
} | ||
import static org.junit.Assert.assertEquals; | ||
|
||
/** | ||
* Tests for {@link KafkaValueOnlyDeserializerWrapper}. | ||
*/ | ||
public class KafkaValueOnlyDeserializerWrapperTest { | ||
@Test | ||
public void testUserCodeClassLoaderIsUsed() throws Exception { | ||
final Map<String, String> config = new HashMap<>(); | ||
final KafkaValueOnlyDeserializerWrapper<String> wrapper = | ||
new KafkaValueOnlyDeserializerWrapper<>(StringDeserializer.class, config); | ||
|
||
testUserClassLoaderIsUsedWhen(() -> { | ||
wrapper.open(deserializationContext); | ||
return null; | ||
}, new StringDeserializer()); | ||
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()); | ||
} | ||
|
||
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); | ||
} | ||
} | ||
} |
58 changes: 0 additions & 58 deletions
58
...est/java/org/apache/flink/streaming/connectors/kafka/testutils/SerializationTestBase.java
This file was deleted.
Oops, something went wrong.
1 change: 0 additions & 1 deletion
1
flink-connector-kafka/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker
This file was deleted.
Oops, something went wrong.