[HUDI-1418] Set up flink client unit test infra (#2281)
This commit is contained in:
@@ -19,9 +19,11 @@
|
||||
package org.apache.hudi;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.operator.InstantGenerateOperator;
|
||||
import org.apache.hudi.operator.KeyedWriteProcessFunction;
|
||||
import org.apache.hudi.operator.KeyedWriteProcessOperator;
|
||||
@@ -42,11 +44,11 @@ import org.apache.flink.runtime.state.filesystem.FsStateBackend;
|
||||
import org.apache.flink.streaming.api.datastream.DataStream;
|
||||
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
|
||||
import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
|
||||
import org.apache.kafka.clients.consumer.ConsumerConfig;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* An Utility which can incrementally consume data from Kafka and apply it to the target table.
|
||||
@@ -73,13 +75,21 @@ public class HoodieFlinkStreamer {
|
||||
env.setStateBackend(new FsStateBackend(cfg.flinkCheckPointPath));
|
||||
}
|
||||
|
||||
Properties kafkaProps = StreamerUtil.getKafkaProps(cfg);
|
||||
TypedProperties props = StreamerUtil.getProps(cfg);
|
||||
|
||||
// add kafka config
|
||||
props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, cfg.kafkaBootstrapServers);
|
||||
props.put(ConsumerConfig.GROUP_ID_CONFIG, cfg.kafkaGroupId);
|
||||
|
||||
// add data source config
|
||||
props.put(HoodieWriteConfig.WRITE_PAYLOAD_CLASS, cfg.payloadClassName);
|
||||
props.put(HoodieWriteConfig.PRECOMBINE_FIELD_PROP, cfg.sourceOrderingField);
|
||||
|
||||
// Read from kafka source
|
||||
DataStream<HoodieRecord> inputRecords =
|
||||
env.addSource(new FlinkKafkaConsumer<>(cfg.kafkaTopic, new SimpleStringSchema(), kafkaProps))
|
||||
env.addSource(new FlinkKafkaConsumer<>(cfg.kafkaTopic, new SimpleStringSchema(), props))
|
||||
.filter(Objects::nonNull)
|
||||
.map(new JsonStringToHoodieRecordMapFunction(cfg))
|
||||
.map(new JsonStringToHoodieRecordMapFunction(props))
|
||||
.name("kafka_to_hudi_record")
|
||||
.uid("kafka_to_hudi_record_uid");
|
||||
|
||||
|
||||
@@ -18,18 +18,21 @@
|
||||
|
||||
package org.apache.hudi.source;
|
||||
|
||||
import org.apache.hudi.HoodieFlinkStreamer;
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieFlinkStreamerException;
|
||||
import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.keygen.KeyGenerator;
|
||||
import org.apache.hudi.keygen.SimpleAvroKeyGenerator;
|
||||
import org.apache.hudi.schema.FilebasedSchemaProvider;
|
||||
import org.apache.hudi.util.AvroConvertor;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.flink.api.common.functions.MapFunction;
|
||||
|
||||
@@ -40,32 +43,46 @@ import java.io.IOException;
|
||||
*/
|
||||
public class JsonStringToHoodieRecordMapFunction implements MapFunction<String, HoodieRecord> {
|
||||
|
||||
private final HoodieFlinkStreamer.Config cfg;
|
||||
private TypedProperties props;
|
||||
private KeyGenerator keyGenerator;
|
||||
private AvroConvertor avroConvertor;
|
||||
private Option<String> schemaStr;
|
||||
private String payloadClassName;
|
||||
private String orderingField;
|
||||
|
||||
public JsonStringToHoodieRecordMapFunction(HoodieFlinkStreamer.Config cfg) {
|
||||
this.cfg = cfg;
|
||||
public JsonStringToHoodieRecordMapFunction(TypedProperties props) {
|
||||
this(props, Option.empty());
|
||||
}
|
||||
|
||||
public JsonStringToHoodieRecordMapFunction(TypedProperties props, Option<String> schemaStr) {
|
||||
this.props = props;
|
||||
this.schemaStr = schemaStr;
|
||||
init();
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieRecord map(String value) throws Exception {
|
||||
GenericRecord gr = avroConvertor.fromJson(value);
|
||||
HoodieRecordPayload payload = StreamerUtil.createPayload(cfg.payloadClassName, gr,
|
||||
(Comparable) HoodieAvroUtils.getNestedFieldVal(gr, cfg.sourceOrderingField, false));
|
||||
GenericRecord gr = this.avroConvertor.fromJson(value);
|
||||
HoodieRecordPayload payload = StreamerUtil.createPayload(this.payloadClassName, gr,
|
||||
(Comparable) HoodieAvroUtils.getNestedFieldVal(gr, this.orderingField, false));
|
||||
|
||||
return new HoodieRecord<>(keyGenerator.getKey(gr), payload);
|
||||
return new HoodieRecord<>(this.keyGenerator.getKey(gr), payload);
|
||||
}
|
||||
|
||||
private void init() {
|
||||
TypedProperties props = StreamerUtil.getProps(cfg);
|
||||
avroConvertor = new AvroConvertor(new FilebasedSchemaProvider(props).getSourceSchema());
|
||||
if (schemaStr.isPresent()) {
|
||||
this.avroConvertor = new AvroConvertor(new Schema.Parser().parse(schemaStr.get()));
|
||||
} else {
|
||||
this.avroConvertor = new AvroConvertor(new FilebasedSchemaProvider(props).getSourceSchema());
|
||||
}
|
||||
this.payloadClassName = props.getString(HoodieWriteConfig.WRITE_PAYLOAD_CLASS,
|
||||
OverwriteWithLatestAvroPayload.class.getName());
|
||||
this.orderingField = props.getString(HoodieWriteConfig.PRECOMBINE_FIELD_PROP, "ts");
|
||||
try {
|
||||
keyGenerator = StreamerUtil.createKeyGenerator(props);
|
||||
this.keyGenerator = StreamerUtil.createKeyGenerator(props);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieFlinkStreamerException(String.format("KeyGenerator %s initialization failed",
|
||||
props.getString("hoodie.datasource.write.keygenerator.class", SimpleAvroKeyGenerator.class.getName())), e);
|
||||
props.getString(HoodieWriteConfig.KEYGENERATOR_CLASS_PROP, SimpleAvroKeyGenerator.class.getName())), e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -38,7 +38,6 @@ import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.kafka.clients.consumer.ConsumerConfig;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
@@ -46,19 +45,11 @@ import java.io.BufferedReader;
|
||||
import java.io.IOException;
|
||||
import java.io.StringReader;
|
||||
import java.util.List;
|
||||
import java.util.Properties;
|
||||
|
||||
public class StreamerUtil {
|
||||
|
||||
private static Logger LOG = LoggerFactory.getLogger(StreamerUtil.class);
|
||||
|
||||
public static Properties getKafkaProps(HoodieFlinkStreamer.Config cfg) {
|
||||
Properties result = new Properties();
|
||||
result.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, cfg.kafkaBootstrapServers);
|
||||
result.put(ConsumerConfig.GROUP_ID_CONFIG, cfg.kafkaGroupId);
|
||||
return result;
|
||||
}
|
||||
|
||||
public static TypedProperties getProps(HoodieFlinkStreamer.Config cfg) {
|
||||
return readConfig(
|
||||
FSUtils.getFs(cfg.propsFilePath, getHadoopConf()),
|
||||
|
||||
Reference in New Issue
Block a user