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-8014] [table] Add Kafka010JsonTableSink.
- Refactor KafkaTableSink tests.
- Loading branch information
Showing
11 changed files
with
269 additions
and
76 deletions.
There are no files selected for viewing
73 changes: 73 additions & 0 deletions
73
...0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSink.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,73 @@ | ||
/* | ||
* 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.streaming.connectors.kafka; | ||
|
||
import org.apache.flink.api.common.serialization.SerializationSchema; | ||
import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; | ||
import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; | ||
import org.apache.flink.types.Row; | ||
|
||
import java.util.Properties; | ||
|
||
/** | ||
* Kafka 0.10 {@link KafkaTableSink} that serializes data in JSON format. | ||
*/ | ||
public class Kafka010JsonTableSink extends KafkaJsonTableSink { | ||
|
||
/** | ||
* Creates {@link KafkaTableSink} to write table rows as JSON-encoded records to a Kafka 0.10 | ||
* topic with fixed partition assignment. | ||
* | ||
* <p>Each parallel TableSink instance will write its rows to a single Kafka partition.</p> | ||
* <ul> | ||
* <li>If the number of Kafka partitions is less than the number of sink instances, different | ||
* sink instances will write to the same partition.</li> | ||
* <li>If the number of Kafka partitions is higher than the number of sink instance, some | ||
* Kafka partitions won't receive data.</li> | ||
* </ul> | ||
* | ||
* @param topic topic in Kafka to which table is written | ||
* @param properties properties to connect to Kafka | ||
*/ | ||
public Kafka010JsonTableSink(String topic, Properties properties) { | ||
super(topic, properties, new FlinkFixedPartitioner<>()); | ||
} | ||
|
||
/** | ||
* Creates {@link KafkaTableSink} to write table rows as JSON-encoded records to a Kafka 0.10 | ||
* topic with custom partition assignment. | ||
* | ||
* @param topic topic in Kafka to which table is written | ||
* @param properties properties to connect to Kafka | ||
* @param partitioner Kafka partitioner | ||
*/ | ||
public Kafka010JsonTableSink(String topic, Properties properties, FlinkKafkaPartitioner<Row> partitioner) { | ||
super(topic, properties, partitioner); | ||
} | ||
|
||
@Override | ||
protected FlinkKafkaProducerBase<Row> createKafkaProducer(String topic, Properties properties, SerializationSchema<Row> serializationSchema, FlinkKafkaPartitioner<Row> partitioner) { | ||
return new FlinkKafkaProducer010<>(topic, serializationSchema, properties, partitioner); | ||
} | ||
|
||
@Override | ||
protected Kafka010JsonTableSink createCopy() { | ||
return new Kafka010JsonTableSink(topic, properties, partitioner); | ||
} | ||
} |
53 changes: 53 additions & 0 deletions
53
.../src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSinkTest.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,53 @@ | ||
/* | ||
* 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.streaming.connectors.kafka; | ||
|
||
import org.apache.flink.api.common.serialization.SerializationSchema; | ||
import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; | ||
import org.apache.flink.streaming.util.serialization.JsonRowSerializationSchema; | ||
import org.apache.flink.types.Row; | ||
|
||
import java.util.Properties; | ||
|
||
/** | ||
* Tests for the {@link Kafka010JsonTableSink}. | ||
*/ | ||
public class Kafka010JsonTableSinkTest extends KafkaTableSinkTestBase { | ||
|
||
@Override | ||
protected KafkaTableSink createTableSink( | ||
String topic, | ||
Properties properties, | ||
FlinkKafkaPartitioner<Row> partitioner) { | ||
|
||
return new Kafka010JsonTableSink(topic, properties, partitioner); | ||
} | ||
|
||
@Override | ||
protected Class<? extends SerializationSchema<Row>> getSerializationSchemaClass() { | ||
return JsonRowSerializationSchema.class; | ||
} | ||
|
||
@Override | ||
protected Class<? extends FlinkKafkaProducerBase> getProducerClass() { | ||
return FlinkKafkaProducer010.class; | ||
} | ||
|
||
} | ||
|
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
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
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
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
Oops, something went wrong.