forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-18075] Wrap the SerializationSchema in KafkaSerializationSchem…
…a in Kafka connector
- Loading branch information
Showing
14 changed files
with
479 additions
and
58 deletions.
There are no files selected for viewing
72 changes: 72 additions & 0 deletions
72
....10/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerTest.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 |
---|---|---|
@@ -0,0 +1,72 @@ | ||
/* | ||
* 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:https://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.streaming.connectors.kafka; | ||
|
||
import org.apache.flink.api.common.serialization.SerializationSchema; | ||
import org.apache.flink.api.common.typeutils.base.IntSerializer; | ||
import org.apache.flink.runtime.jobgraph.OperatorID; | ||
import org.apache.flink.streaming.api.operators.StreamSink; | ||
import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; | ||
|
||
import org.junit.Test; | ||
|
||
import static org.hamcrest.CoreMatchers.equalTo; | ||
import static org.junit.Assert.assertThat; | ||
|
||
/** | ||
* Tests for {@link FlinkKafkaProducer010}. | ||
*/ | ||
public class FlinkKafkaProducerTest { | ||
@Test | ||
public void testOpenProducer() throws Exception { | ||
|
||
OpenTestingSerializationSchema schema = new OpenTestingSerializationSchema(); | ||
FlinkKafkaProducer010<Integer> kafkaProducer = new FlinkKafkaProducer010<>( | ||
"localhost:9092", | ||
"test-topic", | ||
schema | ||
); | ||
|
||
OneInputStreamOperatorTestHarness<Integer, Object> testHarness = new OneInputStreamOperatorTestHarness<>( | ||
new StreamSink<>(kafkaProducer), | ||
1, | ||
1, | ||
0, | ||
IntSerializer.INSTANCE, | ||
new OperatorID(1, 1)); | ||
|
||
testHarness.open(); | ||
|
||
assertThat(schema.openCalled, equalTo(true)); | ||
} | ||
|
||
private static class OpenTestingSerializationSchema implements SerializationSchema<Integer> { | ||
private boolean openCalled; | ||
|
||
@Override | ||
public void open(InitializationContext context) throws Exception { | ||
openCalled = true; | ||
} | ||
|
||
@Override | ||
public byte[] serialize(Integer element) { | ||
return new byte[0]; | ||
} | ||
} | ||
} |
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
72 changes: 72 additions & 0 deletions
72
....11/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerTest.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 |
---|---|---|
@@ -0,0 +1,72 @@ | ||
/* | ||
* 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:https://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.streaming.connectors.kafka; | ||
|
||
import org.apache.flink.api.common.serialization.SerializationSchema; | ||
import org.apache.flink.api.common.typeutils.base.IntSerializer; | ||
import org.apache.flink.runtime.jobgraph.OperatorID; | ||
import org.apache.flink.streaming.api.operators.StreamSink; | ||
import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; | ||
|
||
import org.junit.Test; | ||
|
||
import static org.hamcrest.CoreMatchers.equalTo; | ||
import static org.junit.Assert.assertThat; | ||
|
||
/** | ||
* Tests for {@link FlinkKafkaProducer011}. | ||
*/ | ||
public class FlinkKafkaProducerTest { | ||
@Test | ||
public void testOpenProducer() throws Exception { | ||
|
||
OpenTestingSerializationSchema schema = new OpenTestingSerializationSchema(); | ||
FlinkKafkaProducer011<Integer> kafkaProducer = new FlinkKafkaProducer011<>( | ||
"localhost:9092", | ||
"test-topic", | ||
schema | ||
); | ||
|
||
OneInputStreamOperatorTestHarness<Integer, Object> testHarness = new OneInputStreamOperatorTestHarness<>( | ||
new StreamSink<>(kafkaProducer), | ||
1, | ||
1, | ||
0, | ||
IntSerializer.INSTANCE, | ||
new OperatorID(1, 1)); | ||
|
||
testHarness.open(); | ||
|
||
assertThat(schema.openCalled, equalTo(true)); | ||
} | ||
|
||
private static class OpenTestingSerializationSchema implements SerializationSchema<Integer> { | ||
private boolean openCalled; | ||
|
||
@Override | ||
public void open(InitializationContext context) throws Exception { | ||
openCalled = true; | ||
} | ||
|
||
@Override | ||
public byte[] serialize(Integer element) { | ||
return new byte[0]; | ||
} | ||
} | ||
} |
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
95 changes: 95 additions & 0 deletions
95
...rg/apache/flink/streaming/connectors/kafka/internals/KafkaSerializationSchemaWrapper.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 |
---|---|---|
@@ -0,0 +1,95 @@ | ||
/* | ||
* 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:https://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.streaming.connectors.kafka.internals; | ||
|
||
import org.apache.flink.api.common.serialization.SerializationSchema; | ||
import org.apache.flink.streaming.connectors.kafka.KafkaContextAware; | ||
import org.apache.flink.streaming.connectors.kafka.KafkaSerializationSchema; | ||
import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; | ||
|
||
import org.apache.kafka.clients.producer.ProducerRecord; | ||
|
||
import javax.annotation.Nullable; | ||
|
||
/** | ||
* An adapter from old style interfaces such as {@link org.apache.flink.api.common.serialization.SerializationSchema}, | ||
* {@link org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner} to the | ||
* {@link KafkaSerializationSchema}. | ||
*/ | ||
public class KafkaSerializationSchemaWrapper<T> implements KafkaSerializationSchema<T>, KafkaContextAware<T> { | ||
|
||
private final FlinkKafkaPartitioner<T> partitioner; | ||
private final SerializationSchema<T> serializationSchema; | ||
private final String topic; | ||
private boolean writeTimestamp; | ||
|
||
private int[] partitions; | ||
|
||
public KafkaSerializationSchemaWrapper( | ||
String topic, | ||
FlinkKafkaPartitioner<T> partitioner, | ||
boolean writeTimestamp, | ||
SerializationSchema<T> serializationSchema) { | ||
this.partitioner = partitioner; | ||
this.serializationSchema = serializationSchema; | ||
this.topic = topic; | ||
this.writeTimestamp = writeTimestamp; | ||
} | ||
|
||
@Override | ||
public void open(SerializationSchema.InitializationContext context) throws Exception { | ||
serializationSchema.open(context); | ||
} | ||
|
||
@Override | ||
public ProducerRecord<byte[], byte[]> serialize( | ||
T element, | ||
@Nullable Long timestamp) { | ||
byte[] serialized = serializationSchema.serialize(element); | ||
final Integer partition; | ||
if (partitioner != null) { | ||
partition = partitioner.partition(element, null, serialized, topic, partitions); | ||
} else { | ||
partition = null; | ||
} | ||
|
||
final Long timestampToWrite; | ||
if (writeTimestamp) { | ||
timestampToWrite = timestamp; | ||
} else { | ||
timestampToWrite = null; | ||
} | ||
|
||
return new ProducerRecord<>(topic, partition, timestampToWrite, null, serialized); | ||
} | ||
|
||
@Override | ||
public String getTargetTopic(T element) { | ||
return topic; | ||
} | ||
|
||
@Override | ||
public void setPartitions(int[] partitions) { | ||
this.partitions = partitions; | ||
} | ||
|
||
public void setWriteTimestamp(boolean writeTimestamp) { | ||
this.writeTimestamp = writeTimestamp; | ||
} | ||
} |
Oops, something went wrong.