diff --git a/hudi-kafka-connect/pom.xml b/hudi-kafka-connect/pom.xml index 304afddd2727..d32d0fffe70f 100644 --- a/hudi-kafka-connect/pom.xml +++ b/hudi-kafka-connect/pom.xml @@ -66,21 +66,6 @@ com.github.os72 protoc-jar-maven-plugin - 3.11.4 - - - generate-sources - - run - - - ${protoc.version} - - src/main/resources - - - - diff --git a/hudi-utilities/pom.xml b/hudi-utilities/pom.xml index 1fd9e55a01b5..b6a5495e15bf 100644 --- a/hudi-utilities/pom.xml +++ b/hudi-utilities/pom.xml @@ -60,6 +60,10 @@ org.apache.rat apache-rat-plugin + + com.github.os72 + protoc-jar-maven-plugin + @@ -73,6 +77,23 @@ + + + com.google.protobuf + protobuf-java + + + com.google.protobuf + protobuf-java-util + test + + + com.google.guava + guava + + + + com.h2database diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/SourceFormatAdapter.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/SourceFormatAdapter.java index 3514ace829ab..9fbb11fd8286 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/SourceFormatAdapter.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/SourceFormatAdapter.java @@ -25,13 +25,11 @@ import org.apache.hudi.utilities.UtilHelpers; import org.apache.hudi.utilities.schema.FilebasedSchemaProvider; import org.apache.hudi.utilities.schema.SchemaProvider; -import org.apache.hudi.utilities.sources.AvroSource; import org.apache.hudi.utilities.sources.InputBatch; -import org.apache.hudi.utilities.sources.JsonSource; -import org.apache.hudi.utilities.sources.RowSource; import org.apache.hudi.utilities.sources.Source; import org.apache.hudi.utilities.sources.helpers.AvroConvertor; +import com.google.protobuf.Message; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.spark.api.java.JavaRDD; @@ -62,15 +60,15 @@ public SourceFormatAdapter(Source source) { public InputBatch> fetchNewDataInAvroFormat(Option lastCkptStr, long sourceLimit) { switch (source.getSourceType()) { case AVRO: - return ((AvroSource) source).fetchNext(lastCkptStr, sourceLimit); + return ((Source>) source).fetchNext(lastCkptStr, sourceLimit); case JSON: { - InputBatch> r = ((JsonSource) source).fetchNext(lastCkptStr, sourceLimit); + InputBatch> r = ((Source>) source).fetchNext(lastCkptStr, sourceLimit); AvroConvertor convertor = new AvroConvertor(r.getSchemaProvider().getSourceSchema()); return new InputBatch<>(Option.ofNullable(r.getBatch().map(rdd -> rdd.map(convertor::fromJson)).orElse(null)), r.getCheckpointForNextBatch(), r.getSchemaProvider()); } case ROW: { - InputBatch> r = ((RowSource) source).fetchNext(lastCkptStr, sourceLimit); + InputBatch> r = ((Source>) source).fetchNext(lastCkptStr, sourceLimit); return new InputBatch<>(Option.ofNullable(r.getBatch().map( rdd -> { SchemaProvider originalProvider = UtilHelpers.getOriginalSchemaProvider(r.getSchemaProvider()); @@ -85,6 +83,12 @@ public InputBatch> fetchNewDataInAvroFormat(Option> r = ((Source>) source).fetchNext(lastCkptStr, sourceLimit); + AvroConvertor convertor = new AvroConvertor(r.getSchemaProvider().getSourceSchema()); + return new InputBatch<>(Option.ofNullable(r.getBatch().map(rdd -> rdd.map(convertor::fromProtoMessage)).orElse(null)), + r.getCheckpointForNextBatch(), r.getSchemaProvider()); + } default: throw new IllegalArgumentException("Unknown source type (" + source.getSourceType() + ")"); } @@ -96,9 +100,9 @@ public InputBatch> fetchNewDataInAvroFormat(Option> fetchNewDataInRowFormat(Option lastCkptStr, long sourceLimit) { switch (source.getSourceType()) { case ROW: - return ((RowSource) source).fetchNext(lastCkptStr, sourceLimit); + return ((Source>) source).fetchNext(lastCkptStr, sourceLimit); case AVRO: { - InputBatch> r = ((AvroSource) source).fetchNext(lastCkptStr, sourceLimit); + InputBatch> r = ((Source>) source).fetchNext(lastCkptStr, sourceLimit); Schema sourceSchema = r.getSchemaProvider().getSourceSchema(); return new InputBatch<>( Option @@ -111,7 +115,7 @@ public InputBatch> fetchNewDataInRowFormat(Option lastCkptS r.getCheckpointForNextBatch(), r.getSchemaProvider()); } case JSON: { - InputBatch> r = ((JsonSource) source).fetchNext(lastCkptStr, sourceLimit); + InputBatch> r = ((Source>) source).fetchNext(lastCkptStr, sourceLimit); Schema sourceSchema = r.getSchemaProvider().getSourceSchema(); StructType dataType = AvroConversionUtils.convertAvroSchemaToStructType(sourceSchema); return new InputBatch<>( @@ -119,6 +123,21 @@ public InputBatch> fetchNewDataInRowFormat(Option lastCkptS r.getBatch().map(rdd -> source.getSparkSession().read().schema(dataType).json(rdd)).orElse(null)), r.getCheckpointForNextBatch(), r.getSchemaProvider()); } + case PROTO: { + InputBatch> r = ((Source>) source).fetchNext(lastCkptStr, sourceLimit); + Schema sourceSchema = r.getSchemaProvider().getSourceSchema(); + AvroConvertor convertor = new AvroConvertor(r.getSchemaProvider().getSourceSchema()); + return new InputBatch<>( + Option + .ofNullable( + r.getBatch() + .map(rdd -> rdd.map(convertor::fromProtoMessage)) + .map(rdd -> AvroConversionUtils.createDataFrame(JavaRDD.toRDD(rdd), sourceSchema.toString(), + source.getSparkSession()) + ) + .orElse(null)), + r.getCheckpointForNextBatch(), r.getSchemaProvider()); + } default: throw new IllegalArgumentException("Unknown source type (" + source.getSourceType() + ")"); } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/ProtoClassBasedSchemaProvider.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/ProtoClassBasedSchemaProvider.java new file mode 100644 index 000000000000..360b8de3c51e --- /dev/null +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/ProtoClassBasedSchemaProvider.java @@ -0,0 +1,78 @@ +/* + * 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.hudi.utilities.schema; + +import org.apache.hudi.DataSourceUtils; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.util.ReflectionUtils; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.utilities.sources.helpers.ProtoConversionUtil; + +import org.apache.avro.Schema; +import org.apache.spark.api.java.JavaSparkContext; + +import java.util.Collections; + +/** + * A schema provider that takes in a class name for a generated protobuf class that is on the classpath. + */ +public class ProtoClassBasedSchemaProvider extends SchemaProvider { + /** + * Configs supported. + */ + public static class Config { + public static final String PROTO_SCHEMA_CLASS_NAME = "hoodie.deltastreamer.schemaprovider.proto.className"; + public static final String PROTO_SCHEMA_FLATTEN_WRAPPED_PRIMITIVES = "hoodie.deltastreamer.schemaprovider.proto.flattenWrappers"; + } + + private final String schemaString; + + /** + * To be lazily inited on executors. + */ + private transient Schema schema; + + public ProtoClassBasedSchemaProvider(TypedProperties props, JavaSparkContext jssc) { + super(props, jssc); + DataSourceUtils.checkRequiredProperties(props, Collections.singletonList( + Config.PROTO_SCHEMA_CLASS_NAME)); + String className = config.getString(Config.PROTO_SCHEMA_CLASS_NAME); + boolean flattenWrappedPrimitives = props.getBoolean(ProtoClassBasedSchemaProvider.Config.PROTO_SCHEMA_FLATTEN_WRAPPED_PRIMITIVES, false); + try { + schemaString = ProtoConversionUtil.getAvroSchemaForMessageClass(ReflectionUtils.getClass(className), flattenWrappedPrimitives).toString(); + } catch (Exception e) { + throw new HoodieException(String.format("Error reading proto source schema for class: %s", className), e); + } + } + + @Override + public Schema getSourceSchema() { + if (schema == null) { + Schema.Parser parser = new Schema.Parser(); + schema = parser.parse(schemaString); + } + return schema; + } + + @Override + public Schema getTargetSchema() { + return getSourceSchema(); + } +} diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/AvroKafkaSource.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/AvroKafkaSource.java index 84c6fd815e83..a20ecbdfbf0e 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/AvroKafkaSource.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/AvroKafkaSource.java @@ -20,16 +20,13 @@ import org.apache.hudi.DataSourceWriteOptions; import org.apache.hudi.common.config.TypedProperties; -import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamerMetrics; import org.apache.hudi.utilities.deser.KafkaAvroSchemaDeserializer; -import org.apache.hudi.utilities.exception.HoodieSourceTimeoutException; import org.apache.hudi.utilities.schema.SchemaProvider; import org.apache.hudi.utilities.sources.helpers.AvroConvertor; import org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen; -import org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen.CheckpointUtils; import org.apache.avro.generic.GenericRecord; import org.apache.kafka.common.serialization.StringDeserializer; @@ -46,24 +43,17 @@ /** * Reads avro serialized Kafka data, based on the confluent schema-registry. */ -public class AvroKafkaSource extends AvroSource { +public class AvroKafkaSource extends KafkaSource { private static final Logger LOG = LogManager.getLogger(AvroKafkaSource.class); - // these are native kafka's config. do not change the config names. - private static final String NATIVE_KAFKA_KEY_DESERIALIZER_PROP = "key.deserializer"; - private static final String NATIVE_KAFKA_VALUE_DESERIALIZER_PROP = "value.deserializer"; // These are settings used to pass things to KafkaAvroDeserializer public static final String KAFKA_AVRO_VALUE_DESERIALIZER_PROPERTY_PREFIX = "hoodie.deltastreamer.source.kafka.value.deserializer."; public static final String KAFKA_AVRO_VALUE_DESERIALIZER_SCHEMA = KAFKA_AVRO_VALUE_DESERIALIZER_PROPERTY_PREFIX + "schema"; - - private final KafkaOffsetGen offsetGen; - private final HoodieDeltaStreamerMetrics metrics; - private final SchemaProvider schemaProvider; private final String deserializerClassName; public AvroKafkaSource(TypedProperties props, JavaSparkContext sparkContext, SparkSession sparkSession, SchemaProvider schemaProvider, HoodieDeltaStreamerMetrics metrics) { - super(props, sparkContext, sparkSession, schemaProvider); + super(props, sparkContext, sparkSession, schemaProvider, SourceType.AVRO, metrics); props.put(NATIVE_KAFKA_KEY_DESERIALIZER_PROP, StringDeserializer.class.getName()); deserializerClassName = props.getString(DataSourceWriteOptions.KAFKA_AVRO_VALUE_DESERIALIZER_CLASS().key(), @@ -82,29 +72,11 @@ public AvroKafkaSource(TypedProperties props, JavaSparkContext sparkContext, Spa LOG.error(error); throw new HoodieException(error, e); } - - this.schemaProvider = schemaProvider; - this.metrics = metrics; - offsetGen = new KafkaOffsetGen(props); + this.offsetGen = new KafkaOffsetGen(props); } @Override - protected InputBatch> fetchNewData(Option lastCheckpointStr, long sourceLimit) { - try { - OffsetRange[] offsetRanges = offsetGen.getNextOffsetRanges(lastCheckpointStr, sourceLimit, metrics); - long totalNewMsgs = CheckpointUtils.totalNewMessages(offsetRanges); - LOG.info("About to read " + totalNewMsgs + " from Kafka for topic :" + offsetGen.getTopicName()); - if (totalNewMsgs <= 0) { - return new InputBatch<>(Option.empty(), CheckpointUtils.offsetsToStr(offsetRanges)); - } - JavaRDD newDataRDD = toRDD(offsetRanges); - return new InputBatch<>(Option.of(newDataRDD), CheckpointUtils.offsetsToStr(offsetRanges)); - } catch (org.apache.kafka.common.errors.TimeoutException e) { - throw new HoodieSourceTimeoutException("Kafka Source timed out " + e.getMessage()); - } - } - - private JavaRDD toRDD(OffsetRange[] offsetRanges) { + JavaRDD toRDD(OffsetRange[] offsetRanges) { if (deserializerClassName.equals(ByteArrayDeserializer.class.getName())) { if (schemaProvider == null) { throw new HoodieException("Please provide a valid schema provider class when use ByteArrayDeserializer!"); @@ -117,11 +89,4 @@ private JavaRDD toRDD(OffsetRange[] offsetRanges) { LocationStrategies.PreferConsistent()).map(obj -> (GenericRecord) obj.value()); } } - - @Override - public void onCommit(String lastCkptStr) { - if (this.props.getBoolean(KafkaOffsetGen.Config.ENABLE_KAFKA_COMMIT_OFFSET.key(), KafkaOffsetGen.Config.ENABLE_KAFKA_COMMIT_OFFSET.defaultValue())) { - offsetGen.commitOffsetToKafka(lastCkptStr); - } - } } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/JsonKafkaSource.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/JsonKafkaSource.java index 0b06d986bbf1..26849d499e92 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/JsonKafkaSource.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/JsonKafkaSource.java @@ -19,20 +19,15 @@ package org.apache.hudi.utilities.sources; import org.apache.hudi.common.config.TypedProperties; -import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.utilities.UtilHelpers; import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamerMetrics; import org.apache.hudi.utilities.exception.HoodieSourcePostProcessException; -import org.apache.hudi.utilities.exception.HoodieSourceTimeoutException; import org.apache.hudi.utilities.schema.SchemaProvider; import org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen; -import org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen.CheckpointUtils; import org.apache.hudi.utilities.sources.processor.JsonKafkaSourcePostProcessor; import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SparkSession; @@ -45,40 +40,18 @@ /** * Read json kafka data. */ -public class JsonKafkaSource extends JsonSource { - - private static final Logger LOG = LogManager.getLogger(JsonKafkaSource.class); - - private final KafkaOffsetGen offsetGen; - - private final HoodieDeltaStreamerMetrics metrics; +public class JsonKafkaSource extends KafkaSource { public JsonKafkaSource(TypedProperties properties, JavaSparkContext sparkContext, SparkSession sparkSession, SchemaProvider schemaProvider, HoodieDeltaStreamerMetrics metrics) { - super(properties, sparkContext, sparkSession, schemaProvider); - this.metrics = metrics; + super(properties, sparkContext, sparkSession, schemaProvider, SourceType.JSON, metrics); properties.put("key.deserializer", StringDeserializer.class.getName()); properties.put("value.deserializer", StringDeserializer.class.getName()); - offsetGen = new KafkaOffsetGen(properties); + this.offsetGen = new KafkaOffsetGen(props); } @Override - protected InputBatch> fetchNewData(Option lastCheckpointStr, long sourceLimit) { - try { - OffsetRange[] offsetRanges = offsetGen.getNextOffsetRanges(lastCheckpointStr, sourceLimit, metrics); - long totalNewMsgs = CheckpointUtils.totalNewMessages(offsetRanges); - LOG.info("About to read " + totalNewMsgs + " from Kafka for topic :" + offsetGen.getTopicName()); - if (totalNewMsgs <= 0) { - return new InputBatch<>(Option.empty(), CheckpointUtils.offsetsToStr(offsetRanges)); - } - JavaRDD newDataRDD = toRDD(offsetRanges); - return new InputBatch<>(Option.of(newDataRDD), CheckpointUtils.offsetsToStr(offsetRanges)); - } catch (org.apache.kafka.common.errors.TimeoutException e) { - throw new HoodieSourceTimeoutException("Kafka Source timed out " + e.getMessage()); - } - } - - private JavaRDD toRDD(OffsetRange[] offsetRanges) { + JavaRDD toRDD(OffsetRange[] offsetRanges) { JavaRDD jsonStringRDD = KafkaUtils.createRDD(sparkContext, offsetGen.getKafkaParams(), offsetRanges, @@ -104,12 +77,4 @@ private JavaRDD postProcess(JavaRDD jsonStringRDD) { return processor.process(jsonStringRDD); } - - @Override - public void onCommit(String lastCkptStr) { - if (this.props.getBoolean(KafkaOffsetGen.Config.ENABLE_KAFKA_COMMIT_OFFSET.key(), - KafkaOffsetGen.Config.ENABLE_KAFKA_COMMIT_OFFSET.defaultValue())) { - offsetGen.commitOffsetToKafka(lastCkptStr); - } - } } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/KafkaSource.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/KafkaSource.java new file mode 100644 index 000000000000..6f2377fc7ce9 --- /dev/null +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/KafkaSource.java @@ -0,0 +1,77 @@ +/* + * 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.hudi.utilities.sources; + +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamerMetrics; +import org.apache.hudi.utilities.exception.HoodieSourceTimeoutException; +import org.apache.hudi.utilities.schema.SchemaProvider; +import org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.streaming.kafka010.OffsetRange; + +abstract class KafkaSource extends Source> { + private static final Logger LOG = LogManager.getLogger(KafkaSource.class); + // these are native kafka's config. do not change the config names. + protected static final String NATIVE_KAFKA_KEY_DESERIALIZER_PROP = "key.deserializer"; + protected static final String NATIVE_KAFKA_VALUE_DESERIALIZER_PROP = "value.deserializer"; + + protected final HoodieDeltaStreamerMetrics metrics; + protected final SchemaProvider schemaProvider; + protected KafkaOffsetGen offsetGen; + + protected KafkaSource(TypedProperties props, JavaSparkContext sparkContext, SparkSession sparkSession, + SchemaProvider schemaProvider, SourceType sourceType, HoodieDeltaStreamerMetrics metrics) { + super(props, sparkContext, sparkSession, schemaProvider, sourceType); + + this.schemaProvider = schemaProvider; + this.metrics = metrics; + } + + @Override + protected InputBatch> fetchNewData(Option lastCheckpointStr, long sourceLimit) { + try { + OffsetRange[] offsetRanges = offsetGen.getNextOffsetRanges(lastCheckpointStr, sourceLimit, metrics); + long totalNewMsgs = KafkaOffsetGen.CheckpointUtils.totalNewMessages(offsetRanges); + LOG.info("About to read " + totalNewMsgs + " from Kafka for topic :" + offsetGen.getTopicName()); + if (totalNewMsgs <= 0) { + return new InputBatch<>(Option.empty(), KafkaOffsetGen.CheckpointUtils.offsetsToStr(offsetRanges)); + } + JavaRDD newDataRDD = toRDD(offsetRanges); + return new InputBatch<>(Option.of(newDataRDD), KafkaOffsetGen.CheckpointUtils.offsetsToStr(offsetRanges)); + } catch (org.apache.kafka.common.errors.TimeoutException e) { + throw new HoodieSourceTimeoutException("Kafka Source timed out " + e.getMessage()); + } + } + + abstract JavaRDD toRDD(OffsetRange[] offsetRanges); + + @Override + public void onCommit(String lastCkptStr) { + if (this.props.getBoolean(KafkaOffsetGen.Config.ENABLE_KAFKA_COMMIT_OFFSET.key(), KafkaOffsetGen.Config.ENABLE_KAFKA_COMMIT_OFFSET.defaultValue())) { + offsetGen.commitOffsetToKafka(lastCkptStr); + } + } +} diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/ProtoKafkaSource.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/ProtoKafkaSource.java new file mode 100644 index 000000000000..ae37273be6b9 --- /dev/null +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/ProtoKafkaSource.java @@ -0,0 +1,105 @@ +/* + * 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.hudi.utilities.sources; + +import org.apache.hudi.DataSourceUtils; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.util.ReflectionUtils; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamerMetrics; +import org.apache.hudi.utilities.schema.ProtoClassBasedSchemaProvider; +import org.apache.hudi.utilities.schema.SchemaProvider; +import org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen; + +import com.google.protobuf.Message; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.streaming.kafka010.KafkaUtils; +import org.apache.spark.streaming.kafka010.LocationStrategies; +import org.apache.spark.streaming.kafka010.OffsetRange; + +import java.io.Serializable; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.Collections; + +/** + * Reads protobuf serialized Kafka data, based on a provided class name. + */ +public class ProtoKafkaSource extends KafkaSource { + + private final String className; + + public ProtoKafkaSource(TypedProperties props, JavaSparkContext sparkContext, + SparkSession sparkSession, SchemaProvider schemaProvider, HoodieDeltaStreamerMetrics metrics) { + super(props, sparkContext, sparkSession, schemaProvider, SourceType.PROTO, metrics); + DataSourceUtils.checkRequiredProperties(props, Collections.singletonList( + ProtoClassBasedSchemaProvider.Config.PROTO_SCHEMA_CLASS_NAME)); + props.put(NATIVE_KAFKA_KEY_DESERIALIZER_PROP, StringDeserializer.class); + props.put(NATIVE_KAFKA_VALUE_DESERIALIZER_PROP, ByteArrayDeserializer.class); + className = props.getString(ProtoClassBasedSchemaProvider.Config.PROTO_SCHEMA_CLASS_NAME); + this.offsetGen = new KafkaOffsetGen(props); + } + + @Override + JavaRDD toRDD(OffsetRange[] offsetRanges) { + ProtoDeserializer deserializer = new ProtoDeserializer(className); + return KafkaUtils.createRDD(sparkContext, offsetGen.getKafkaParams(), offsetRanges, + LocationStrategies.PreferConsistent()).map(obj -> deserializer.parse(obj.value())); + } + + private static class ProtoDeserializer implements Serializable { + private final String className; + private transient Class protoClass; + private transient Method parseMethod; + + public ProtoDeserializer(String className) { + this.className = className; + } + + public Message parse(byte[] bytes) { + try { + return (Message) getParseMethod().invoke(getClass(), bytes); + } catch (IllegalAccessException | InvocationTargetException ex) { + throw new HoodieException("Failed to parse proto message from kafka", ex); + } + } + + private Class getProtoClass() { + if (protoClass == null) { + protoClass = ReflectionUtils.getClass(className); + } + return protoClass; + } + + private Method getParseMethod() { + if (parseMethod == null) { + try { + parseMethod = getProtoClass().getMethod("parseFrom", byte[].class); + } catch (NoSuchMethodException ex) { + throw new HoodieException("Unable to get proto parsing method from specified class: " + className, ex); + } + } + return parseMethod; + } + } +} diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/Source.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/Source.java index 6d610d5c8cbd..2e858167e05d 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/Source.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/Source.java @@ -38,7 +38,7 @@ public abstract class Source implements SourceCommitCallback, Serializable { public enum SourceType { - JSON, AVRO, ROW + JSON, AVRO, ROW, PROTO } protected transient TypedProperties props; diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/AvroConvertor.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/AvroConvertor.java index 449db10b2d52..860c67f4e2ad 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/AvroConvertor.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/AvroConvertor.java @@ -20,6 +20,7 @@ import org.apache.hudi.avro.MercifulJsonConverter; +import com.google.protobuf.Message; import com.twitter.bijection.Injection; import com.twitter.bijection.avro.GenericAvroCodecs; import org.apache.avro.Schema; @@ -95,4 +96,9 @@ public GenericRecord fromAvroBinary(byte[] avroBinary) { initInjection(); return recordInjection.invert(avroBinary).get(); } + + public GenericRecord fromProtoMessage(Message message) { + initSchema(); + return ProtoConversionUtil.convertToAvro(schema, message); + } } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/ProtoConversionUtil.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/ProtoConversionUtil.java new file mode 100644 index 000000000000..240f0a4bd912 --- /dev/null +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/ProtoConversionUtil.java @@ -0,0 +1,359 @@ +/* + * 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.hudi.utilities.sources.helpers; + +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieException; + +import com.google.protobuf.BoolValue; +import com.google.protobuf.ByteString; +import com.google.protobuf.BytesValue; +import com.google.protobuf.Descriptors; +import com.google.protobuf.DoubleValue; +import com.google.protobuf.FloatValue; +import com.google.protobuf.Int32Value; +import com.google.protobuf.Int64Value; +import com.google.protobuf.Message; +import com.google.protobuf.StringValue; +import com.google.protobuf.UInt32Value; +import com.google.protobuf.UInt64Value; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericFixed; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.util.Utf8; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Function; + +/** + * A utility class to help translate from Proto to Avro. + */ +public class ProtoConversionUtil { + + /** + * Creates an Avro {@link Schema} for the provided class. Assumes that the class is a protobuf {@link Message}. + * @param clazz The protobuf class + * @param flattenWrappedPrimitives set to true to treat wrapped primitives like nullable fields instead of nested messages. + * @return An Avro schema + */ + public static Schema getAvroSchemaForMessageClass(Class clazz, boolean flattenWrappedPrimitives) { + return AvroSupport.get().getSchema(clazz, flattenWrappedPrimitives); + } + + /** + * Converts the provided {@link Message} into an avro {@link GenericRecord} with the provided schema. + * @param schema target schema to convert into + * @param message the source message to convert + * @return an Avro GenericRecord + */ + public static GenericRecord convertToAvro(Schema schema, Message message) { + return AvroSupport.get().convert(schema, message); + } + + /** + * This class provides support for generating schemas and converting from proto to avro. We don't directly use Avro's ProtobufData class so we can: + * 1. Customize how schemas are generated for protobufs. We treat Enums as strings and provide an option to treat wrapped primitives like {@link Int32Value} and {@link StringValue} as messages + * (default behavior) or as nullable versions of those primitives. + * 2. Convert directly from a protobuf {@link Message} to a {@link GenericRecord} while properly handling enums and wrapped primitives mentioned above. + */ + private static class AvroSupport { + private static final AvroSupport INSTANCE = new AvroSupport(); + // A cache of the proto class name paired with whether wrapped primitives should be flattened as the key and the generated avro schema as the value + private static final Map, Schema> SCHEMA_CACHE = new ConcurrentHashMap<>(); + // A cache with a key as the pair target avro schema and the proto descriptor for the source and the value as an array of proto field descriptors where the order matches the avro ordering. + // When converting from proto to avro, we want to be able to iterate over the fields in the proto in the same order as they appear in the avro schema. + private static final Map, Descriptors.FieldDescriptor[]> FIELD_CACHE = new ConcurrentHashMap<>(); + + + private static final Schema STRINGS = Schema.create(Schema.Type.STRING); + + private static final Schema NULL = Schema.create(Schema.Type.NULL); + private static final Map WRAPPER_DESCRIPTORS_TO_TYPE = getWrapperDescriptorsToType(); + + private static Map getWrapperDescriptorsToType() { + Map wrapperDescriptorsToType = new HashMap<>(); + wrapperDescriptorsToType.put(StringValue.getDescriptor(), Schema.Type.STRING); + wrapperDescriptorsToType.put(Int32Value.getDescriptor(), Schema.Type.INT); + wrapperDescriptorsToType.put(UInt32Value.getDescriptor(), Schema.Type.INT); + wrapperDescriptorsToType.put(Int64Value.getDescriptor(), Schema.Type.LONG); + wrapperDescriptorsToType.put(UInt64Value.getDescriptor(), Schema.Type.LONG); + wrapperDescriptorsToType.put(BoolValue.getDescriptor(), Schema.Type.BOOLEAN); + wrapperDescriptorsToType.put(BytesValue.getDescriptor(), Schema.Type.BYTES); + wrapperDescriptorsToType.put(DoubleValue.getDescriptor(), Schema.Type.DOUBLE); + wrapperDescriptorsToType.put(FloatValue.getDescriptor(), Schema.Type.FLOAT); + return wrapperDescriptorsToType; + } + + private AvroSupport() { + } + + public static AvroSupport get() { + return INSTANCE; + } + + public GenericRecord convert(Schema schema, Message message) { + return (GenericRecord) convertObject(schema, message); + } + + public Schema getSchema(Class c, boolean flattenWrappedPrimitives) { + return SCHEMA_CACHE.computeIfAbsent(Pair.of(c, flattenWrappedPrimitives), key -> { + try { + Object descriptor = c.getMethod("getDescriptor").invoke(null); + if (c.isEnum()) { + return getEnumSchema((Descriptors.EnumDescriptor) descriptor); + } else { + return getMessageSchema((Descriptors.Descriptor) descriptor, new HashMap<>(), flattenWrappedPrimitives); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + } + + private Schema getEnumSchema(Descriptors.EnumDescriptor enumDescriptor) { + List symbols = new ArrayList<>(enumDescriptor.getValues().size()); + for (Descriptors.EnumValueDescriptor valueDescriptor : enumDescriptor.getValues()) { + symbols.add(valueDescriptor.getName()); + } + return Schema.createEnum(enumDescriptor.getName(), null, getNamespace(enumDescriptor.getFullName()), symbols); + } + + private Schema getMessageSchema(Descriptors.Descriptor descriptor, Map seen, boolean flattenWrappedPrimitives) { + if (seen.containsKey(descriptor)) { + return seen.get(descriptor); + } + Schema result = Schema.createRecord(descriptor.getName(), null, + getNamespace(descriptor.getFullName()), false); + + seen.put(descriptor, result); + + List fields = new ArrayList<>(descriptor.getFields().size()); + for (Descriptors.FieldDescriptor f : descriptor.getFields()) { + fields.add(new Schema.Field(f.getName(), getFieldSchema(f, seen, flattenWrappedPrimitives), null, getDefault(f))); + } + result.setFields(fields); + return result; + } + + private Schema getFieldSchema(Descriptors.FieldDescriptor f, Map seen, boolean flattenWrappedPrimitives) { + Function schemaFinalizer = f.isRepeated() ? Schema::createArray : Function.identity(); + switch (f.getType()) { + case BOOL: + return schemaFinalizer.apply(Schema.create(Schema.Type.BOOLEAN)); + case FLOAT: + return schemaFinalizer.apply(Schema.create(Schema.Type.FLOAT)); + case DOUBLE: + return schemaFinalizer.apply(Schema.create(Schema.Type.DOUBLE)); + case ENUM: + return schemaFinalizer.apply(getEnumSchema(f.getEnumType())); + case STRING: + Schema s = Schema.create(Schema.Type.STRING); + GenericData.setStringType(s, GenericData.StringType.String); + return schemaFinalizer.apply(s); + case BYTES: + return schemaFinalizer.apply(Schema.create(Schema.Type.BYTES)); + case INT32: + case SINT32: + case FIXED32: + case SFIXED32: + return schemaFinalizer.apply(Schema.create(Schema.Type.INT)); + case UINT32: + case INT64: + case UINT64: + case SINT64: + case FIXED64: + case SFIXED64: + return schemaFinalizer.apply(Schema.create(Schema.Type.LONG)); + case MESSAGE: + if (flattenWrappedPrimitives && WRAPPER_DESCRIPTORS_TO_TYPE.containsKey(f.getMessageType())) { + // all wrapper types have a single field, so we can get the first field in the message's schema + return schemaFinalizer.apply(Schema.createUnion(Arrays.asList(NULL, getFieldSchema(f.getMessageType().getFields().get(0), seen, flattenWrappedPrimitives)))); + } + // if message field is repeated (like a list), elements are non-null + if (f.isRepeated()) { + return schemaFinalizer.apply(getMessageSchema(f.getMessageType(), seen, flattenWrappedPrimitives)); + } + // otherwise we create a nullable field schema + return schemaFinalizer.apply(Schema.createUnion(Arrays.asList(NULL, getMessageSchema(f.getMessageType(), seen, flattenWrappedPrimitives)))); + case GROUP: // groups are deprecated + default: + throw new RuntimeException("Unexpected type: " + f.getType()); + } + } + + private Object getDefault(Descriptors.FieldDescriptor f) { + if (f.isRepeated()) { // empty array as repeated fields' default value + return Collections.emptyList(); + } + + switch (f.getType()) { // generate default for type + case BOOL: + return false; + case FLOAT: + return 0.0F; + case DOUBLE: + return 0.0D; + case INT32: + case UINT32: + case SINT32: + case FIXED32: + case SFIXED32: + case INT64: + case UINT64: + case SINT64: + case FIXED64: + case SFIXED64: + return 0; + case STRING: + case BYTES: + return ""; + case ENUM: + return f.getEnumType().getValues().get(0).getName(); + case MESSAGE: + return Schema.Field.NULL_VALUE; + case GROUP: // groups are deprecated + default: + throw new RuntimeException("Unexpected type: " + f.getType()); + } + } + + private Descriptors.FieldDescriptor[] getOrderedFields(Schema schema, Message message) { + Descriptors.Descriptor descriptor = message.getDescriptorForType(); + return FIELD_CACHE.computeIfAbsent(Pair.of(schema, descriptor), key -> { + Descriptors.FieldDescriptor[] fields = new Descriptors.FieldDescriptor[key.getLeft().getFields().size()]; + for (Schema.Field f : key.getLeft().getFields()) { + fields[f.pos()] = key.getRight().findFieldByName(f.name()); + } + return fields; + }); + } + + private Object convertObject(Schema schema, Object value) { + if (value == null) { + return null; + } + + switch (schema.getType()) { + case ARRAY: + List arrayValue = (List) value; + List arrayCopy = new GenericData.Array<>(arrayValue.size(), schema); + for (Object obj : arrayValue) { + arrayCopy.add(convertObject(schema.getElementType(), obj)); + } + return arrayCopy; + case BYTES: + ByteBuffer byteBufferValue; + if (value instanceof ByteString) { + byteBufferValue = ((ByteString) value).asReadOnlyByteBuffer(); + } else if (value instanceof Message) { + byteBufferValue = ((ByteString) getWrappedValue(value)).asReadOnlyByteBuffer(); + } else { + byteBufferValue = (ByteBuffer) value; + } + int start = byteBufferValue.position(); + int length = byteBufferValue.limit() - start; + byte[] bytesCopy = new byte[length]; + byteBufferValue.get(bytesCopy, 0, length); + byteBufferValue.position(start); + return ByteBuffer.wrap(bytesCopy, 0, length); + case ENUM: + return GenericData.get().createEnum(value.toString(), schema); + case FIXED: + return GenericData.get().createFixed(null, ((GenericFixed) value).bytes(), schema); + case BOOLEAN: + case DOUBLE: + case FLOAT: + case INT: + if (value instanceof Message) { + return getWrappedValue(value); + } + return value; // immutable + case LONG: + Object tmpValue = value; + if (value instanceof Message) { + tmpValue = getWrappedValue(value); + } + // unsigned ints need to be casted to long + if (tmpValue instanceof Integer) { + tmpValue = new Long((Integer) tmpValue); + } + return tmpValue; + case MAP: + Map mapValue = (Map) value; + Map mapCopy = new HashMap<>(mapValue.size()); + for (Map.Entry entry : mapValue.entrySet()) { + mapCopy.put(convertObject(STRINGS, entry.getKey()), convertObject(schema.getValueType(), entry.getValue())); + } + return mapCopy; + case NULL: + return null; + case RECORD: + GenericData.Record newRecord = new GenericData.Record(schema); + Message messageValue = (Message) value; + for (Schema.Field f : schema.getFields()) { + int position = f.pos(); + Descriptors.FieldDescriptor fieldDescriptor = getOrderedFields(schema, messageValue)[position]; + Object convertedValue; + if (fieldDescriptor.getType() == Descriptors.FieldDescriptor.Type.MESSAGE && !fieldDescriptor.isRepeated() && !messageValue.hasField(fieldDescriptor)) { + convertedValue = null; + } else { + convertedValue = convertObject(f.schema(), messageValue.getField(fieldDescriptor)); + } + newRecord.put(position, convertedValue); + } + return newRecord; + case STRING: + if (value instanceof String) { + return value; + } else if (value instanceof StringValue) { + return ((StringValue) value).getValue(); + } else { + return new Utf8(value.toString()); + } + case UNION: + // Unions only occur for nullable fields when working with proto + avro and null is the first schema in the union + return convertObject(schema.getTypes().get(1), value); + default: + throw new HoodieException("Proto to Avro conversion failed for schema \"" + schema + "\" and value \"" + value + "\""); + } + } + + /** + * Returns the wrapped field, assumes all wrapped fields have a single value + * @param value wrapper message like {@link Int32Value} or {@link StringValue} + * @return the wrapped object + */ + private Object getWrappedValue(Object value) { + Message valueAsMessage = (Message) value; + return valueAsMessage.getField(valueAsMessage.getDescriptorForType().getFields().get(0)); + } + + private String getNamespace(String descriptorFullName) { + int lastDotIndex = descriptorFullName.lastIndexOf('.'); + return descriptorFullName.substring(0, lastDotIndex); + } + } +} diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/schema/TestProtoClassBasedSchemaProvider.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/schema/TestProtoClassBasedSchemaProvider.java new file mode 100644 index 000000000000..2d069f7f591c --- /dev/null +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/schema/TestProtoClassBasedSchemaProvider.java @@ -0,0 +1,61 @@ +/* + * 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.hudi.utilities.schema; + +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.utilities.test.proto.Sample; + +import org.apache.avro.Schema; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.Scanner; + +public class TestProtoClassBasedSchemaProvider { + + @Test + public void validateDefaultSchemaGeneration() { + TypedProperties properties = new TypedProperties(); + properties.setProperty(ProtoClassBasedSchemaProvider.Config.PROTO_SCHEMA_CLASS_NAME, Sample.class.getName()); + ProtoClassBasedSchemaProvider protoToAvroSchemaProvider = new ProtoClassBasedSchemaProvider(properties, null); + Schema protoSchema = protoToAvroSchemaProvider.getSourceSchema(); + Schema.Parser parser = new Schema.Parser(); + Schema expectedSchema = parser.parse(getExpectedSchema("schema-provider/proto/sample_schema_nested.txt")); + Assertions.assertEquals(expectedSchema, protoSchema); + } + + @Test + public void validateFlattenedSchemaGeneration() { + TypedProperties properties = new TypedProperties(); + properties.setProperty(ProtoClassBasedSchemaProvider.Config.PROTO_SCHEMA_CLASS_NAME, Sample.class.getName()); + properties.setProperty(ProtoClassBasedSchemaProvider.Config.PROTO_SCHEMA_FLATTEN_WRAPPED_PRIMITIVES, "true"); + ProtoClassBasedSchemaProvider protoToAvroSchemaProvider = new ProtoClassBasedSchemaProvider(properties, null); + Schema protoSchema = protoToAvroSchemaProvider.getSourceSchema(); + Schema.Parser parser = new Schema.Parser(); + Schema expectedSchema = parser.parse(getExpectedSchema("schema-provider/proto/sample_schema_flattened.txt")); + Assertions.assertEquals(expectedSchema, protoSchema); + } + + private String getExpectedSchema(String pathToExpectedSchema) { + try (Scanner scanner = new Scanner(getClass().getClassLoader().getResourceAsStream(pathToExpectedSchema))) { + return scanner.next(); + } + } +} diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/BaseTestKafkaSource.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/BaseTestKafkaSource.java new file mode 100644 index 000000000000..2b99f19b27b2 --- /dev/null +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/BaseTestKafkaSource.java @@ -0,0 +1,280 @@ +/* + * 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.hudi.utilities.sources; + +import org.apache.hudi.AvroConversionUtils; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieNotSupportedException; +import org.apache.hudi.testutils.SparkClientFunctionalTestHarness; +import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamerMetrics; +import org.apache.hudi.utilities.deltastreamer.SourceFormatAdapter; +import org.apache.hudi.utilities.exception.HoodieDeltaStreamerException; +import org.apache.hudi.utilities.schema.SchemaProvider; +import org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen; + +import org.apache.avro.generic.GenericRecord; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.TopicPartition; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.streaming.kafka010.KafkaTestUtils; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import static org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen.Config.ENABLE_KAFKA_COMMIT_OFFSET; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.Mockito.mock; + +/** + * Generic tests for all {@link KafkaSource} to ensure all implementations properly handle offsets, fetch limits, failure modes, etc. + */ +abstract class BaseTestKafkaSource extends SparkClientFunctionalTestHarness { + protected static final String TEST_TOPIC_PREFIX = "hoodie_test_"; + protected static KafkaTestUtils testUtils; + + protected final HoodieDeltaStreamerMetrics metrics = mock(HoodieDeltaStreamerMetrics.class); + + protected SchemaProvider schemaProvider; + + @BeforeAll + public static void initClass() { + testUtils = new KafkaTestUtils(); + testUtils.setup(); + } + + @AfterAll + public static void cleanupClass() { + testUtils.teardown(); + } + + abstract TypedProperties createPropsForKafkaSource(String topic, Long maxEventsToReadFromKafkaSource, String resetStrategy); + + abstract SourceFormatAdapter createSource(TypedProperties props); + + abstract void sendMessagesToKafka(String topic, int count, int numPartitions); + + @Test + public void testKafkaSource() { + + // topic setup. + final String topic = TEST_TOPIC_PREFIX + "testKafkaSource"; + testUtils.createTopic(topic, 2); + TypedProperties props = createPropsForKafkaSource(topic, null, "earliest"); + SourceFormatAdapter kafkaSource = createSource(props); + + // 1. Extract without any checkpoint => get all the data, respecting sourceLimit + assertEquals(Option.empty(), kafkaSource.fetchNewDataInAvroFormat(Option.empty(), Long.MAX_VALUE).getBatch()); + sendMessagesToKafka(topic, 1000, 2); + InputBatch> fetch1 = kafkaSource.fetchNewDataInAvroFormat(Option.empty(), 900); + assertEquals(900, fetch1.getBatch().get().count()); + // Test Avro To DataFrame path + Dataset fetch1AsRows = AvroConversionUtils.createDataFrame(JavaRDD.toRDD(fetch1.getBatch().get()), + schemaProvider.getSourceSchema().toString(), kafkaSource.getSource().getSparkSession()); + assertEquals(900, fetch1AsRows.count()); + + // 2. Produce new data, extract new data + sendMessagesToKafka(topic, 1000, 2); + InputBatch> fetch2 = + kafkaSource.fetchNewDataInRowFormat(Option.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE); + assertEquals(1100, fetch2.getBatch().get().count()); + + // 3. Extract with previous checkpoint => gives same data back (idempotent) + InputBatch> fetch3 = + kafkaSource.fetchNewDataInAvroFormat(Option.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE); + assertEquals(fetch2.getBatch().get().count(), fetch3.getBatch().get().count()); + assertEquals(fetch2.getCheckpointForNextBatch(), fetch3.getCheckpointForNextBatch()); + // Same using Row API + InputBatch> fetch3AsRows = + kafkaSource.fetchNewDataInRowFormat(Option.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE); + assertEquals(fetch2.getBatch().get().count(), fetch3AsRows.getBatch().get().count()); + assertEquals(fetch2.getCheckpointForNextBatch(), fetch3AsRows.getCheckpointForNextBatch()); + + // 4. Extract with latest checkpoint => no new data returned + InputBatch> fetch4 = + kafkaSource.fetchNewDataInAvroFormat(Option.of(fetch2.getCheckpointForNextBatch()), Long.MAX_VALUE); + assertEquals(Option.empty(), fetch4.getBatch()); + // Same using Row API + InputBatch> fetch4AsRows = + kafkaSource.fetchNewDataInRowFormat(Option.of(fetch2.getCheckpointForNextBatch()), Long.MAX_VALUE); + assertEquals(Option.empty(), fetch4AsRows.getBatch()); + } + + // test case with kafka offset reset strategy + @Test + public void testKafkaSourceResetStrategy() { + // topic setup. + final String topic = TEST_TOPIC_PREFIX + "testKafkaSourceResetStrategy"; + testUtils.createTopic(topic, 2); + + TypedProperties earliestProps = createPropsForKafkaSource(topic, null, "earliest"); + SourceFormatAdapter earliestKafkaSource = createSource(earliestProps); + + TypedProperties latestProps = createPropsForKafkaSource(topic, null, "latest"); + SourceFormatAdapter latestKafkaSource = createSource(latestProps); + + // 1. Extract with a none data kafka checkpoint + // => get a checkpoint string like "hoodie_test,0:0,1:0", latest checkpoint should be equals to earliest checkpoint + InputBatch> earFetch0 = earliestKafkaSource.fetchNewDataInAvroFormat(Option.empty(), Long.MAX_VALUE); + InputBatch> latFetch0 = latestKafkaSource.fetchNewDataInAvroFormat(Option.empty(), Long.MAX_VALUE); + assertEquals(earFetch0.getBatch(), latFetch0.getBatch()); + assertEquals(earFetch0.getCheckpointForNextBatch(), latFetch0.getCheckpointForNextBatch()); + + sendMessagesToKafka(topic, 1000, 2); + + // 2. Extract new checkpoint with a null / empty string pre checkpoint + // => earliest fetch with max source limit will get all of data and a end offset checkpoint + InputBatch> earFetch1 = earliestKafkaSource.fetchNewDataInAvroFormat(Option.empty(), Long.MAX_VALUE); + + // => [a null pre checkpoint] latest reset fetch will get a end offset checkpoint same to earliest + InputBatch> latFetch1 = latestKafkaSource.fetchNewDataInAvroFormat(Option.empty(), Long.MAX_VALUE); + assertEquals(earFetch1.getCheckpointForNextBatch(), latFetch1.getCheckpointForNextBatch()); + } + + @Test + public void testProtoKafkaSourceInsertRecordsLessSourceLimit() { + // topic setup. + final String topic = TEST_TOPIC_PREFIX + "testKafkaSourceInsertRecordsLessSourceLimit"; + testUtils.createTopic(topic, 2); + TypedProperties props = createPropsForKafkaSource(topic, Long.MAX_VALUE, "earliest"); + SourceFormatAdapter kafkaSource = createSource(props); + props.setProperty("hoodie.deltastreamer.kafka.source.maxEvents", "500"); + + /* + 1. maxEventsFromKafkaSourceProp set to more than generated insert records + and sourceLimit less than the generated insert records num. + */ + sendMessagesToKafka(topic, 400, 2); + InputBatch> fetch1 = kafkaSource.fetchNewDataInAvroFormat(Option.empty(), 300); + assertEquals(300, fetch1.getBatch().get().count()); + + /* + 2. Produce new data, extract new data based on sourceLimit + and sourceLimit less than the generated insert records num. + */ + sendMessagesToKafka(topic, 600, 2); + InputBatch> fetch2 = + kafkaSource.fetchNewDataInRowFormat(Option.of(fetch1.getCheckpointForNextBatch()), 300); + assertEquals(300, fetch2.getBatch().get().count()); + } + + @Test + public void testCommitOffsetToKafka() { + // topic setup. + final String topic = TEST_TOPIC_PREFIX + "testCommitOffsetToKafka"; + testUtils.createTopic(topic, 2); + List topicPartitions = new ArrayList<>(); + TopicPartition topicPartition0 = new TopicPartition(topic, 0); + topicPartitions.add(topicPartition0); + TopicPartition topicPartition1 = new TopicPartition(topic, 1); + topicPartitions.add(topicPartition1); + + TypedProperties props = createPropsForKafkaSource(topic, null, "earliest"); + props.put(ENABLE_KAFKA_COMMIT_OFFSET.key(), "true"); + SourceFormatAdapter kafkaSource = createSource(props); + + // 1. Extract without any checkpoint => get all the data, respecting sourceLimit + assertEquals(Option.empty(), kafkaSource.fetchNewDataInAvroFormat(Option.empty(), Long.MAX_VALUE).getBatch()); + sendMessagesToKafka(topic, 1000, 2); + + InputBatch> fetch1 = kafkaSource.fetchNewDataInAvroFormat(Option.empty(), 599); + // commit to kafka after first batch + kafkaSource.getSource().onCommit(fetch1.getCheckpointForNextBatch()); + try (KafkaConsumer consumer = new KafkaConsumer(props)) { + consumer.assign(topicPartitions); + + OffsetAndMetadata offsetAndMetadata = consumer.committed(topicPartition0); + assertNotNull(offsetAndMetadata); + assertEquals(300, offsetAndMetadata.offset()); + offsetAndMetadata = consumer.committed(topicPartition1); + assertNotNull(offsetAndMetadata); + assertEquals(299, offsetAndMetadata.offset()); + // end offsets will point to 500 for each partition because we consumed less messages from first batch + Map endOffsets = consumer.endOffsets(topicPartitions); + assertEquals(500L, endOffsets.get(topicPartition0)); + assertEquals(500L, endOffsets.get(topicPartition1)); + + sendMessagesToKafka(topic, 500, 2); + InputBatch> fetch2 = + kafkaSource.fetchNewDataInRowFormat(Option.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE); + + // commit to Kafka after second batch is processed completely + kafkaSource.getSource().onCommit(fetch2.getCheckpointForNextBatch()); + + offsetAndMetadata = consumer.committed(topicPartition0); + assertNotNull(offsetAndMetadata); + assertEquals(750, offsetAndMetadata.offset()); + offsetAndMetadata = consumer.committed(topicPartition1); + assertNotNull(offsetAndMetadata); + assertEquals(750, offsetAndMetadata.offset()); + + endOffsets = consumer.endOffsets(topicPartitions); + assertEquals(750L, endOffsets.get(topicPartition0)); + assertEquals(750L, endOffsets.get(topicPartition1)); + } + // check failure case + props.remove(ConsumerConfig.GROUP_ID_CONFIG); + assertThrows(HoodieNotSupportedException.class,() -> kafkaSource.getSource().onCommit("")); + } + + @Test + public void testFailOnDataLoss() throws Exception { + // create a topic with very short retention + final String topic = TEST_TOPIC_PREFIX + "testFailOnDataLoss"; + Properties topicConfig = new Properties(); + topicConfig.setProperty("retention.ms", "10000"); + testUtils.createTopic(topic, 1, topicConfig); + + TypedProperties failOnDataLossProps = createPropsForKafkaSource(topic, null, "earliest"); + failOnDataLossProps.setProperty(KafkaOffsetGen.Config.ENABLE_FAIL_ON_DATA_LOSS.key(), Boolean.toString(true)); + SourceFormatAdapter kafkaSource = createSource(failOnDataLossProps); + sendMessagesToKafka(topic, 10, 2); + // send 10 records, extract 2 records to generate a checkpoint + InputBatch> fetch1 = kafkaSource.fetchNewDataInAvroFormat(Option.empty(), 2); + assertEquals(2, fetch1.getBatch().get().count()); + + // wait for the checkpoint to expire + Thread.sleep(10001); + Throwable t = assertThrows(HoodieDeltaStreamerException.class, () -> { + kafkaSource.fetchNewDataInAvroFormat(Option.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE); + }); + assertEquals( + "Some data may have been lost because they are not available in Kafka any more;" + + " either the data was aged out by Kafka or the topic may have been deleted before all the data in the topic was processed.", + t.getMessage()); + t = assertThrows(HoodieDeltaStreamerException.class, () -> { + kafkaSource.fetchNewDataInRowFormat(Option.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE); + }); + assertEquals( + "Some data may have been lost because they are not available in Kafka any more;" + + " either the data was aged out by Kafka or the topic may have been deleted before all the data in the topic was processed.", + t.getMessage()); + } +} diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJsonKafkaSource.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJsonKafkaSource.java index 05d79e0449fa..d77789140f3c 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJsonKafkaSource.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJsonKafkaSource.java @@ -18,69 +18,33 @@ package org.apache.hudi.utilities.sources; -import org.apache.hudi.AvroConversionUtils; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.util.Option; -import org.apache.hudi.exception.HoodieNotSupportedException; -import org.apache.hudi.testutils.SparkClientFunctionalTestHarness; -import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamerMetrics; import org.apache.hudi.utilities.deltastreamer.SourceFormatAdapter; -import org.apache.hudi.utilities.exception.HoodieDeltaStreamerException; import org.apache.hudi.utilities.schema.FilebasedSchemaProvider; import org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen.Config; import org.apache.avro.generic.GenericRecord; import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.clients.consumer.OffsetAndMetadata; -import org.apache.kafka.common.TopicPartition; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.streaming.kafka010.KafkaTestUtils; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import java.net.URL; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; import java.util.Objects; -import java.util.Properties; import java.util.UUID; -import static org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen.Config.ENABLE_KAFKA_COMMIT_OFFSET; import static org.apache.hudi.utilities.testutils.UtilitiesTestBase.Helpers.jsonifyRecords; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertNotNull; -import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.mockito.Mockito.mock; /** * Tests against {@link JsonKafkaSource}. */ -public class TestJsonKafkaSource extends SparkClientFunctionalTestHarness { - - protected static final String TEST_TOPIC_PREFIX = "hoodie_test_"; - private static final URL SCHEMA_FILE_URL = TestJsonKafkaSource.class.getClassLoader().getResource("delta-streamer-config/source.avsc"); - protected static KafkaTestUtils testUtils; - - protected final HoodieDeltaStreamerMetrics metrics = mock(HoodieDeltaStreamerMetrics.class); - protected FilebasedSchemaProvider schemaProvider; - - @BeforeAll - public static void initClass() throws Exception { - testUtils = new KafkaTestUtils(); - testUtils.setup(); - } - - @AfterAll - public static void cleanupClass() { - testUtils.teardown(); - } +public class TestJsonKafkaSource extends BaseTestKafkaSource { + static final URL SCHEMA_FILE_URL = TestJsonKafkaSource.class.getClassLoader().getResource("delta-streamer-config/source.avsc"); @BeforeEach public void init() throws Exception { @@ -90,10 +54,15 @@ public void init() throws Exception { schemaProvider = new FilebasedSchemaProvider(props, jsc()); } - protected TypedProperties createPropsForJsonSource(String topic, Long maxEventsToReadFromKafkaSource, String resetStrategy) { + @Override + TypedProperties createPropsForKafkaSource(String topic, Long maxEventsToReadFromKafkaSource, String resetStrategy) { + return createPropsForJsonKafkaSource(testUtils.brokerAddress(), topic, maxEventsToReadFromKafkaSource, resetStrategy); + } + + static TypedProperties createPropsForJsonKafkaSource(String brokerAddress, String topic, Long maxEventsToReadFromKafkaSource, String resetStrategy) { TypedProperties props = new TypedProperties(); props.setProperty("hoodie.deltastreamer.source.kafka.topic", topic); - props.setProperty("bootstrap.servers", testUtils.brokerAddress()); + props.setProperty("bootstrap.servers", brokerAddress); props.setProperty("auto.offset.reset", resetStrategy); props.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); props.setProperty("hoodie.deltastreamer.kafka.source.maxEvents", @@ -103,53 +72,9 @@ protected TypedProperties createPropsForJsonSource(String topic, Long maxEventsT return props; } - @Test - public void testJsonKafkaSource() { - - // topic setup. - final String topic = TEST_TOPIC_PREFIX + "testJsonKafkaSource"; - testUtils.createTopic(topic, 2); - HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); - TypedProperties props = createPropsForJsonSource(topic, null, "earliest"); - - Source jsonSource = new JsonKafkaSource(props, jsc(), spark(), schemaProvider, metrics); - SourceFormatAdapter kafkaSource = new SourceFormatAdapter(jsonSource); - - // 1. Extract without any checkpoint => get all the data, respecting sourceLimit - assertEquals(Option.empty(), kafkaSource.fetchNewDataInAvroFormat(Option.empty(), Long.MAX_VALUE).getBatch()); - testUtils.sendMessages(topic, jsonifyRecords(dataGenerator.generateInserts("000", 1000))); - InputBatch> fetch1 = kafkaSource.fetchNewDataInAvroFormat(Option.empty(), 900); - assertEquals(900, fetch1.getBatch().get().count()); - // Test Avro To DataFrame path - Dataset fetch1AsRows = AvroConversionUtils.createDataFrame(JavaRDD.toRDD(fetch1.getBatch().get()), - schemaProvider.getSourceSchema().toString(), jsonSource.getSparkSession()); - assertEquals(900, fetch1AsRows.count()); - - // 2. Produce new data, extract new data - testUtils.sendMessages(topic, jsonifyRecords(dataGenerator.generateInserts("001", 1000))); - InputBatch> fetch2 = - kafkaSource.fetchNewDataInRowFormat(Option.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE); - assertEquals(1100, fetch2.getBatch().get().count()); - - // 3. Extract with previous checkpoint => gives same data back (idempotent) - InputBatch> fetch3 = - kafkaSource.fetchNewDataInAvroFormat(Option.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE); - assertEquals(fetch2.getBatch().get().count(), fetch3.getBatch().get().count()); - assertEquals(fetch2.getCheckpointForNextBatch(), fetch3.getCheckpointForNextBatch()); - // Same using Row API - InputBatch> fetch3AsRows = - kafkaSource.fetchNewDataInRowFormat(Option.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE); - assertEquals(fetch2.getBatch().get().count(), fetch3AsRows.getBatch().get().count()); - assertEquals(fetch2.getCheckpointForNextBatch(), fetch3AsRows.getCheckpointForNextBatch()); - - // 4. Extract with latest checkpoint => no new data returned - InputBatch> fetch4 = - kafkaSource.fetchNewDataInAvroFormat(Option.of(fetch2.getCheckpointForNextBatch()), Long.MAX_VALUE); - assertEquals(Option.empty(), fetch4.getBatch()); - // Same using Row API - InputBatch> fetch4AsRows = - kafkaSource.fetchNewDataInRowFormat(Option.of(fetch2.getCheckpointForNextBatch()), Long.MAX_VALUE); - assertEquals(Option.empty(), fetch4AsRows.getBatch()); + @Override + SourceFormatAdapter createSource(TypedProperties props) { + return new SourceFormatAdapter(new JsonKafkaSource(props, jsc(), spark(), schemaProvider, metrics)); } // test whether empty messages can be filtered @@ -159,7 +84,7 @@ public void testJsonKafkaSourceFilterNullMsg() { final String topic = TEST_TOPIC_PREFIX + "testJsonKafkaSourceFilterNullMsg"; testUtils.createTopic(topic, 2); HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); - TypedProperties props = createPropsForJsonSource(topic, null, "earliest"); + TypedProperties props = createPropsForKafkaSource(topic, null, "earliest"); Source jsonSource = new JsonKafkaSource(props, jsc(), spark(), schemaProvider, metrics); SourceFormatAdapter kafkaSource = new SourceFormatAdapter(jsonSource); @@ -175,47 +100,13 @@ public void testJsonKafkaSourceFilterNullMsg() { assertEquals(1000, fetch1.getBatch().get().count()); } - // test case with kafka offset reset strategy - @Test - public void testJsonKafkaSourceResetStrategy() { - // topic setup. - final String topic = TEST_TOPIC_PREFIX + "testJsonKafkaSourceResetStrategy"; - testUtils.createTopic(topic, 2); - HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); - - TypedProperties earliestProps = createPropsForJsonSource(topic, null, "earliest"); - Source earliestJsonSource = new JsonKafkaSource(earliestProps, jsc(), spark(), schemaProvider, metrics); - SourceFormatAdapter earliestKafkaSource = new SourceFormatAdapter(earliestJsonSource); - - TypedProperties latestProps = createPropsForJsonSource(topic, null, "latest"); - Source latestJsonSource = new JsonKafkaSource(latestProps, jsc(), spark(), schemaProvider, metrics); - SourceFormatAdapter latestKafkaSource = new SourceFormatAdapter(latestJsonSource); - - // 1. Extract with a none data kafka checkpoint - // => get a checkpoint string like "hoodie_test,0:0,1:0", latest checkpoint should be equals to earliest checkpoint - InputBatch> earFetch0 = earliestKafkaSource.fetchNewDataInAvroFormat(Option.empty(), Long.MAX_VALUE); - InputBatch> latFetch0 = latestKafkaSource.fetchNewDataInAvroFormat(Option.empty(), Long.MAX_VALUE); - assertEquals(earFetch0.getBatch(), latFetch0.getBatch()); - assertEquals(earFetch0.getCheckpointForNextBatch(), latFetch0.getCheckpointForNextBatch()); - - testUtils.sendMessages(topic, jsonifyRecords(dataGenerator.generateInserts("000", 1000))); - - // 2. Extract new checkpoint with a null / empty string pre checkpoint - // => earliest fetch with max source limit will get all of data and a end offset checkpoint - InputBatch> earFetch1 = earliestKafkaSource.fetchNewDataInAvroFormat(Option.empty(), Long.MAX_VALUE); - - // => [a null pre checkpoint] latest reset fetch will get a end offset checkpoint same to earliest - InputBatch> latFetch1 = latestKafkaSource.fetchNewDataInAvroFormat(Option.empty(), Long.MAX_VALUE); - assertEquals(earFetch1.getCheckpointForNextBatch(), latFetch1.getCheckpointForNextBatch()); - } - @Test public void testJsonKafkaSourceWithDefaultUpperCap() { // topic setup. final String topic = TEST_TOPIC_PREFIX + "testJsonKafkaSourceWithDefaultUpperCap"; testUtils.createTopic(topic, 2); HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); - TypedProperties props = createPropsForJsonSource(topic, Long.MAX_VALUE, "earliest"); + TypedProperties props = createPropsForKafkaSource(topic, Long.MAX_VALUE, "earliest"); Source jsonSource = new JsonKafkaSource(props, jsc(), spark(), schemaProvider, metrics); SourceFormatAdapter kafkaSource = new SourceFormatAdapter(jsonSource); @@ -235,43 +126,13 @@ public void testJsonKafkaSourceWithDefaultUpperCap() { assertEquals(1000, fetch2.getBatch().get().count()); } - @Test - public void testJsonKafkaSourceInsertRecordsLessSourceLimit() { - // topic setup. - final String topic = TEST_TOPIC_PREFIX + "testJsonKafkaSourceInsertRecordsLessSourceLimit"; - testUtils.createTopic(topic, 2); - HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); - TypedProperties props = createPropsForJsonSource(topic, Long.MAX_VALUE, "earliest"); - - Source jsonSource = new JsonKafkaSource(props, jsc(), spark(), schemaProvider, metrics); - SourceFormatAdapter kafkaSource = new SourceFormatAdapter(jsonSource); - props.setProperty("hoodie.deltastreamer.kafka.source.maxEvents", "500"); - - /* - 1. maxEventsFromKafkaSourceProp set to more than generated insert records - and sourceLimit less than the generated insert records num. - */ - testUtils.sendMessages(topic, jsonifyRecords(dataGenerator.generateInserts("000", 400))); - InputBatch> fetch1 = kafkaSource.fetchNewDataInAvroFormat(Option.empty(), 300); - assertEquals(300, fetch1.getBatch().get().count()); - - /* - 2. Produce new data, extract new data based on sourceLimit - and sourceLimit less than the generated insert records num. - */ - testUtils.sendMessages(topic, jsonifyRecords(dataGenerator.generateInserts("001", 600))); - InputBatch> fetch2 = - kafkaSource.fetchNewDataInRowFormat(Option.of(fetch1.getCheckpointForNextBatch()), 300); - assertEquals(300, fetch2.getBatch().get().count()); - } - @Test public void testJsonKafkaSourceWithConfigurableUpperCap() { // topic setup. final String topic = TEST_TOPIC_PREFIX + "testJsonKafkaSourceWithConfigurableUpperCap"; testUtils.createTopic(topic, 2); HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); - TypedProperties props = createPropsForJsonSource(topic, 500L, "earliest"); + TypedProperties props = createPropsForKafkaSource(topic, 500L, "earliest"); Source jsonSource = new JsonKafkaSource(props, jsc(), spark(), schemaProvider, metrics); SourceFormatAdapter kafkaSource = new SourceFormatAdapter(jsonSource); @@ -309,101 +170,9 @@ public void testJsonKafkaSourceWithConfigurableUpperCap() { assertEquals(Option.empty(), fetch6.getBatch()); } - @Test - public void testCommitOffsetToKafka() { - // topic setup. - final String topic = TEST_TOPIC_PREFIX + "testCommitOffsetToKafka"; - testUtils.createTopic(topic, 2); - List topicPartitions = new ArrayList<>(); - TopicPartition topicPartition0 = new TopicPartition(topic, 0); - topicPartitions.add(topicPartition0); - TopicPartition topicPartition1 = new TopicPartition(topic, 1); - topicPartitions.add(topicPartition1); - + @Override + void sendMessagesToKafka(String topic, int count, int numPartitions) { HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); - TypedProperties props = createPropsForJsonSource(topic, null, "earliest"); - props.put(ENABLE_KAFKA_COMMIT_OFFSET.key(), "true"); - Source jsonSource = new JsonKafkaSource(props, jsc(), spark(), schemaProvider, metrics); - SourceFormatAdapter kafkaSource = new SourceFormatAdapter(jsonSource); - - // 1. Extract without any checkpoint => get all the data, respecting sourceLimit - assertEquals(Option.empty(), kafkaSource.fetchNewDataInAvroFormat(Option.empty(), Long.MAX_VALUE).getBatch()); - testUtils.sendMessages(topic, jsonifyRecords(dataGenerator.generateInserts("000", 1000))); - - InputBatch> fetch1 = kafkaSource.fetchNewDataInAvroFormat(Option.empty(), 599); - // commit to kafka after first batch - kafkaSource.getSource().onCommit(fetch1.getCheckpointForNextBatch()); - try (KafkaConsumer consumer = new KafkaConsumer(props)) { - consumer.assign(topicPartitions); - - OffsetAndMetadata offsetAndMetadata = consumer.committed(topicPartition0); - assertNotNull(offsetAndMetadata); - assertEquals(300, offsetAndMetadata.offset()); - offsetAndMetadata = consumer.committed(topicPartition1); - assertNotNull(offsetAndMetadata); - assertEquals(299, offsetAndMetadata.offset()); - // end offsets will point to 500 for each partition because we consumed less messages from first batch - Map endOffsets = consumer.endOffsets(topicPartitions); - assertEquals(500L, endOffsets.get(topicPartition0)); - assertEquals(500L, endOffsets.get(topicPartition1)); - - testUtils.sendMessages(topic, jsonifyRecords(dataGenerator.generateInserts("001", 500))); - InputBatch> fetch2 = - kafkaSource.fetchNewDataInRowFormat(Option.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE); - - // commit to Kafka after second batch is processed completely - kafkaSource.getSource().onCommit(fetch2.getCheckpointForNextBatch()); - - offsetAndMetadata = consumer.committed(topicPartition0); - assertNotNull(offsetAndMetadata); - assertEquals(750, offsetAndMetadata.offset()); - offsetAndMetadata = consumer.committed(topicPartition1); - assertNotNull(offsetAndMetadata); - assertEquals(750, offsetAndMetadata.offset()); - - endOffsets = consumer.endOffsets(topicPartitions); - assertEquals(750L, endOffsets.get(topicPartition0)); - assertEquals(750L, endOffsets.get(topicPartition1)); - } - // check failure case - props.remove(ConsumerConfig.GROUP_ID_CONFIG); - assertThrows(HoodieNotSupportedException.class,() -> kafkaSource.getSource().onCommit("")); - } - - @Test - public void testFailOnDataLoss() throws Exception { - // create a topic with very short retention - final String topic = TEST_TOPIC_PREFIX + "testFailOnDataLoss"; - Properties topicConfig = new Properties(); - topicConfig.setProperty("retention.ms", "10000"); - testUtils.createTopic(topic, 1, topicConfig); - - HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); - TypedProperties failOnDataLossProps = createPropsForJsonSource(topic, null, "earliest"); - failOnDataLossProps.setProperty(Config.ENABLE_FAIL_ON_DATA_LOSS.key(), Boolean.toString(true)); - - Source jsonSource = new JsonKafkaSource(failOnDataLossProps, jsc(), spark(), schemaProvider, metrics); - SourceFormatAdapter kafkaSource = new SourceFormatAdapter(jsonSource); - testUtils.sendMessages(topic, jsonifyRecords(dataGenerator.generateInserts("000", 10))); - // send 10 records, extract 2 records to generate a checkpoint - InputBatch> fetch1 = kafkaSource.fetchNewDataInAvroFormat(Option.empty(), 2); - assertEquals(2, fetch1.getBatch().get().count()); - - // wait for the checkpoint to expire - Thread.sleep(10001); - Throwable t = assertThrows(HoodieDeltaStreamerException.class, () -> { - kafkaSource.fetchNewDataInAvroFormat(Option.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE); - }); - assertEquals( - "Some data may have been lost because they are not available in Kafka any more;" - + " either the data was aged out by Kafka or the topic may have been deleted before all the data in the topic was processed.", - t.getMessage()); - t = assertThrows(HoodieDeltaStreamerException.class, () -> { - kafkaSource.fetchNewDataInRowFormat(Option.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE); - }); - assertEquals( - "Some data may have been lost because they are not available in Kafka any more;" - + " either the data was aged out by Kafka or the topic may have been deleted before all the data in the topic was processed.", - t.getMessage()); + testUtils.sendMessages(topic, jsonifyRecords(dataGenerator.generateInserts("000", count))); } } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJsonKafkaSourcePostProcessor.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJsonKafkaSourcePostProcessor.java index 1d80b68449c7..7ecf12b15ed6 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJsonKafkaSourcePostProcessor.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJsonKafkaSourcePostProcessor.java @@ -25,15 +25,23 @@ import org.apache.hudi.common.util.DateTimeUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.testutils.SparkClientFunctionalTestHarness; +import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamerMetrics; import org.apache.hudi.utilities.deltastreamer.SourceFormatAdapter; import org.apache.hudi.utilities.exception.HoodieSourcePostProcessException; +import org.apache.hudi.utilities.schema.FilebasedSchemaProvider; +import org.apache.hudi.utilities.schema.SchemaProvider; import org.apache.hudi.utilities.sources.processor.JsonKafkaSourcePostProcessor; import org.apache.hudi.utilities.sources.processor.maxwell.MaxwellJsonKafkaSourcePostProcessor; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.avro.generic.GenericRecord; import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.streaming.kafka010.KafkaTestUtils; +import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import java.io.IOException; @@ -48,17 +56,41 @@ import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; -public class TestJsonKafkaSourcePostProcessor extends TestJsonKafkaSource { +public class TestJsonKafkaSourcePostProcessor extends SparkClientFunctionalTestHarness { + private static KafkaTestUtils testUtils; + + private final HoodieDeltaStreamerMetrics metrics = mock(HoodieDeltaStreamerMetrics.class); + private SchemaProvider schemaProvider; + + @BeforeAll + public static void initClass() { + testUtils = new KafkaTestUtils(); + testUtils.setup(); + } + + @AfterAll + public static void cleanupClass() { + testUtils.teardown(); + } + + @BeforeEach + public void init() throws Exception { + String schemaFilePath = Objects.requireNonNull(TestJsonKafkaSource.SCHEMA_FILE_URL).toURI().getPath(); + TypedProperties props = new TypedProperties(); + props.put("hoodie.deltastreamer.schemaprovider.source.schema.file", schemaFilePath); + schemaProvider = new FilebasedSchemaProvider(props, jsc()); + } @Test public void testNoPostProcessor() { // topic setup. - final String topic = TEST_TOPIC_PREFIX + "testNoPostProcessor"; + final String topic = BaseTestKafkaSource.TEST_TOPIC_PREFIX + "testNoPostProcessor"; testUtils.createTopic(topic, 2); HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); - TypedProperties props = createPropsForJsonSource(topic, null, "earliest"); + TypedProperties props = TestJsonKafkaSource.createPropsForJsonKafkaSource(testUtils.brokerAddress(), topic, null, "earliest"); Source jsonSource = new JsonKafkaSource(props, jsc(), spark(), schemaProvider, metrics); SourceFormatAdapter kafkaSource = new SourceFormatAdapter(jsonSource); @@ -72,11 +104,11 @@ public void testNoPostProcessor() { @Test public void testSampleJsonKafkaSourcePostProcessor() { // topic setup. - final String topic = TEST_TOPIC_PREFIX + "testSampleJsonKafkaSourcePostProcessor"; + final String topic = BaseTestKafkaSource.TEST_TOPIC_PREFIX + "testSampleJsonKafkaSourcePostProcessor"; testUtils.createTopic(topic, 2); HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); - TypedProperties props = createPropsForJsonSource(topic, null, "earliest"); + TypedProperties props = TestJsonKafkaSource.createPropsForJsonKafkaSource(testUtils.brokerAddress(), topic, null, "earliest"); // processor class name setup props.setProperty(JSON_KAFKA_PROCESSOR_CLASS_OPT.key(), SampleJsonKafkaSourcePostProcessor.class.getName()); @@ -93,11 +125,11 @@ public void testSampleJsonKafkaSourcePostProcessor() { @Test public void testInvalidJsonKafkaSourcePostProcessor() { // topic setup. - final String topic = TEST_TOPIC_PREFIX + "testInvalidJsonKafkaSourcePostProcessor"; + final String topic = BaseTestKafkaSource.TEST_TOPIC_PREFIX + "testInvalidJsonKafkaSourcePostProcessor"; testUtils.createTopic(topic, 2); HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); - TypedProperties props = createPropsForJsonSource(topic, null, "earliest"); + TypedProperties props = TestJsonKafkaSource.createPropsForJsonKafkaSource(testUtils.brokerAddress(), topic, null, "earliest"); // processor class name setup props.setProperty(JSON_KAFKA_PROCESSOR_CLASS_OPT.key(), "InvalidJsonKafkaSourcePostProcessor"); @@ -113,11 +145,11 @@ public void testInvalidJsonKafkaSourcePostProcessor() { @Test public void testChainedJsonKafkaSourcePostProcessor() { // topic setup. - final String topic = TEST_TOPIC_PREFIX + "testChainedJsonKafkaSourcePostProcessor"; + final String topic = BaseTestKafkaSource.TEST_TOPIC_PREFIX + "testChainedJsonKafkaSourcePostProcessor"; testUtils.createTopic(topic, 2); HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); - TypedProperties props = createPropsForJsonSource(topic, null, "earliest"); + TypedProperties props = TestJsonKafkaSource.createPropsForJsonKafkaSource(testUtils.brokerAddress(), topic, null, "earliest"); // processor class name setup props.setProperty(JSON_KAFKA_PROCESSOR_CLASS_OPT.key(), SampleJsonKafkaSourcePostProcessor.class.getName() diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestProtoKafkaSource.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestProtoKafkaSource.java new file mode 100644 index 000000000000..0b2834623de0 --- /dev/null +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestProtoKafkaSource.java @@ -0,0 +1,216 @@ +/* + * 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.hudi.utilities.sources; + +import org.apache.hudi.AvroConversionUtils; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.utilities.deltastreamer.SourceFormatAdapter; +import org.apache.hudi.utilities.schema.ProtoClassBasedSchemaProvider; +import org.apache.hudi.utilities.schema.SchemaProvider; +import org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen.Config; +import org.apache.hudi.utilities.test.proto.Nested; +import org.apache.hudi.utilities.test.proto.Sample; +import org.apache.hudi.utilities.test.proto.SampleEnum; + +import com.google.protobuf.BoolValue; +import com.google.protobuf.ByteString; +import com.google.protobuf.BytesValue; +import com.google.protobuf.DoubleValue; +import com.google.protobuf.FloatValue; +import com.google.protobuf.Int32Value; +import com.google.protobuf.Int64Value; +import com.google.protobuf.StringValue; +import com.google.protobuf.UInt32Value; +import com.google.protobuf.UInt64Value; +import com.google.protobuf.util.Timestamps; +import org.apache.avro.generic.GenericRecord; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Random; +import java.util.UUID; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +/** + * Tests against {@link ProtoKafkaSource}. + */ +public class TestProtoKafkaSource extends BaseTestKafkaSource { + private static final Random RANDOM = new Random(); + + protected TypedProperties createPropsForKafkaSource(String topic, Long maxEventsToReadFromKafkaSource, String resetStrategy) { + TypedProperties props = new TypedProperties(); + props.setProperty("hoodie.deltastreamer.source.kafka.topic", topic); + props.setProperty("bootstrap.servers", testUtils.brokerAddress()); + props.setProperty("auto.offset.reset", resetStrategy); + props.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); + props.setProperty("hoodie.deltastreamer.kafka.source.maxEvents", + maxEventsToReadFromKafkaSource != null ? String.valueOf(maxEventsToReadFromKafkaSource) : + String.valueOf(Config.MAX_EVENTS_FROM_KAFKA_SOURCE_PROP.defaultValue())); + props.setProperty(ConsumerConfig.GROUP_ID_CONFIG, UUID.randomUUID().toString()); + props.setProperty(ProtoClassBasedSchemaProvider.Config.PROTO_SCHEMA_CLASS_NAME, Sample.class.getName()); + return props; + } + + @Override + SourceFormatAdapter createSource(TypedProperties props) { + this.schemaProvider = new ProtoClassBasedSchemaProvider(props, jsc()); + Source protoKafkaSource = new ProtoKafkaSource(props, jsc(), spark(), schemaProvider, metrics); + return new SourceFormatAdapter(protoKafkaSource); + } + + @Test + public void testProtoKafkaSourceWithFlattenWrappedPrimitives() { + + // topic setup. + final String topic = TEST_TOPIC_PREFIX + "testProtoKafkaSourceFlatten"; + testUtils.createTopic(topic, 2); + TypedProperties props = createPropsForKafkaSource(topic, null, "earliest"); + props.setProperty(ProtoClassBasedSchemaProvider.Config.PROTO_SCHEMA_FLATTEN_WRAPPED_PRIMITIVES, "true"); + SchemaProvider schemaProvider = new ProtoClassBasedSchemaProvider(props, jsc()); + Source protoKafkaSource = new ProtoKafkaSource(props, jsc(), spark(), schemaProvider, metrics); + SourceFormatAdapter kafkaSource = new SourceFormatAdapter(protoKafkaSource); + + // 1. Extract without any checkpoint => get all the data, respecting sourceLimit + assertEquals(Option.empty(), kafkaSource.fetchNewDataInAvroFormat(Option.empty(), Long.MAX_VALUE).getBatch()); + sendMessagesToKafka(topic, 1000, 2); + InputBatch> fetch1 = kafkaSource.fetchNewDataInAvroFormat(Option.empty(), 900); + assertEquals(900, fetch1.getBatch().get().count()); + // Test Avro To DataFrame path + Dataset fetch1AsRows = AvroConversionUtils.createDataFrame(JavaRDD.toRDD(fetch1.getBatch().get()), + schemaProvider.getSourceSchema().toString(), protoKafkaSource.getSparkSession()); + assertEquals(900, fetch1AsRows.count()); + + // 2. Produce new data, extract new data + sendMessagesToKafka(topic, 1000, 2); + InputBatch> fetch2 = + kafkaSource.fetchNewDataInRowFormat(Option.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE); + assertEquals(1100, fetch2.getBatch().get().count()); + + // 3. Extract with previous checkpoint => gives same data back (idempotent) + InputBatch> fetch3 = + kafkaSource.fetchNewDataInAvroFormat(Option.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE); + assertEquals(fetch2.getBatch().get().count(), fetch3.getBatch().get().count()); + assertEquals(fetch2.getCheckpointForNextBatch(), fetch3.getCheckpointForNextBatch()); + // Same using Row API + InputBatch> fetch3AsRows = + kafkaSource.fetchNewDataInRowFormat(Option.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE); + assertEquals(fetch2.getBatch().get().count(), fetch3AsRows.getBatch().get().count()); + assertEquals(fetch2.getCheckpointForNextBatch(), fetch3AsRows.getCheckpointForNextBatch()); + + // 4. Extract with latest checkpoint => no new data returned + InputBatch> fetch4 = + kafkaSource.fetchNewDataInAvroFormat(Option.of(fetch2.getCheckpointForNextBatch()), Long.MAX_VALUE); + assertEquals(Option.empty(), fetch4.getBatch()); + // Same using Row API + InputBatch> fetch4AsRows = + kafkaSource.fetchNewDataInRowFormat(Option.of(fetch2.getCheckpointForNextBatch()), Long.MAX_VALUE); + assertEquals(Option.empty(), fetch4AsRows.getBatch()); + } + + private static List createSampleMessages(int count) { + return IntStream.range(0, count).mapToObj(unused -> { + Sample.Builder builder = Sample.newBuilder() + .setPrimitiveDouble(RANDOM.nextDouble()) + .setPrimitiveFloat(RANDOM.nextFloat()) + .setPrimitiveInt(RANDOM.nextInt()) + .setPrimitiveLong(RANDOM.nextLong()) + .setPrimitiveUnsignedInt(RANDOM.nextInt()) + .setPrimitiveUnsignedLong(RANDOM.nextLong()) + .setPrimitiveSignedInt(RANDOM.nextInt()) + .setPrimitiveSignedLong(RANDOM.nextLong()) + .setPrimitiveFixedInt(RANDOM.nextInt()) + .setPrimitiveFixedLong(RANDOM.nextLong()) + .setPrimitiveFixedSignedInt(RANDOM.nextInt()) + .setPrimitiveFixedSignedLong(RANDOM.nextLong()) + .setPrimitiveBoolean(RANDOM.nextBoolean()) + .setPrimitiveString(UUID.randomUUID().toString()) + .setPrimitiveBytes(ByteString.copyFrom(UUID.randomUUID().toString().getBytes())); + + // randomly set nested messages, lists, and maps to test edge cases + if (RANDOM.nextBoolean()) { + Map primitiveMap = new HashMap<>(); + primitiveMap.put(UUID.randomUUID().toString(), RANDOM.nextInt()); + Map messageMap = new HashMap<>(); + messageMap.put(UUID.randomUUID().toString(), generateRandomNestedMessage()); + builder.addAllRepeatedPrimitive(Arrays.asList(RANDOM.nextInt(), RANDOM.nextInt())) + .putAllMapPrimitive(primitiveMap) + .setNestedMessage(generateRandomNestedMessage()) + .addAllRepeatedMessage(Arrays.asList(generateRandomNestedMessage(), generateRandomNestedMessage())) + .putAllMapMessage(messageMap) + .setWrappedString(StringValue.of(UUID.randomUUID().toString())) + .setWrappedInt(Int32Value.of(RANDOM.nextInt())) + .setWrappedLong(Int64Value.of(RANDOM.nextLong())) + .setWrappedUnsignedInt(UInt32Value.of(RANDOM.nextInt())) + .setWrappedUnsignedLong(UInt64Value.of(RANDOM.nextLong())) + .setWrappedDouble(DoubleValue.of(RANDOM.nextDouble())) + .setWrappedFloat(FloatValue.of(RANDOM.nextFloat())) + .setWrappedBoolean(BoolValue.of(RANDOM.nextBoolean())) + .setWrappedBytes(BytesValue.of(ByteString.copyFrom(UUID.randomUUID().toString().getBytes()))) + .setEnum(SampleEnum.SECOND) + .setTimestamp(Timestamps.fromMillis(System.currentTimeMillis())); + } + return builder.build(); + }).collect(Collectors.toList()); + } + + private static Nested generateRandomNestedMessage() { + return Nested.newBuilder() + .setNestedInt(RANDOM.nextInt()) + .build(); + } + + @Override + void sendMessagesToKafka(String topic, int count, int numPartitions) { + List messages = createSampleMessages(count); + try (Producer producer = new KafkaProducer<>(getProducerProperties())) { + for (int i = 0; i < messages.size(); i++) { + // use consistent keys to get even spread over partitions for test expectations + producer.send(new ProducerRecord<>(topic, Integer.toString(i % numPartitions), messages.get(i).toByteArray())); + } + } + } + + private Properties getProducerProperties() { + Properties props = new Properties(); + props.put("bootstrap.servers", testUtils.brokerAddress()); + props.put("value.serializer", ByteArraySerializer.class.getName()); + // Key serializer is required. + props.put("key.serializer", StringSerializer.class.getName()); + // wait for all in-sync replicas to ack sends + props.put("acks", "all"); + return props; + } +} diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/helpers/TestProtoConversionUtil.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/helpers/TestProtoConversionUtil.java new file mode 100644 index 000000000000..937d914e1aea --- /dev/null +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/helpers/TestProtoConversionUtil.java @@ -0,0 +1,367 @@ +/* + * 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.hudi.utilities.sources.helpers; + +import org.apache.hudi.utilities.test.proto.Nested; +import org.apache.hudi.utilities.test.proto.Sample; +import org.apache.hudi.utilities.test.proto.SampleEnum; + +import com.google.protobuf.BoolValue; +import com.google.protobuf.ByteString; +import com.google.protobuf.BytesValue; +import com.google.protobuf.DoubleValue; +import com.google.protobuf.FloatValue; +import com.google.protobuf.Int32Value; +import com.google.protobuf.Int64Value; +import com.google.protobuf.StringValue; +import com.google.protobuf.Timestamp; +import com.google.protobuf.UInt32Value; +import com.google.protobuf.UInt64Value; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import com.google.protobuf.util.Timestamps; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Scanner; +import java.util.function.Function; +import java.util.stream.Collectors; + +public class TestProtoConversionUtil { + @Test + public void allFieldsSet_wellKnownTypesAreNested() { + List primitiveList = Arrays.asList(1, 2, 3); + Map primitiveMap = new HashMap<>(); + primitiveMap.put("key1", 1); + primitiveMap.put("key2", 2); + + Nested nestedMessage = Nested.newBuilder().setNestedInt(1).build(); + List nestedList = Arrays.asList(Nested.newBuilder().setNestedInt(2).build(), Nested.newBuilder().setNestedInt(3).build()); + Map nestedMap = new HashMap<>(); + nestedMap.put("1Key", Nested.newBuilder().setNestedInt(123).build()); + nestedMap.put("2Key", Nested.newBuilder().setNestedInt(321).build()); + Timestamp time = Timestamps.fromMillis(System.currentTimeMillis()); + + Sample sample = Sample.newBuilder() + .setPrimitiveDouble(1.1) + .setPrimitiveFloat(2.1f) + .setPrimitiveInt(1) + .setPrimitiveLong(2L) + .setPrimitiveUnsignedInt(3) + .setPrimitiveUnsignedLong(4L) + .setPrimitiveSignedInt(5) + .setPrimitiveSignedLong(6L) + .setPrimitiveFixedInt(7) + .setPrimitiveFixedLong(8L) + .setPrimitiveFixedSignedInt(9) + .setPrimitiveFixedSignedLong(10L) + .setPrimitiveBoolean(true) + .setPrimitiveString("I am a string!") + .setPrimitiveBytes(ByteString.copyFrom("I am just bytes".getBytes())) + .addAllRepeatedPrimitive(primitiveList) + .putAllMapPrimitive(primitiveMap) + .setNestedMessage(nestedMessage) + .addAllRepeatedMessage(nestedList) + .putAllMapMessage(nestedMap) + .setWrappedString(StringValue.of("I am a wrapped string")) + .setWrappedInt(Int32Value.of(11)) + .setWrappedLong(Int64Value.of(12L)) + .setWrappedUnsignedInt(UInt32Value.of(13)) + .setWrappedUnsignedLong(UInt64Value.of(14L)) + .setWrappedDouble(DoubleValue.of(15.5)) + .setWrappedFloat(FloatValue.of(16.6f)) + .setWrappedBoolean(BoolValue.of(true)) + .setWrappedBytes(BytesValue.of(ByteString.copyFrom("I am wrapped bytes".getBytes()))) + .setEnum(SampleEnum.SECOND) + .setTimestamp(time) + .build(); + Schema.Parser parser = new Schema.Parser(); + Schema protoSchema = parser.parse(getSchema("schema-provider/proto/sample_schema_nested.txt")); + + GenericRecord actual = ProtoConversionUtil.convertToAvro(protoSchema, sample); + + Schema nestedMessageSchema = protoSchema.getField("nested_message").schema().getTypes().get(1); + + GenericData.Record wrappedStringRecord = getWrappedRecord(protoSchema, "wrapped_string", "I am a wrapped string"); + GenericData.Record wrappedIntRecord = getWrappedRecord(protoSchema, "wrapped_int", 11); + GenericData.Record wrappedLongRecord = getWrappedRecord(protoSchema, "wrapped_long", 12L); + GenericData.Record wrappedUIntRecord = getWrappedRecord(protoSchema, "wrapped_unsigned_int", 13L); + GenericData.Record wrappedULongRecord = getWrappedRecord(protoSchema, "wrapped_unsigned_long", 14L); + GenericData.Record wrappedDoubleRecord = getWrappedRecord(protoSchema, "wrapped_double", 15.5); + GenericData.Record wrappedFloatRecord = getWrappedRecord(protoSchema, "wrapped_float", 16.6f); + GenericData.Record wrappedBooleanRecord = getWrappedRecord(protoSchema, "wrapped_boolean", true); + GenericData.Record wrappedBytesRecord = getWrappedRecord(protoSchema, "wrapped_bytes", ByteBuffer.wrap("I am wrapped bytes".getBytes())); + + GenericData.Record expectedRecord = new GenericData.Record(protoSchema); + expectedRecord.put("primitive_double", 1.1); + expectedRecord.put("primitive_float", 2.1f); + expectedRecord.put("primitive_int", 1); + expectedRecord.put("primitive_long", 2L); + expectedRecord.put("primitive_unsigned_int", 3L); + expectedRecord.put("primitive_unsigned_long", 4L); + expectedRecord.put("primitive_signed_int", 5); + expectedRecord.put("primitive_signed_long", 6L); + expectedRecord.put("primitive_fixed_int", 7); + expectedRecord.put("primitive_fixed_long", 8L); + expectedRecord.put("primitive_fixed_signed_int", 9); + expectedRecord.put("primitive_fixed_signed_long", 10L); + expectedRecord.put("primitive_boolean", true); + expectedRecord.put("primitive_string", "I am a string!"); + expectedRecord.put("primitive_bytes", ByteBuffer.wrap("I am just bytes".getBytes())); + expectedRecord.put("repeated_primitive", primitiveList); + expectedRecord.put("map_primitive", convertMapToList(protoSchema, "map_primitive", primitiveMap)); + expectedRecord.put("nested_message", convertNestedMessage(nestedMessageSchema, nestedMessage)); + expectedRecord.put("repeated_message", nestedList.stream().map(m -> convertNestedMessage(nestedMessageSchema, m)).collect(Collectors.toList())); + expectedRecord.put("map_message", convertMapToList(protoSchema, "map_message", nestedMap, value -> convertNestedMessage(nestedMessageSchema, value))); + expectedRecord.put("wrapped_string", wrappedStringRecord); + expectedRecord.put("wrapped_int", wrappedIntRecord); + expectedRecord.put("wrapped_long", wrappedLongRecord); + expectedRecord.put("wrapped_unsigned_int", wrappedUIntRecord); + expectedRecord.put("wrapped_unsigned_long", wrappedULongRecord); + expectedRecord.put("wrapped_double", wrappedDoubleRecord); + expectedRecord.put("wrapped_float", wrappedFloatRecord); + expectedRecord.put("wrapped_boolean", wrappedBooleanRecord); + expectedRecord.put("wrapped_bytes", wrappedBytesRecord); + expectedRecord.put("enum", SampleEnum.SECOND.name()); + expectedRecord.put("timestamp", getTimestampRecord(protoSchema, time)); + Assertions.assertEquals(expectedRecord, actual); + } + + @Test + public void noFieldsSet_wellKnownTypesAreNested() { + Sample sample = Sample.newBuilder().build(); + Schema.Parser parser = new Schema.Parser(); + Schema protoSchema = parser.parse(getSchema("schema-provider/proto/sample_schema_nested.txt")); + + GenericRecord actual = ProtoConversionUtil.convertToAvro(protoSchema, sample); + + // all fields will have default values + GenericData.Record expectedRecord = new GenericData.Record(protoSchema); + expectedRecord.put("primitive_double", 0.0); + expectedRecord.put("primitive_float", 0.0f); + expectedRecord.put("primitive_int", 0); + expectedRecord.put("primitive_long", 0L); + expectedRecord.put("primitive_unsigned_int", 0L); + expectedRecord.put("primitive_unsigned_long", 0L); + expectedRecord.put("primitive_signed_int", 0); + expectedRecord.put("primitive_signed_long", 0L); + expectedRecord.put("primitive_fixed_int", 0); + expectedRecord.put("primitive_fixed_long", 0L); + expectedRecord.put("primitive_fixed_signed_int", 0); + expectedRecord.put("primitive_fixed_signed_long", 0L); + expectedRecord.put("primitive_boolean", false); + expectedRecord.put("primitive_string", ""); + expectedRecord.put("primitive_bytes", ByteBuffer.wrap("".getBytes())); + expectedRecord.put("repeated_primitive", Collections.emptyList()); + expectedRecord.put("map_primitive", Collections.emptyList()); + expectedRecord.put("nested_message", null); + expectedRecord.put("repeated_message", Collections.emptyList()); + expectedRecord.put("map_message", Collections.emptyList()); + expectedRecord.put("wrapped_string", null); + expectedRecord.put("wrapped_int", null); + expectedRecord.put("wrapped_long", null); + expectedRecord.put("wrapped_unsigned_int", null); + expectedRecord.put("wrapped_unsigned_long", null); + expectedRecord.put("wrapped_double", null); + expectedRecord.put("wrapped_float", null); + expectedRecord.put("wrapped_boolean", null); + expectedRecord.put("wrapped_bytes", null); + expectedRecord.put("enum", SampleEnum.FIRST.name()); + expectedRecord.put("timestamp", null); + Assertions.assertEquals(expectedRecord, actual); + } + + @Test + public void allFieldsSet_wellKnownTypesAreFlattened() { + List primitiveList = Arrays.asList(1, 2, 3); + Map primitiveMap = new HashMap<>(); + primitiveMap.put("key1", 1); + primitiveMap.put("key2", 2); + + Nested nestedMessage = Nested.newBuilder().setNestedInt(1).build(); + List nestedList = Arrays.asList(Nested.newBuilder().setNestedInt(2).build(), Nested.newBuilder().setNestedInt(3).build()); + Map nestedMap = new HashMap<>(); + nestedMap.put("1Key", Nested.newBuilder().setNestedInt(123).build()); + nestedMap.put("2Key", Nested.newBuilder().setNestedInt(321).build()); + Timestamp time = Timestamps.fromMillis(System.currentTimeMillis()); + + Sample sample = Sample.newBuilder() + .setPrimitiveDouble(1.1) + .setPrimitiveFloat(2.1f) + .setPrimitiveInt(1) + .setPrimitiveLong(2L) + .setPrimitiveUnsignedInt(3) + .setPrimitiveUnsignedLong(4L) + .setPrimitiveSignedInt(5) + .setPrimitiveSignedLong(6L) + .setPrimitiveFixedInt(7) + .setPrimitiveFixedLong(8L) + .setPrimitiveFixedSignedInt(9) + .setPrimitiveFixedSignedLong(10L) + .setPrimitiveBoolean(true) + .setPrimitiveString("I am a string!") + .setPrimitiveBytes(ByteString.copyFrom("I am just bytes".getBytes())) + .addAllRepeatedPrimitive(primitiveList) + .putAllMapPrimitive(primitiveMap) + .setNestedMessage(nestedMessage) + .addAllRepeatedMessage(nestedList) + .putAllMapMessage(nestedMap) + .setWrappedString(StringValue.of("I am a wrapped string")) + .setWrappedInt(Int32Value.of(11)) + .setWrappedLong(Int64Value.of(12L)) + .setWrappedUnsignedInt(UInt32Value.of(13)) + .setWrappedUnsignedLong(UInt64Value.of(14L)) + .setWrappedDouble(DoubleValue.of(15.5)) + .setWrappedFloat(FloatValue.of(16.6f)) + .setWrappedBoolean(BoolValue.of(true)) + .setWrappedBytes(BytesValue.of(ByteString.copyFrom("I am wrapped bytes".getBytes()))) + .setEnum(SampleEnum.SECOND) + .setTimestamp(time) + .build(); + Schema.Parser parser = new Schema.Parser(); + Schema protoSchema = parser.parse(getSchema("schema-provider/proto/sample_schema_flattened.txt")); + + GenericRecord actual = ProtoConversionUtil.convertToAvro(protoSchema, sample); + + Schema nestedMessageSchema = protoSchema.getField("nested_message").schema().getTypes().get(1); + + GenericData.Record expectedRecord = new GenericData.Record(protoSchema); + expectedRecord.put("primitive_double", 1.1); + expectedRecord.put("primitive_float", 2.1f); + expectedRecord.put("primitive_int", 1); + expectedRecord.put("primitive_long", 2L); + expectedRecord.put("primitive_unsigned_int", 3L); + expectedRecord.put("primitive_unsigned_long", 4L); + expectedRecord.put("primitive_signed_int", 5); + expectedRecord.put("primitive_signed_long", 6L); + expectedRecord.put("primitive_fixed_int", 7); + expectedRecord.put("primitive_fixed_long", 8L); + expectedRecord.put("primitive_fixed_signed_int", 9); + expectedRecord.put("primitive_fixed_signed_long", 10L); + expectedRecord.put("primitive_boolean", true); + expectedRecord.put("primitive_string", "I am a string!"); + expectedRecord.put("primitive_bytes", ByteBuffer.wrap("I am just bytes".getBytes())); + expectedRecord.put("repeated_primitive", primitiveList); + expectedRecord.put("map_primitive", convertMapToList(protoSchema, "map_primitive", primitiveMap)); + expectedRecord.put("nested_message", convertNestedMessage(nestedMessageSchema, nestedMessage)); + expectedRecord.put("repeated_message", nestedList.stream().map(m -> convertNestedMessage(nestedMessageSchema, m)).collect(Collectors.toList())); + expectedRecord.put("map_message", convertMapToList(protoSchema, "map_message", nestedMap, value -> convertNestedMessage(nestedMessageSchema, value))); + expectedRecord.put("wrapped_string", "I am a wrapped string"); + expectedRecord.put("wrapped_int", 11); + expectedRecord.put("wrapped_long", 12L); + expectedRecord.put("wrapped_unsigned_int", 13L); + expectedRecord.put("wrapped_unsigned_long", 14L); + expectedRecord.put("wrapped_double", 15.5); + expectedRecord.put("wrapped_float", 16.6f); + expectedRecord.put("wrapped_boolean", true); + expectedRecord.put("wrapped_bytes", ByteBuffer.wrap("I am wrapped bytes".getBytes())); + expectedRecord.put("enum", SampleEnum.SECOND.name()); + expectedRecord.put("timestamp", getTimestampRecord(protoSchema, time)); + Assertions.assertEquals(expectedRecord, actual); + } + + @Test + public void noFieldsSet_wellKnownTypesAreFlattened() { + Sample sample = Sample.newBuilder().build(); + Schema.Parser parser = new Schema.Parser(); + Schema protoSchema = parser.parse(getSchema("schema-provider/proto/sample_schema_flattened.txt")); + + GenericRecord actual = ProtoConversionUtil.convertToAvro(protoSchema, sample); + + // all fields will have default values + GenericData.Record expectedRecord = new GenericData.Record(protoSchema); + expectedRecord.put("primitive_double", 0.0); + expectedRecord.put("primitive_float", 0.0f); + expectedRecord.put("primitive_int", 0); + expectedRecord.put("primitive_long", 0L); + expectedRecord.put("primitive_unsigned_int", 0L); + expectedRecord.put("primitive_unsigned_long", 0L); + expectedRecord.put("primitive_signed_int", 0); + expectedRecord.put("primitive_signed_long", 0L); + expectedRecord.put("primitive_fixed_int", 0); + expectedRecord.put("primitive_fixed_long", 0L); + expectedRecord.put("primitive_fixed_signed_int", 0); + expectedRecord.put("primitive_fixed_signed_long", 0L); + expectedRecord.put("primitive_boolean", false); + expectedRecord.put("primitive_string", ""); + expectedRecord.put("primitive_bytes", ByteBuffer.wrap("".getBytes())); + expectedRecord.put("repeated_primitive", Collections.emptyList()); + expectedRecord.put("map_primitive", Collections.emptyList()); + expectedRecord.put("nested_message", null); + expectedRecord.put("repeated_message", Collections.emptyList()); + expectedRecord.put("map_message", Collections.emptyList()); + expectedRecord.put("wrapped_string", null); + expectedRecord.put("wrapped_int", null); + expectedRecord.put("wrapped_long", null); + expectedRecord.put("wrapped_unsigned_int", null); + expectedRecord.put("wrapped_unsigned_long", null); + expectedRecord.put("wrapped_double", null); + expectedRecord.put("wrapped_float", null); + expectedRecord.put("wrapped_boolean", null); + expectedRecord.put("wrapped_bytes", null); + expectedRecord.put("enum", SampleEnum.FIRST.name()); + expectedRecord.put("timestamp", null); + Assertions.assertEquals(expectedRecord, actual); + } + + private GenericData.Record getTimestampRecord(Schema protoSchema, Timestamp time) { + GenericData.Record timestampRecord = new GenericData.Record(protoSchema.getField("timestamp").schema().getTypes().get(1)); + timestampRecord.put("seconds", time.getSeconds()); + timestampRecord.put("nanos", time.getNanos()); + return timestampRecord; + } + + private GenericData.Record getWrappedRecord(Schema protoSchema, String fieldName, Object value) { + GenericData.Record wrappedRecord = new GenericData.Record(protoSchema.getField(fieldName).schema().getTypes().get(1)); + wrappedRecord.put("value", value); + return wrappedRecord; + } + + private GenericRecord convertNestedMessage(final Schema schema, Nested message) { + GenericData.Record record = new GenericData.Record(schema); + record.put("nested_int", message.getNestedInt()); + return record; + } + + private static List convertMapToList(final Schema protoSchema, final String fieldName, final Map originalMap, final Function valueConverter) { + return originalMap.entrySet().stream().map(entry -> { + GenericData.Record record = new GenericData.Record(protoSchema.getField(fieldName).schema().getElementType()); + record.put("key", entry.getKey()); + record.put("value", valueConverter.apply(entry.getValue())); + return record; + }).collect(Collectors.toList()); + } + + private static List convertMapToList(final Schema protoSchema, final String fieldName, final Map originalMap) { + return convertMapToList(protoSchema, fieldName, originalMap, Function.identity()); + } + + private String getSchema(String pathToSchema) { + try (Scanner scanner = new Scanner(getClass().getClassLoader().getResourceAsStream(pathToSchema))) { + return scanner.next(); + } + } +} diff --git a/hudi-utilities/src/test/resources/schema-provider/proto/sample.proto b/hudi-utilities/src/test/resources/schema-provider/proto/sample.proto new file mode 100644 index 000000000000..3ade2942c6bf --- /dev/null +++ b/hudi-utilities/src/test/resources/schema-provider/proto/sample.proto @@ -0,0 +1,77 @@ +/* + * 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. + * + */ + +syntax = "proto3"; + +package test; + +option java_package = "org.apache.hudi.utilities.test.proto"; +option java_multiple_files = true; + +import "google/protobuf/timestamp.proto"; +import "google/protobuf/wrappers.proto"; + +// Creates a message to cover all the various types you can encounter in a proto3 message +message Sample { + // primitive types + double primitive_double = 1; + float primitive_float = 2; + int32 primitive_int = 3; + int64 primitive_long = 4; + uint32 primitive_unsigned_int = 5; + uint64 primitive_unsigned_long = 6; + sint32 primitive_signed_int = 7; + sint64 primitive_signed_long = 8; + fixed32 primitive_fixed_int = 9; + fixed64 primitive_fixed_long = 10; + sfixed32 primitive_fixed_signed_int = 11; + sfixed64 primitive_fixed_signed_long = 12; + bool primitive_boolean = 13; + string primitive_string = 14; + bytes primitive_bytes = 15; + repeated int32 repeated_primitive = 16; + map map_primitive = 17; + // nested custom message + Nested nested_message = 18; + repeated Nested repeated_message = 19; + map map_message = 20; + // well known types + google.protobuf.StringValue wrapped_string = 21; + google.protobuf.Int32Value wrapped_int = 22; + google.protobuf.Int64Value wrapped_long = 23; + google.protobuf.UInt32Value wrapped_unsigned_int = 24; + google.protobuf.UInt64Value wrapped_unsigned_long = 25; + google.protobuf.DoubleValue wrapped_double = 26; + google.protobuf.FloatValue wrapped_float = 27; + google.protobuf.BoolValue wrapped_boolean = 28; + google.protobuf.BytesValue wrapped_bytes = 29; + // enum + SampleEnum enum = 30; + // timestamp + google.protobuf.Timestamp timestamp = 31; +} + +message Nested { + int32 nested_int = 1; +} + +enum SampleEnum { + FIRST = 0; + SECOND = 1; +} \ No newline at end of file diff --git a/hudi-utilities/src/test/resources/schema-provider/proto/sample_schema_flattened.txt b/hudi-utilities/src/test/resources/schema-provider/proto/sample_schema_flattened.txt new file mode 100644 index 000000000000..f3d372486d9e --- /dev/null +++ b/hudi-utilities/src/test/resources/schema-provider/proto/sample_schema_flattened.txt @@ -0,0 +1 @@ +{"type":"record","name":"Sample","namespace":"test","fields":[{"name":"primitive_double","type":"double","default":0.0},{"name":"primitive_float","type":"float","default":0.0},{"name":"primitive_int","type":"int","default":0},{"name":"primitive_long","type":"long","default":0},{"name":"primitive_unsigned_int","type":"long","default":0},{"name":"primitive_unsigned_long","type":"long","default":0},{"name":"primitive_signed_int","type":"int","default":0},{"name":"primitive_signed_long","type":"long","default":0},{"name":"primitive_fixed_int","type":"int","default":0},{"name":"primitive_fixed_long","type":"long","default":0},{"name":"primitive_fixed_signed_int","type":"int","default":0},{"name":"primitive_fixed_signed_long","type":"long","default":0},{"name":"primitive_boolean","type":"boolean","default":false},{"name":"primitive_string","type":{"type":"string","avro.java.string":"String"},"default":""},{"name":"primitive_bytes","type":"bytes","default":""},{"name":"repeated_primitive","type":{"type":"array","items":"int"},"default":[]},{"name":"map_primitive","type":{"type":"array","items":{"type":"record","name":"MapPrimitiveEntry","namespace":"test.Sample","fields":[{"name":"key","type":{"type":"string","avro.java.string":"String"},"default":""},{"name":"value","type":"int","default":0}]}},"default":[]},{"name":"nested_message","type":["null",{"type":"record","name":"Nested","fields":[{"name":"nested_int","type":"int","default":0}]}],"default":null},{"name":"repeated_message","type":{"type":"array","items":"Nested"},"default":[]},{"name":"map_message","type":{"type":"array","items":{"type":"record","name":"MapMessageEntry","namespace":"test.Sample","fields":[{"name":"key","type":{"type":"string","avro.java.string":"String"},"default":""},{"name":"value","type":["null","test.Nested"],"default":null}]}},"default":[]},{"name":"wrapped_string","type":["null",{"type":"string","avro.java.string":"String"}],"default":null},{"name":"wrapped_int","type":["null","int"],"default":null},{"name":"wrapped_long","type":["null","long"],"default":null},{"name":"wrapped_unsigned_int","type":["null","long"],"default":null},{"name":"wrapped_unsigned_long","type":["null","long"],"default":null},{"name":"wrapped_double","type":["null","double"],"default":null},{"name":"wrapped_float","type":["null","float"],"default":null},{"name":"wrapped_boolean","type":["null","boolean"],"default":null},{"name":"wrapped_bytes","type":["null","bytes"],"default":null},{"name":"enum","type":{"type":"enum","name":"SampleEnum","symbols":["FIRST","SECOND"]},"default":"FIRST"},{"name":"timestamp","type":["null",{"type":"record","name":"Timestamp","namespace":"google.protobuf","fields":[{"name":"seconds","type":"long","default":0},{"name":"nanos","type":"int","default":0}]}],"default":null}]} \ No newline at end of file diff --git a/hudi-utilities/src/test/resources/schema-provider/proto/sample_schema_nested.txt b/hudi-utilities/src/test/resources/schema-provider/proto/sample_schema_nested.txt new file mode 100644 index 000000000000..ba8892e02127 --- /dev/null +++ b/hudi-utilities/src/test/resources/schema-provider/proto/sample_schema_nested.txt @@ -0,0 +1 @@ +{"type":"record","name":"Sample","namespace":"test","fields":[{"name":"primitive_double","type":"double","default":0.0},{"name":"primitive_float","type":"float","default":0.0},{"name":"primitive_int","type":"int","default":0},{"name":"primitive_long","type":"long","default":0},{"name":"primitive_unsigned_int","type":"long","default":0},{"name":"primitive_unsigned_long","type":"long","default":0},{"name":"primitive_signed_int","type":"int","default":0},{"name":"primitive_signed_long","type":"long","default":0},{"name":"primitive_fixed_int","type":"int","default":0},{"name":"primitive_fixed_long","type":"long","default":0},{"name":"primitive_fixed_signed_int","type":"int","default":0},{"name":"primitive_fixed_signed_long","type":"long","default":0},{"name":"primitive_boolean","type":"boolean","default":false},{"name":"primitive_string","type":{"type":"string","avro.java.string":"String"},"default":""},{"name":"primitive_bytes","type":"bytes","default":""},{"name":"repeated_primitive","type":{"type":"array","items":"int"},"default":[]},{"name":"map_primitive","type":{"type":"array","items":{"type":"record","name":"MapPrimitiveEntry","namespace":"test.Sample","fields":[{"name":"key","type":{"type":"string","avro.java.string":"String"},"default":""},{"name":"value","type":"int","default":0}]}},"default":[]},{"name":"nested_message","type":["null",{"type":"record","name":"Nested","fields":[{"name":"nested_int","type":"int","default":0}]}],"default":null},{"name":"repeated_message","type":{"type":"array","items":"Nested"},"default":[]},{"name":"map_message","type":{"type":"array","items":{"type":"record","name":"MapMessageEntry","namespace":"test.Sample","fields":[{"name":"key","type":{"type":"string","avro.java.string":"String"},"default":""},{"name":"value","type":["null","test.Nested"],"default":null}]}},"default":[]},{"name":"wrapped_string","type":["null",{"type":"record","name":"StringValue","namespace":"google.protobuf","fields":[{"name":"value","type":{"type":"string","avro.java.string":"String"},"default":""}]}],"default":null},{"name":"wrapped_int","type":["null",{"type":"record","name":"Int32Value","namespace":"google.protobuf","fields":[{"name":"value","type":"int","default":0}]}],"default":null},{"name":"wrapped_long","type":["null",{"type":"record","name":"Int64Value","namespace":"google.protobuf","fields":[{"name":"value","type":"long","default":0}]}],"default":null},{"name":"wrapped_unsigned_int","type":["null",{"type":"record","name":"UInt32Value","namespace":"google.protobuf","fields":[{"name":"value","type":"long","default":0}]}],"default":null},{"name":"wrapped_unsigned_long","type":["null",{"type":"record","name":"UInt64Value","namespace":"google.protobuf","fields":[{"name":"value","type":"long","default":0}]}],"default":null},{"name":"wrapped_double","type":["null",{"type":"record","name":"DoubleValue","namespace":"google.protobuf","fields":[{"name":"value","type":"double","default":0.0}]}],"default":null},{"name":"wrapped_float","type":["null",{"type":"record","name":"FloatValue","namespace":"google.protobuf","fields":[{"name":"value","type":"float","default":0.0}]}],"default":null},{"name":"wrapped_boolean","type":["null",{"type":"record","name":"BoolValue","namespace":"google.protobuf","fields":[{"name":"value","type":"boolean","default":false}]}],"default":null},{"name":"wrapped_bytes","type":["null",{"type":"record","name":"BytesValue","namespace":"google.protobuf","fields":[{"name":"value","type":"bytes","default":""}]}],"default":null},{"name":"enum","type":{"type":"enum","name":"SampleEnum","symbols":["FIRST","SECOND"]},"default":"FIRST"},{"name":"timestamp","type":["null",{"type":"record","name":"Timestamp","namespace":"google.protobuf","fields":[{"name":"seconds","type":"long","default":0},{"name":"nanos","type":"int","default":0}]}],"default":null}]} \ No newline at end of file diff --git a/pom.xml b/pom.xml index 856dd7de0f8d..76e257212f18 100644 --- a/pom.xml +++ b/pom.xml @@ -510,6 +510,7 @@ **/dependency-reduced-pom.xml **/test/resources/*.data **/test/resources/*.commit + **/test/resources/**/*.txt **/target/** **/generated-sources/** .github/** @@ -569,6 +570,42 @@ + + com.github.os72 + protoc-jar-maven-plugin + 3.11.4 + + + proto-compile + generate-sources + + run + + + + src/main/resources + + + + + proto-test-compile + generate-test-sources + + run + + + test + + src/test/resources + + + + + + ${protoc.version} + true + + @@ -1302,6 +1339,18 @@ ${zk-curator.version} + + + com.google.protobuf + protobuf-java + ${proto.version} + + + com.google.protobuf + protobuf-java-util + ${proto.version} + + org.junit.jupiter