1
0

[HUDI-1296] Support Metadata Table in Spark Datasource (#4789)

* Bootstrapping initial support for Metadata Table in Spark Datasource

- Consolidated Avro/Row conversion utilities to center around Spark's AvroDeserializer ; removed duplication
- Bootstrapped HoodieBaseRelation
- Updated HoodieMergeOnReadRDD to be able to handle Metadata Table
- Modified MOR relations to be able to read different Base File formats (Parquet, HFile)
This commit is contained in:
Alexey Kudinkin
2022-02-24 13:23:13 -08:00
committed by GitHub
parent 521338b4d9
commit 85e8a5c4de
56 changed files with 1634 additions and 1010 deletions

View File

@@ -18,28 +18,26 @@
package org.apache.hudi.keygen;
import org.apache.avro.generic.GenericRecord;
import org.apache.hudi.ApiMaturityLevel;
import org.apache.hudi.AvroConversionHelper;
import org.apache.hudi.AvroConversionUtils;
import org.apache.hudi.HoodieSparkUtils;
import org.apache.hudi.PublicAPIMethod;
import org.apache.hudi.client.utils.SparkRowSerDe;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.HoodieKeyException;
import org.apache.avro.generic.GenericRecord;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.types.DataType;
import org.apache.spark.sql.types.StructType;
import scala.Function1;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import scala.Function1;
/**
* Base class for the built-in key generators. Contains methods structured for
* code reuse amongst them.
@@ -48,7 +46,7 @@ public abstract class BuiltinKeyGenerator extends BaseKeyGenerator implements Sp
private static final String STRUCT_NAME = "hoodieRowTopLevelField";
private static final String NAMESPACE = "hoodieRow";
private transient Function1<Object, Object> converterFn = null;
private transient Function1<Row, GenericRecord> converterFn = null;
private SparkRowSerDe sparkRowSerDe;
protected StructType structType;
@@ -69,10 +67,9 @@ public abstract class BuiltinKeyGenerator extends BaseKeyGenerator implements Sp
@PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
public String getRecordKey(Row row) {
if (null == converterFn) {
converterFn = AvroConversionHelper.createConverterToAvro(row.schema(), STRUCT_NAME, NAMESPACE);
converterFn = AvroConversionUtils.createConverterToAvro(row.schema(), STRUCT_NAME, NAMESPACE);
}
GenericRecord genericRecord = (GenericRecord) converterFn.apply(row);
return getKey(genericRecord).getRecordKey();
return getKey(converterFn.apply(row)).getRecordKey();
}
/**
@@ -84,10 +81,9 @@ public abstract class BuiltinKeyGenerator extends BaseKeyGenerator implements Sp
@PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
public String getPartitionPath(Row row) {
if (null == converterFn) {
converterFn = AvroConversionHelper.createConverterToAvro(row.schema(), STRUCT_NAME, NAMESPACE);
converterFn = AvroConversionUtils.createConverterToAvro(row.schema(), STRUCT_NAME, NAMESPACE);
}
GenericRecord genericRecord = (GenericRecord) converterFn.apply(row);
return getKey(genericRecord).getPartitionPath();
return getKey(converterFn.apply(row)).getPartitionPath();
}
/**

View File

@@ -37,10 +37,14 @@ public class ComplexKeyGenerator extends BuiltinKeyGenerator {
public ComplexKeyGenerator(TypedProperties props) {
super(props);
this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key())
.split(",")).map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toList());
this.partitionPathFields = Arrays.stream(props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key())
.split(",")).map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toList());
this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()).split(","))
.map(String::trim)
.filter(s -> !s.isEmpty())
.collect(Collectors.toList());
this.partitionPathFields = Arrays.stream(props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key()).split(","))
.map(String::trim)
.filter(s -> !s.isEmpty())
.collect(Collectors.toList());
complexAvroKeyGenerator = new ComplexAvroKeyGenerator(props);
}

View File

@@ -1,380 +0,0 @@
/*
* 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
import java.nio.ByteBuffer
import java.sql.{Date, Timestamp}
import java.time.Instant
import org.apache.avro.Conversions.DecimalConversion
import org.apache.avro.LogicalTypes.{TimestampMicros, TimestampMillis}
import org.apache.avro.Schema.Type._
import org.apache.avro.generic.GenericData.{Fixed, Record}
import org.apache.avro.generic.{GenericData, GenericFixed, GenericRecord}
import org.apache.avro.{LogicalTypes, Schema}
import org.apache.spark.sql.Row
import org.apache.spark.sql.avro.SchemaConverters
import org.apache.spark.sql.catalyst.expressions.GenericRow
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.types._
import org.apache.hudi.AvroConversionUtils._
import org.apache.hudi.exception.HoodieIncompatibleSchemaException
import scala.collection.JavaConverters._
object AvroConversionHelper {
private def createDecimal(decimal: java.math.BigDecimal, precision: Int, scale: Int): Decimal = {
if (precision <= Decimal.MAX_LONG_DIGITS) {
// Constructs a `Decimal` with an unscaled `Long` value if possible.
Decimal(decimal.unscaledValue().longValue(), precision, scale)
} else {
// Otherwise, resorts to an unscaled `BigInteger` instead.
Decimal(decimal, precision, scale)
}
}
/**
*
* Returns a converter function to convert row in avro format to GenericRow of catalyst.
*
* @param sourceAvroSchema Source schema before conversion inferred from avro file by passed in
* by user.
* @param targetSqlType Target catalyst sql type after the conversion.
* @return returns a converter function to convert row in avro format to GenericRow of catalyst.
*/
def createConverterToRow(sourceAvroSchema: Schema,
targetSqlType: DataType): AnyRef => AnyRef = {
def createConverter(avroSchema: Schema, sqlType: DataType, path: List[String]): AnyRef => AnyRef = {
val avroType = avroSchema.getType
(sqlType, avroType) match {
// Avro strings are in Utf8, so we have to call toString on them
case (StringType, STRING) | (StringType, ENUM) =>
(item: AnyRef) => if (item == null) null else item.toString
// Byte arrays are reused by avro, so we have to make a copy of them.
case (IntegerType, INT) | (BooleanType, BOOLEAN) | (DoubleType, DOUBLE) |
(FloatType, FLOAT) | (LongType, LONG) =>
identity
case (BinaryType, FIXED) =>
(item: AnyRef) =>
if (item == null) {
null
} else {
item.asInstanceOf[Fixed].bytes().clone()
}
case (BinaryType, BYTES) =>
(item: AnyRef) =>
if (item == null) {
null
} else {
val byteBuffer = item.asInstanceOf[ByteBuffer]
val bytes = new Array[Byte](byteBuffer.remaining)
byteBuffer.get(bytes)
bytes
}
case (d: DecimalType, FIXED) =>
(item: AnyRef) =>
if (item == null) {
null
} else {
val decimalConversion = new DecimalConversion
val bigDecimal = decimalConversion.fromFixed(item.asInstanceOf[GenericFixed], avroSchema,
LogicalTypes.decimal(d.precision, d.scale))
createDecimal(bigDecimal, d.precision, d.scale)
}
case (d: DecimalType, BYTES) =>
(item: AnyRef) =>
if (item == null) {
null
} else {
val decimalConversion = new DecimalConversion
val bigDecimal = decimalConversion.fromBytes(item.asInstanceOf[ByteBuffer], avroSchema,
LogicalTypes.decimal(d.precision, d.scale))
createDecimal(bigDecimal, d.precision, d.scale)
}
case (DateType, INT) =>
(item: AnyRef) =>
if (item == null) {
null
} else {
item match {
case integer: Integer => DateTimeUtils.toJavaDate(integer)
case _ => new Date(item.asInstanceOf[Long])
}
}
case (TimestampType, LONG) =>
(item: AnyRef) =>
if (item == null) {
null
} else {
avroSchema.getLogicalType match {
case _: TimestampMillis =>
new Timestamp(item.asInstanceOf[Long])
case _: TimestampMicros =>
new Timestamp(item.asInstanceOf[Long] / 1000)
case null =>
new Timestamp(item.asInstanceOf[Long])
case other =>
throw new HoodieIncompatibleSchemaException(
s"Cannot convert Avro logical type $other to Catalyst Timestamp type.")
}
}
case (struct: StructType, RECORD) =>
val length = struct.fields.length
val converters = new Array[AnyRef => AnyRef](length)
val avroFieldIndexes = new Array[Int](length)
var i = 0
while (i < length) {
val sqlField = struct.fields(i)
val avroField = avroSchema.getField(sqlField.name)
if (avroField != null) {
val converter = createConverter(avroField.schema(), sqlField.dataType,
path :+ sqlField.name)
converters(i) = converter
avroFieldIndexes(i) = avroField.pos()
} else if (!sqlField.nullable) {
throw new HoodieIncompatibleSchemaException(
s"Cannot find non-nullable field ${sqlField.name} at path ${path.mkString(".")} " +
"in Avro schema\n" +
s"Source Avro schema: $sourceAvroSchema.\n" +
s"Target Catalyst type: $targetSqlType")
}
i += 1
}
(item: AnyRef) => {
if (item == null) {
null
} else {
val record = item.asInstanceOf[GenericRecord]
val result = new Array[Any](length)
var i = 0
while (i < converters.length) {
if (converters(i) != null) {
val converter = converters(i)
result(i) = converter(record.get(avroFieldIndexes(i)))
}
i += 1
}
new GenericRow(result)
}
}
case (arrayType: ArrayType, ARRAY) =>
val elementConverter = createConverter(avroSchema.getElementType, arrayType.elementType,
path)
val allowsNull = arrayType.containsNull
(item: AnyRef) => {
if (item == null) {
null
} else {
item.asInstanceOf[java.lang.Iterable[AnyRef]].asScala.map { element =>
if (element == null && !allowsNull) {
throw new RuntimeException(s"Array value at path ${path.mkString(".")} is not " +
"allowed to be null")
} else {
elementConverter(element)
}
}
}
}
case (mapType: MapType, MAP) if mapType.keyType == StringType =>
val valueConverter = createConverter(avroSchema.getValueType, mapType.valueType, path)
val allowsNull = mapType.valueContainsNull
(item: AnyRef) => {
if (item == null) {
null
} else {
item.asInstanceOf[java.util.Map[AnyRef, AnyRef]].asScala.map { x =>
if (x._2 == null && !allowsNull) {
throw new RuntimeException(s"Map value at path ${path.mkString(".")} is not " +
"allowed to be null")
} else {
(x._1.toString, valueConverter(x._2))
}
}.toMap
}
}
case (sqlType, UNION) =>
if (avroSchema.getTypes.asScala.exists(_.getType == NULL)) {
val remainingUnionTypes = avroSchema.getTypes.asScala.filterNot(_.getType == NULL)
if (remainingUnionTypes.size == 1) {
createConverter(remainingUnionTypes.head, sqlType, path)
} else {
createConverter(Schema.createUnion(remainingUnionTypes.asJava), sqlType, path)
}
} else avroSchema.getTypes.asScala.map(_.getType) match {
case Seq(_) => createConverter(avroSchema.getTypes.get(0), sqlType, path)
case Seq(a, b) if Set(a, b) == Set(INT, LONG) && sqlType == LongType =>
(item: AnyRef) => {
item match {
case null => null
case l: java.lang.Long => l
case i: java.lang.Integer => new java.lang.Long(i.longValue())
}
}
case Seq(a, b) if Set(a, b) == Set(FLOAT, DOUBLE) && sqlType == DoubleType =>
(item: AnyRef) => {
item match {
case null => null
case d: java.lang.Double => d
case f: java.lang.Float => new java.lang.Double(f.doubleValue())
}
}
case other =>
sqlType match {
case t: StructType if t.fields.length == avroSchema.getTypes.size =>
val fieldConverters = t.fields.zip(avroSchema.getTypes.asScala).map {
case (field, schema) =>
createConverter(schema, field.dataType, path :+ field.name)
}
(item: AnyRef) =>
if (item == null) {
null
} else {
val i = GenericData.get().resolveUnion(avroSchema, item)
val converted = new Array[Any](fieldConverters.length)
converted(i) = fieldConverters(i)(item)
new GenericRow(converted)
}
case _ => throw new HoodieIncompatibleSchemaException(
s"Cannot convert Avro schema to catalyst type because schema at path " +
s"${path.mkString(".")} is not compatible " +
s"(avroType = $other, sqlType = $sqlType). \n" +
s"Source Avro schema: $sourceAvroSchema.\n" +
s"Target Catalyst type: $targetSqlType")
}
}
case (left, right) =>
throw new HoodieIncompatibleSchemaException(
s"Cannot convert Avro schema to catalyst type because schema at path " +
s"${path.mkString(".")} is not compatible (avroType = $left, sqlType = $right). \n" +
s"Source Avro schema: $sourceAvroSchema.\n" +
s"Target Catalyst type: $targetSqlType")
}
}
createConverter(sourceAvroSchema, targetSqlType, List.empty[String])
}
def createConverterToAvro(dataType: DataType,
structName: String,
recordNamespace: String): Any => Any = {
dataType match {
case BinaryType => (item: Any) =>
item match {
case null => null
case bytes: Array[Byte] => ByteBuffer.wrap(bytes)
}
case IntegerType | LongType |
FloatType | DoubleType | StringType | BooleanType => identity
case ByteType => (item: Any) =>
if (item == null) null else item.asInstanceOf[Byte].intValue
case ShortType => (item: Any) =>
if (item == null) null else item.asInstanceOf[Short].intValue
case dec: DecimalType =>
val schema = SchemaConverters.toAvroType(dec, nullable = false, structName, recordNamespace)
(item: Any) => {
Option(item).map { _ =>
val bigDecimalValue = item.asInstanceOf[java.math.BigDecimal]
val decimalConversions = new DecimalConversion()
decimalConversions.toFixed(bigDecimalValue, schema, LogicalTypes.decimal(dec.precision, dec.scale))
}.orNull
}
case TimestampType => (item: Any) =>
if (item == null) {
null
} else {
val timestamp = item match {
case i: Instant => Timestamp.from(i)
case t: Timestamp => t
}
// Convert time to microseconds since spark-avro by default converts TimestampType to
// Avro Logical TimestampMicros
timestamp.getTime * 1000
}
case DateType => (item: Any) =>
Option(item).map(_.asInstanceOf[Date].toLocalDate.toEpochDay.toInt).orNull
case ArrayType(elementType, _) =>
val elementConverter = createConverterToAvro(
elementType,
structName,
recordNamespace)
(item: Any) => {
if (item == null) {
null
} else {
val sourceArray = item.asInstanceOf[Seq[Any]]
val sourceArraySize = sourceArray.size
val targetList = new java.util.ArrayList[Any](sourceArraySize)
var idx = 0
while (idx < sourceArraySize) {
targetList.add(elementConverter(sourceArray(idx)))
idx += 1
}
targetList
}
}
case MapType(StringType, valueType, _) =>
val valueConverter = createConverterToAvro(
valueType,
structName,
recordNamespace)
(item: Any) => {
if (item == null) {
null
} else {
val javaMap = new java.util.HashMap[String, Any]()
item.asInstanceOf[Map[String, Any]].foreach { case (key, value) =>
javaMap.put(key, valueConverter(value))
}
javaMap
}
}
case structType: StructType =>
val schema: Schema = convertStructTypeToAvroSchema(structType, structName, recordNamespace)
val childNameSpace = if (recordNamespace != "") s"$recordNamespace.$structName" else structName
val fieldConverters = structType.fields.map(field =>
createConverterToAvro(
field.dataType,
field.name,
childNameSpace))
(item: Any) => {
if (item == null) {
null
} else {
val record = new Record(schema)
val convertersIterator = fieldConverters.iterator
val fieldNamesIterator = dataType.asInstanceOf[StructType].fieldNames.iterator
val rowIterator = item.asInstanceOf[Row].toSeq.iterator
while (convertersIterator.hasNext && rowIterator.hasNext) {
val converter = convertersIterator.next()
record.put(fieldNamesIterator.next(), converter(rowIterator.next()))
}
record
}
}
}
}
}

View File

@@ -18,20 +18,105 @@
package org.apache.hudi
import org.apache.avro.Schema
import org.apache.avro.JsonProperties
import org.apache.avro.Schema.Type
import org.apache.avro.generic.{GenericRecord, GenericRecordBuilder, IndexedRecord}
import org.apache.avro.{AvroRuntimeException, JsonProperties, Schema}
import org.apache.hudi.HoodieSparkUtils.sparkAdapter
import org.apache.hudi.avro.HoodieAvroUtils
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.avro.SchemaConverters
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.encoders.RowEncoder
import org.apache.spark.sql.types.{DataType, StructType}
import org.apache.spark.sql.{Dataset, Row, SparkSession}
import scala.collection.JavaConverters._
import scala.collection.JavaConversions._
import scala.collection.JavaConverters._
object AvroConversionUtils {
/**
* Check the nullability of the input Avro type and resolve it when it is nullable. The first
* return value is a [[Boolean]] indicating if the input Avro type is nullable. The second
* return value is either provided Avro type if it's not nullable, or its resolved non-nullable part
* in case it is
*/
def resolveAvroTypeNullability(avroType: Schema): (Boolean, Schema) = {
if (avroType.getType == Type.UNION) {
val fields = avroType.getTypes.asScala
val actualType = fields.filter(_.getType != Type.NULL)
if (fields.length != 2 || actualType.length != 1) {
throw new AvroRuntimeException(
s"Unsupported Avro UNION type $avroType: Only UNION of a null type and a non-null " +
"type is supported")
}
(true, actualType.head)
} else {
(false, avroType)
}
}
/**
* Creates converter to transform Avro payload into Spark's Catalyst one
*
* @param rootAvroType Avro [[Schema]] to be transformed from
* @param rootCatalystType Catalyst [[StructType]] to be transformed into
* @return converter accepting Avro payload and transforming it into a Catalyst one (in the form of [[InternalRow]])
*/
def createAvroToInternalRowConverter(rootAvroType: Schema, rootCatalystType: StructType): GenericRecord => Option[InternalRow] =
record => sparkAdapter.createAvroDeserializer(rootAvroType, rootCatalystType)
.deserialize(record)
.map(_.asInstanceOf[InternalRow])
/**
* Creates converter to transform Catalyst payload into Avro one
*
* @param rootCatalystType Catalyst [[StructType]] to be transformed from
* @param rootAvroType Avro [[Schema]] to be transformed into
* @param nullable whether Avro record is nullable
* @return converter accepting Catalyst payload (in the form of [[InternalRow]]) and transforming it into an Avro one
*/
def createInternalRowToAvroConverter(rootCatalystType: StructType, rootAvroType: Schema, nullable: Boolean): InternalRow => GenericRecord = {
row => sparkAdapter.createAvroSerializer(rootCatalystType, rootAvroType, nullable)
.serialize(row)
.asInstanceOf[GenericRecord]
}
/**
* @deprecated please use [[AvroConversionUtils.createAvroToInternalRowConverter]]
*/
@Deprecated
def createConverterToRow(sourceAvroSchema: Schema,
targetSqlType: StructType): GenericRecord => Row = {
val encoder = RowEncoder.apply(targetSqlType).resolveAndBind()
val serde = sparkAdapter.createSparkRowSerDe(encoder)
val converter = AvroConversionUtils.createAvroToInternalRowConverter(sourceAvroSchema, targetSqlType)
avro => converter.apply(avro).map(serde.deserializeRow).get
}
/**
* @deprecated please use [[AvroConversionUtils.createInternalRowToAvroConverter]]
*/
@Deprecated
def createConverterToAvro(sourceSqlType: StructType,
structName: String,
recordNamespace: String): Row => GenericRecord = {
val encoder = RowEncoder.apply(sourceSqlType).resolveAndBind()
val serde = sparkAdapter.createSparkRowSerDe(encoder)
val avroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(sourceSqlType, structName, recordNamespace)
val (nullable, _) = resolveAvroTypeNullability(avroSchema)
val converter = AvroConversionUtils.createInternalRowToAvroConverter(sourceSqlType, avroSchema, nullable)
row => converter.apply(serde.serializeRow(row))
}
/**
* Creates [[org.apache.spark.sql.DataFrame]] from the provided [[RDD]] of [[GenericRecord]]s
*
* TODO convert directly from GenericRecord into InternalRow instead
*/
def createDataFrame(rdd: RDD[GenericRecord], schemaStr: String, ss: SparkSession): Dataset[Row] = {
if (rdd.isEmpty()) {
ss.emptyDataFrame
@@ -41,8 +126,8 @@ object AvroConversionUtils {
else {
val schema = new Schema.Parser().parse(schemaStr)
val dataType = convertAvroSchemaToStructType(schema)
val convertor = AvroConversionHelper.createConverterToRow(schema, dataType)
records.map { x => convertor(x).asInstanceOf[Row] }
val converter = createConverterToRow(schema, dataType)
records.map { r => converter(r) }
}
}, convertAvroSchemaToStructType(new Schema.Parser().parse(schemaStr)))
}
@@ -57,7 +142,7 @@ object AvroConversionUtils {
* @param recordNamespace Avro record namespace.
* @return Avro schema corresponding to given struct type.
*/
def convertStructTypeToAvroSchema(structType: StructType,
def convertStructTypeToAvroSchema(structType: DataType,
structName: String,
recordNamespace: String): Schema = {
getAvroSchemaWithDefaults(SchemaConverters.toAvroType(structType, nullable = false, structName, recordNamespace))

View File

@@ -18,13 +18,10 @@
package org.apache.hudi
import java.util.Properties
import org.apache.avro.Schema
import org.apache.avro.generic.GenericRecord
import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.hudi.avro.HoodieAvroUtils.rewriteRecord
import org.apache.hudi.client.utils.SparkRowSerDe
import org.apache.hudi.common.config.TypedProperties
import org.apache.hudi.common.model.HoodieRecord
@@ -32,7 +29,6 @@ import org.apache.hudi.common.table.HoodieTableMetaClient
import org.apache.hudi.keygen.constant.KeyGeneratorOptions
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory
import org.apache.hudi.keygen.{BaseKeyGenerator, CustomAvroKeyGenerator, CustomKeyGenerator, KeyGenerator}
import org.apache.spark.SPARK_VERSION
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.encoders.RowEncoder
@@ -42,8 +38,8 @@ import org.apache.spark.sql.sources._
import org.apache.spark.sql.types.{StringType, StructField, StructType}
import org.apache.spark.sql.{DataFrame, SparkSession}
import java.util.Properties
import scala.collection.JavaConverters._
import scala.collection.JavaConverters.asScalaBufferConverter
object HoodieSparkUtils extends SparkAdapterSupport {
@@ -127,46 +123,49 @@ object HoodieSparkUtils extends SparkAdapterSupport {
new InMemoryFileIndex(sparkSession, globbedPaths, Map(), Option.empty, fileStatusCache)
}
def createRdd(df: DataFrame, structName: String, recordNamespace: String, reconcileToLatestSchema: Boolean, latestTableSchema:
org.apache.hudi.common.util.Option[Schema] = org.apache.hudi.common.util.Option.empty()): RDD[GenericRecord] = {
val dfWriteSchema = AvroConversionUtils.convertStructTypeToAvroSchema(df.schema, structName, recordNamespace)
var writeSchema : Schema = null;
var toReconcileSchema : Schema = null;
if (reconcileToLatestSchema && latestTableSchema.isPresent) {
// if reconcileToLatestSchema is set to true and latestSchema is present, then try to leverage latestTableSchema.
// this code path will handle situations where records are serialized in odl schema, but callers wish to convert
// to Rdd[GenericRecord] using different schema(could be evolved schema or could be latest table schema)
writeSchema = dfWriteSchema
toReconcileSchema = latestTableSchema.get()
} else {
// there are paths where callers wish to use latestTableSchema to convert to Rdd[GenericRecords] and not use
// row's schema. So use latestTableSchema if present. if not available, fallback to using row's schema.
writeSchema = if (latestTableSchema.isPresent) { latestTableSchema.get()} else { dfWriteSchema}
}
createRddInternal(df, writeSchema, toReconcileSchema, structName, recordNamespace)
/**
* @deprecated please use other overload [[createRdd]]
*/
def createRdd(df: DataFrame, structName: String, recordNamespace: String, reconcileToLatestSchema: Boolean,
latestTableSchema: org.apache.hudi.common.util.Option[Schema] = org.apache.hudi.common.util.Option.empty()): RDD[GenericRecord] = {
val latestTableSchemaConverted = if (latestTableSchema.isPresent && reconcileToLatestSchema) Some(latestTableSchema.get()) else None
createRdd(df, structName, recordNamespace, latestTableSchemaConverted)
}
def createRddInternal(df: DataFrame, writeSchema: Schema, latestTableSchema: Schema, structName: String, recordNamespace: String)
: RDD[GenericRecord] = {
// Use the write avro schema to derive the StructType which has the correct nullability information
val writeDataType = AvroConversionUtils.convertAvroSchemaToStructType(writeSchema)
val encoder = RowEncoder.apply(writeDataType).resolveAndBind()
val deserializer = sparkAdapter.createSparkRowSerDe(encoder)
// if records were serialized with old schema, but an evolved schema was passed in with latestTableSchema, we need
// latestTableSchema equivalent datatype to be passed in to AvroConversionHelper.createConverterToAvro()
val reconciledDataType =
if (latestTableSchema != null) AvroConversionUtils.convertAvroSchemaToStructType(latestTableSchema) else writeDataType
// Note: deserializer.deserializeRow(row) is not capable of handling evolved schema. i.e. if Row was serialized in
// old schema, but deserializer was created with an encoder with evolved schema, deserialization fails.
// Hence we always need to deserialize in the same schema as serialized schema.
df.queryExecution.toRdd.map(row => deserializer.deserializeRow(row))
.mapPartitions { records =>
if (records.isEmpty) Iterator.empty
else {
val convertor = AvroConversionHelper.createConverterToAvro(reconciledDataType, structName, recordNamespace)
records.map { x => convertor(x).asInstanceOf[GenericRecord] }
}
def createRdd(df: DataFrame, structName: String, recordNamespace: String, readerAvroSchemaOpt: Option[Schema]): RDD[GenericRecord] = {
val writerSchema = df.schema
val writerAvroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(writerSchema, structName, recordNamespace)
val readerAvroSchema = readerAvroSchemaOpt.getOrElse(writerAvroSchema)
// We check whether passed in reader schema is identical to writer schema to avoid costly serde loop of
// making Spark deserialize its internal representation [[InternalRow]] into [[Row]] for subsequent conversion
// (and back)
val sameSchema = writerAvroSchema.equals(readerAvroSchema)
val (nullable, _) = AvroConversionUtils.resolveAvroTypeNullability(writerAvroSchema)
// NOTE: We have to serialize Avro schema, and then subsequently parse it on the executor node, since Spark
// serializer is not able to digest it
val readerAvroSchemaStr = readerAvroSchema.toString
val writerAvroSchemaStr = writerAvroSchema.toString
// NOTE: We're accessing toRdd here directly to avoid [[InternalRow]] to [[Row]] conversion
df.queryExecution.toRdd.mapPartitions { rows =>
if (rows.isEmpty) {
Iterator.empty
} else {
val transform: GenericRecord => GenericRecord =
if (sameSchema) identity
else {
val readerAvroSchema = new Schema.Parser().parse(readerAvroSchemaStr)
rewriteRecord(_, readerAvroSchema)
}
// Since caller might request to get records in a different ("evolved") schema, we will be rewriting from
// existing Writer's schema into Reader's (avro) schema
val writerAvroSchema = new Schema.Parser().parse(writerAvroSchemaStr)
val convert = AvroConversionUtils.createInternalRowToAvroConverter(writerSchema, writerAvroSchema, nullable = nullable)
rows.map { ir => transform(convert(ir)) }
}
}
}
def getDeserializer(structType: StructType) : SparkRowSerDe = {

View File

@@ -0,0 +1,35 @@
/*
* 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.spark.sql.avro
/**
* Deserializes Avro payload into Catalyst object
*
* NOTE: This is low-level component operating on Spark internal data-types (comprising [[InternalRow]]).
* If you're looking to convert Avro into "deserialized" [[Row]] (comprised of Java native types),
* please check [[AvroConversionUtils]]
*/
trait HoodieAvroDeserializerTrait {
final def deserialize(data: Any): Option[Any] =
doDeserialize(data) match {
case opt: Option[_] => opt // As of Spark 3.1, this will return data wrapped with Option, so we fetch the data
case row => Some(row) // For other Spark versions, return the data as is
}
protected def doDeserialize(data: Any): Any
}

View File

@@ -0,0 +1,28 @@
/*
* 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.spark.sql.avro
/**
* Serializes Catalyst payload into Avro object
*
* NOTE: This is low-level component operating on Spark internal data-types (comprising [[InternalRow]]).
* If you're looking to convert "deserialized" [[Row]] into Avro, please check [[AvroConversionUtils]]
*/
trait HoodieAvroSerializerTrait {
def serialize(catalystData: Any): Any
}

View File

@@ -18,8 +18,9 @@
package org.apache.spark.sql.hudi
import org.apache.hudi.HoodieSparkUtils.sparkAdapter
import org.apache.avro.Schema
import org.apache.hudi.client.utils.SparkRowSerDe
import org.apache.spark.sql.avro.{HoodieAvroDeserializerTrait, HoodieAvroSerializerTrait}
import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
import org.apache.spark.sql.catalyst.catalog.CatalogTable
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
@@ -30,6 +31,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, SubqueryA
import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier}
import org.apache.spark.sql.execution.datasources.{FilePartition, LogicalRelation, PartitionedFile, SparkParsePartitionUtil}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.DataType
import org.apache.spark.sql.{Row, SparkSession}
import java.util.Locale
@@ -40,6 +42,18 @@ import java.util.Locale
*/
trait SparkAdapter extends Serializable {
/**
* Creates instance of [[HoodieAvroSerializerTrait]] providing for ability to serialize
* Spark's [[InternalRow]] into Avro payloads
*/
def createAvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: Boolean): HoodieAvroSerializerTrait
/**
* Creates instance of [[HoodieAvroDeserializerTrait]] providing for ability to deserialize
* Avro payloads into Spark's [[InternalRow]]
*/
def createAvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType): HoodieAvroDeserializerTrait
/**
* Create the SparkRowSerDe.
*/

View File

@@ -1,179 +0,0 @@
/*
* 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.keygen;
import org.apache.hudi.AvroConversionUtils;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.exception.HoodieKeyException;
import org.apache.avro.generic.GenericRecord;
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
import org.apache.hudi.testutils.KeyGeneratorTestUtilities;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.catalyst.InternalRow;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
import static junit.framework.TestCase.assertEquals;
public class TestComplexKeyGenerator extends KeyGeneratorTestUtilities {
private TypedProperties getCommonProps(boolean getComplexRecordKey) {
TypedProperties properties = new TypedProperties();
if (getComplexRecordKey) {
properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key, pii_col");
} else {
properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key");
}
properties.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE.key(), "true");
return properties;
}
private TypedProperties getPropertiesWithoutPartitionPathProp() {
return getCommonProps(false);
}
private TypedProperties getPropertiesWithoutRecordKeyProp() {
TypedProperties properties = new TypedProperties();
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp");
return properties;
}
private TypedProperties getWrongRecordKeyFieldProps() {
TypedProperties properties = new TypedProperties();
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp");
properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_wrong_key");
return properties;
}
private TypedProperties getProps() {
TypedProperties properties = getCommonProps(true);
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp,ts_ms");
return properties;
}
@Test
public void testNullPartitionPathFields() {
Assertions.assertThrows(IllegalArgumentException.class, () -> new ComplexKeyGenerator(getPropertiesWithoutPartitionPathProp()));
}
@Test
public void testNullRecordKeyFields() {
Assertions.assertThrows(IllegalArgumentException.class, () -> new ComplexKeyGenerator(getPropertiesWithoutRecordKeyProp()));
}
@Test
public void testWrongRecordKeyField() {
ComplexKeyGenerator keyGenerator = new ComplexKeyGenerator(getWrongRecordKeyFieldProps());
Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord()));
Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.buildFieldPositionMapIfNeeded(KeyGeneratorTestUtilities.structType));
}
@Test
public void testHappyFlow() {
ComplexKeyGenerator keyGenerator = new ComplexKeyGenerator(getProps());
GenericRecord record = getRecord();
HoodieKey key = keyGenerator.getKey(record);
Assertions.assertEquals(key.getRecordKey(), "_row_key:key1,pii_col:pi");
Assertions.assertEquals(key.getPartitionPath(), "timestamp=4357686/ts_ms=2020-03-21");
Row row = KeyGeneratorTestUtilities.getRow(record);
Assertions.assertEquals(keyGenerator.getRecordKey(row), "_row_key:key1,pii_col:pi");
Assertions.assertEquals(keyGenerator.getPartitionPath(row), "timestamp=4357686/ts_ms=2020-03-21");
InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row);
Assertions.assertEquals(keyGenerator.getPartitionPath(internalRow, row.schema()), "timestamp=4357686/ts_ms=2020-03-21");
}
@Test
public void testSingleValueKeyGenerator() {
TypedProperties properties = new TypedProperties();
properties.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key");
properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp");
ComplexKeyGenerator compositeKeyGenerator = new ComplexKeyGenerator(properties);
assertEquals(compositeKeyGenerator.getRecordKeyFields().size(), 1);
assertEquals(compositeKeyGenerator.getPartitionPathFields().size(), 1);
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator();
GenericRecord record = dataGenerator.generateGenericRecords(1).get(0);
String rowKey = record.get("_row_key").toString();
String partitionPath = record.get("timestamp").toString();
HoodieKey hoodieKey = compositeKeyGenerator.getKey(record);
assertEquals("_row_key:" + rowKey, hoodieKey.getRecordKey());
assertEquals(partitionPath, hoodieKey.getPartitionPath());
Row row = KeyGeneratorTestUtilities.getRow(record, HoodieTestDataGenerator.AVRO_SCHEMA,
AvroConversionUtils.convertAvroSchemaToStructType(HoodieTestDataGenerator.AVRO_SCHEMA));
Assertions.assertEquals(compositeKeyGenerator.getPartitionPath(row), partitionPath);
InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row);
Assertions.assertEquals(compositeKeyGenerator.getPartitionPath(internalRow, row.schema()), partitionPath);
}
@Test
public void testMultipleValueKeyGenerator() {
TypedProperties properties = new TypedProperties();
properties.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key,timestamp");
properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "rider,driver");
ComplexKeyGenerator compositeKeyGenerator = new ComplexKeyGenerator(properties);
assertEquals(compositeKeyGenerator.getRecordKeyFields().size(), 2);
assertEquals(compositeKeyGenerator.getPartitionPathFields().size(), 2);
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator();
GenericRecord record = dataGenerator.generateGenericRecords(1).get(0);
String rowKey =
"_row_key" + ComplexAvroKeyGenerator.DEFAULT_RECORD_KEY_SEPARATOR + record.get("_row_key").toString() + ","
+ "timestamp" + ComplexAvroKeyGenerator.DEFAULT_RECORD_KEY_SEPARATOR + record.get("timestamp").toString();
String partitionPath = record.get("rider").toString() + "/" + record.get("driver").toString();
HoodieKey hoodieKey = compositeKeyGenerator.getKey(record);
assertEquals(rowKey, hoodieKey.getRecordKey());
assertEquals(partitionPath, hoodieKey.getPartitionPath());
Row row = KeyGeneratorTestUtilities.getRow(record, HoodieTestDataGenerator.AVRO_SCHEMA,
AvroConversionUtils.convertAvroSchemaToStructType(HoodieTestDataGenerator.AVRO_SCHEMA));
Assertions.assertEquals(compositeKeyGenerator.getPartitionPath(row), partitionPath);
InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row);
Assertions.assertEquals(compositeKeyGenerator.getPartitionPath(internalRow, row.schema()), partitionPath);
}
@Test
public void testMultipleValueKeyGeneratorNonPartitioned() {
TypedProperties properties = new TypedProperties();
properties.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key,timestamp");
properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "");
ComplexKeyGenerator compositeKeyGenerator = new ComplexKeyGenerator(properties);
assertEquals(compositeKeyGenerator.getRecordKeyFields().size(), 2);
assertEquals(compositeKeyGenerator.getPartitionPathFields().size(), 0);
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator();
GenericRecord record = dataGenerator.generateGenericRecords(1).get(0);
String rowKey =
"_row_key" + ComplexAvroKeyGenerator.DEFAULT_RECORD_KEY_SEPARATOR + record.get("_row_key").toString() + ","
+ "timestamp" + ComplexAvroKeyGenerator.DEFAULT_RECORD_KEY_SEPARATOR + record.get("timestamp").toString();
String partitionPath = "";
HoodieKey hoodieKey = compositeKeyGenerator.getKey(record);
assertEquals(rowKey, hoodieKey.getRecordKey());
assertEquals(partitionPath, hoodieKey.getPartitionPath());
Row row = KeyGeneratorTestUtilities.getRow(record, HoodieTestDataGenerator.AVRO_SCHEMA,
AvroConversionUtils.convertAvroSchemaToStructType(HoodieTestDataGenerator.AVRO_SCHEMA));
Assertions.assertEquals(compositeKeyGenerator.getPartitionPath(row), partitionPath);
InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row);
Assertions.assertEquals(compositeKeyGenerator.getPartitionPath(internalRow, row.schema()), partitionPath);
}
}

View File

@@ -1,386 +0,0 @@
/*
* 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.keygen;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.avro.generic.GenericRecord;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
import org.apache.hudi.keygen.constant.KeyGeneratorType;
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
import org.apache.hudi.testutils.KeyGeneratorTestUtilities;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.catalyst.InternalRow;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
import java.io.IOException;
import java.io.PrintWriter;
import java.io.StringWriter;
public class TestCustomKeyGenerator extends KeyGeneratorTestUtilities {
/**
* Method to create props used for common cases.
*
* @param getComplexRecordKey Use complex record key or not
* @param useKeyGeneratorClassName Use KeyGenerator class name initialize KeyGenerator or not.
* true use {@code HoodieWriteConfig.KEYGENERATOR_CLASS_PROP},
* false use {@code HoodieWriteConfig.KEYGENERATOR_TYPE_PROP}
* @return TypedProperties used to initialize KeyGenerator.
*/
private TypedProperties getCommonProps(boolean getComplexRecordKey, boolean useKeyGeneratorClassName) {
TypedProperties properties = new TypedProperties();
if (getComplexRecordKey) {
properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key, pii_col");
} else {
properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key");
}
if (useKeyGeneratorClassName) {
properties.put(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key(), CustomKeyGenerator.class.getName());
} else {
properties.put(HoodieWriteConfig.KEYGENERATOR_TYPE.key(), KeyGeneratorType.CUSTOM.name());
}
properties.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE.key(), "true");
return properties;
}
private TypedProperties getPropertiesForSimpleKeyGen(boolean useKeyGeneratorClassName) {
TypedProperties properties = getCommonProps(false, useKeyGeneratorClassName);
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp:simple");
return properties;
}
private TypedProperties getImproperPartitionFieldFormatProp(boolean useKeyGeneratorClassName) {
TypedProperties properties = getCommonProps(false, useKeyGeneratorClassName);
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp");
return properties;
}
private TypedProperties getInvalidPartitionKeyTypeProps(boolean useKeyGeneratorClassName) {
TypedProperties properties = getCommonProps(false, useKeyGeneratorClassName);
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp:dummy");
return properties;
}
private TypedProperties getComplexRecordKeyWithSimplePartitionProps(boolean useKeyGeneratorClassName) {
TypedProperties properties = getCommonProps(true, useKeyGeneratorClassName);
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp:simple");
return properties;
}
private TypedProperties getComplexRecordKeyAndPartitionPathProps(boolean useKeyGeneratorClassName) {
TypedProperties properties = getCommonProps(true, useKeyGeneratorClassName);
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp:simple,ts_ms:timestamp");
populateNecessaryPropsForTimestampBasedKeyGen(properties);
return properties;
}
private TypedProperties getPropsWithoutRecordKeyFieldProps(boolean useKeyGeneratorClassName) {
TypedProperties properties = new TypedProperties();
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp:simple");
if (useKeyGeneratorClassName) {
properties.put(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key(), CustomKeyGenerator.class.getName());
} else {
properties.put(HoodieWriteConfig.KEYGENERATOR_TYPE.key(), KeyGeneratorType.CUSTOM.name());
}
return properties;
}
private void populateNecessaryPropsForTimestampBasedKeyGen(TypedProperties properties) {
properties.put("hoodie.deltastreamer.keygen.timebased.timestamp.type", "DATE_STRING");
properties.put("hoodie.deltastreamer.keygen.timebased.input.dateformat", "yyyy-MM-dd");
properties.put("hoodie.deltastreamer.keygen.timebased.output.dateformat", "yyyyMMdd");
}
private TypedProperties getPropertiesForTimestampBasedKeyGen(boolean useKeyGeneratorClassName) {
TypedProperties properties = getCommonProps(false, useKeyGeneratorClassName);
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "ts_ms:timestamp");
populateNecessaryPropsForTimestampBasedKeyGen(properties);
return properties;
}
private TypedProperties getPropertiesForNonPartitionedKeyGen(boolean useKeyGeneratorClassName) {
TypedProperties properties = getCommonProps(false, useKeyGeneratorClassName);
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "");
return properties;
}
private String stackTraceToString(Throwable e) {
StringWriter sw = new StringWriter();
PrintWriter pw = new PrintWriter(sw);
e.printStackTrace(pw);
return sw.toString();
}
@Test
public void testSimpleKeyGeneratorWithKeyGeneratorClass() throws IOException {
testSimpleKeyGenerator(getPropertiesForSimpleKeyGen(true));
}
@Test
public void testSimpleKeyGeneratorWithKeyGeneratorType() throws IOException {
testSimpleKeyGenerator(getPropertiesForSimpleKeyGen(false));
}
public void testSimpleKeyGenerator(TypedProperties props) throws IOException {
BuiltinKeyGenerator keyGenerator =
(BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(props);
GenericRecord record = getRecord();
HoodieKey key = keyGenerator.getKey(record);
Assertions.assertEquals(key.getRecordKey(), "key1");
Assertions.assertEquals(key.getPartitionPath(), "timestamp=4357686");
Row row = KeyGeneratorTestUtilities.getRow(record);
Assertions.assertEquals(keyGenerator.getRecordKey(row), "key1");
Assertions.assertEquals(keyGenerator.getPartitionPath(row), "timestamp=4357686");
InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row);
Assertions.assertEquals(keyGenerator.getPartitionPath(internalRow, row.schema()), "timestamp=4357686");
}
@Test
public void testTimestampBasedKeyGeneratorWithKeyGeneratorClass() throws IOException {
testTimestampBasedKeyGenerator(getPropertiesForTimestampBasedKeyGen(true));
}
@Test
public void testTimestampBasedKeyGeneratorWithKeyGeneratorType() throws IOException {
testTimestampBasedKeyGenerator(getPropertiesForTimestampBasedKeyGen(false));
}
public void testTimestampBasedKeyGenerator(TypedProperties props) throws IOException {
BuiltinKeyGenerator keyGenerator =
(BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(props);
GenericRecord record = getRecord();
HoodieKey key = keyGenerator.getKey(record);
Assertions.assertEquals(key.getRecordKey(), "key1");
Assertions.assertEquals(key.getPartitionPath(), "ts_ms=20200321");
Row row = KeyGeneratorTestUtilities.getRow(record);
Assertions.assertEquals(keyGenerator.getRecordKey(row), "key1");
Assertions.assertEquals(keyGenerator.getPartitionPath(row), "ts_ms=20200321");
InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row);
Assertions.assertEquals(keyGenerator.getPartitionPath(internalRow, row.schema()), "ts_ms=20200321");
}
@Test
public void testNonPartitionedKeyGeneratorWithKeyGeneratorClass() throws IOException {
testNonPartitionedKeyGenerator(getPropertiesForNonPartitionedKeyGen(true));
}
@Test
public void testNonPartitionedKeyGeneratorWithKeyGeneratorType() throws IOException {
testNonPartitionedKeyGenerator(getPropertiesForNonPartitionedKeyGen(false));
}
public void testNonPartitionedKeyGenerator(TypedProperties props) throws IOException {
BuiltinKeyGenerator keyGenerator =
(BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(props);
GenericRecord record = getRecord();
HoodieKey key = keyGenerator.getKey(record);
Assertions.assertEquals(key.getRecordKey(), "key1");
Assertions.assertTrue(key.getPartitionPath().isEmpty());
Row row = KeyGeneratorTestUtilities.getRow(record);
Assertions.assertEquals(keyGenerator.getRecordKey(row), "key1");
Assertions.assertTrue(keyGenerator.getPartitionPath(row).isEmpty());
InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row);
Assertions.assertTrue(keyGenerator.getPartitionPath(internalRow, row.schema()).isEmpty());
}
@Test
public void testInvalidPartitionKeyTypeWithKeyGeneratorClass() {
testInvalidPartitionKeyType(getInvalidPartitionKeyTypeProps(true));
}
@Test
public void testInvalidPartitionKeyTypeWithKeyGeneratorType() {
testInvalidPartitionKeyType(getInvalidPartitionKeyTypeProps(false));
}
public void testInvalidPartitionKeyType(TypedProperties props) {
try {
BuiltinKeyGenerator keyGenerator =
(BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(props);
keyGenerator.getKey(getRecord());
Assertions.fail("should fail when invalid PartitionKeyType is provided!");
} catch (Exception e) {
Assertions.assertTrue(e.getMessage().contains("No enum constant org.apache.hudi.keygen.CustomAvroKeyGenerator.PartitionKeyType.DUMMY"));
}
try {
BuiltinKeyGenerator keyGenerator =
(BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(props);
GenericRecord record = getRecord();
Row row = KeyGeneratorTestUtilities.getRow(record);
keyGenerator.getPartitionPath(row);
Assertions.fail("should fail when invalid PartitionKeyType is provided!");
} catch (Exception e) {
Assertions.assertTrue(e.getMessage().contains("No enum constant org.apache.hudi.keygen.CustomAvroKeyGenerator.PartitionKeyType.DUMMY"));
}
}
@Test
public void testNoRecordKeyFieldPropWithKeyGeneratorClass() {
testNoRecordKeyFieldProp(true);
}
@Test
public void testNoRecordKeyFieldPropWithKeyGeneratorType() {
testNoRecordKeyFieldProp(false);
}
public void testNoRecordKeyFieldProp(boolean useKeyGeneratorClassName) {
TypedProperties propsWithoutRecordKeyFieldProps = getPropsWithoutRecordKeyFieldProps(useKeyGeneratorClassName);
try {
BuiltinKeyGenerator keyGenerator =
(BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(propsWithoutRecordKeyFieldProps);
keyGenerator.getKey(getRecord());
Assertions.fail("should fail when record key field is not provided!");
} catch (Exception e) {
if (useKeyGeneratorClassName) {
// "Property hoodie.datasource.write.recordkey.field not found" exception cause CustomKeyGenerator init fail
Assertions.assertTrue(e
.getCause()
.getCause()
.getCause()
.getMessage()
.contains("Property hoodie.datasource.write.recordkey.field not found"));
} else {
Assertions.assertTrue(stackTraceToString(e).contains("Property hoodie.datasource.write.recordkey.field not found"));
}
}
try {
BuiltinKeyGenerator keyGenerator =
(BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(propsWithoutRecordKeyFieldProps);
GenericRecord record = getRecord();
Row row = KeyGeneratorTestUtilities.getRow(record);
keyGenerator.getRecordKey(row);
Assertions.fail("should fail when record key field is not provided!");
} catch (Exception e) {
if (useKeyGeneratorClassName) {
// "Property hoodie.datasource.write.recordkey.field not found" exception cause CustomKeyGenerator init fail
Assertions.assertTrue(e
.getCause()
.getCause()
.getCause()
.getMessage()
.contains("Property hoodie.datasource.write.recordkey.field not found"));
} else {
Assertions.assertTrue(stackTraceToString(e).contains("Property hoodie.datasource.write.recordkey.field not found"));
}
}
}
@Test
public void testPartitionFieldsInImproperFormatWithKeyGeneratorClass() {
testPartitionFieldsInImproperFormat(getImproperPartitionFieldFormatProp(true));
}
@Test
public void testPartitionFieldsInImproperFormatWithKeyGeneratorType() {
testPartitionFieldsInImproperFormat(getImproperPartitionFieldFormatProp(false));
}
public void testPartitionFieldsInImproperFormat(TypedProperties props) {
try {
BuiltinKeyGenerator keyGenerator =
(BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(props);
keyGenerator.getKey(getRecord());
Assertions.fail("should fail when partition key field is provided in improper format!");
} catch (Exception e) {
Assertions.assertTrue(e.getMessage().contains("Unable to find field names for partition path in proper format"));
}
try {
BuiltinKeyGenerator keyGenerator =
(BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(props);
GenericRecord record = getRecord();
Row row = KeyGeneratorTestUtilities.getRow(record);
keyGenerator.getPartitionPath(row);
Assertions.fail("should fail when partition key field is provided in improper format!");
} catch (Exception e) {
Assertions.assertTrue(e.getMessage().contains("Unable to find field names for partition path in proper format"));
}
}
@Test
public void testComplexRecordKeyWithSimplePartitionPathWithKeyGeneratorClass() throws IOException {
testComplexRecordKeyWithSimplePartitionPath(getComplexRecordKeyWithSimplePartitionProps(true));
}
@Test
public void testComplexRecordKeyWithSimplePartitionPathWithKeyGeneratorType() throws IOException {
testComplexRecordKeyWithSimplePartitionPath(getComplexRecordKeyWithSimplePartitionProps(false));
}
public void testComplexRecordKeyWithSimplePartitionPath(TypedProperties props) throws IOException {
BuiltinKeyGenerator keyGenerator =
(BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(props);
GenericRecord record = getRecord();
HoodieKey key = keyGenerator.getKey(record);
Assertions.assertEquals(key.getRecordKey(), "_row_key:key1,pii_col:pi");
Assertions.assertEquals(key.getPartitionPath(), "timestamp=4357686");
Row row = KeyGeneratorTestUtilities.getRow(record);
Assertions.assertEquals(keyGenerator.getRecordKey(row), "_row_key:key1,pii_col:pi");
Assertions.assertEquals(keyGenerator.getPartitionPath(row), "timestamp=4357686");
InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row);
Assertions.assertEquals(keyGenerator.getPartitionPath(internalRow, row.schema()), "timestamp=4357686");
}
@Test
public void testComplexRecordKeysWithComplexPartitionPathWithKeyGeneratorClass() throws IOException {
testComplexRecordKeysWithComplexPartitionPath(getComplexRecordKeyAndPartitionPathProps(true));
}
@Test
public void testComplexRecordKeysWithComplexPartitionPathWithKeyGeneratorType() throws IOException {
testComplexRecordKeysWithComplexPartitionPath(getComplexRecordKeyAndPartitionPathProps(false));
}
public void testComplexRecordKeysWithComplexPartitionPath(TypedProperties props) throws IOException {
BuiltinKeyGenerator keyGenerator =
(BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(props);
GenericRecord record = getRecord();
HoodieKey key = keyGenerator.getKey(record);
Assertions.assertEquals(key.getRecordKey(), "_row_key:key1,pii_col:pi");
Assertions.assertEquals(key.getPartitionPath(), "timestamp=4357686/ts_ms=20200321");
Row row = KeyGeneratorTestUtilities.getRow(record);
Assertions.assertEquals(keyGenerator.getRecordKey(row), "_row_key:key1,pii_col:pi");
Assertions.assertEquals(keyGenerator.getPartitionPath(row), "timestamp=4357686/ts_ms=20200321");
InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row);
Assertions.assertEquals(keyGenerator.getPartitionPath(internalRow, row.schema()), "timestamp=4357686/ts_ms=20200321");
}
}

View File

@@ -1,87 +0,0 @@
/*
* 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.keygen;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.exception.HoodieKeyException;
import org.apache.avro.generic.GenericRecord;
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
import org.apache.hudi.testutils.KeyGeneratorTestUtilities;
import org.apache.spark.sql.Row;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
public class TestGlobalDeleteKeyGenerator extends KeyGeneratorTestUtilities {
private TypedProperties getCommonProps(boolean getComplexRecordKey) {
TypedProperties properties = new TypedProperties();
if (getComplexRecordKey) {
properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key,pii_col");
} else {
properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key");
}
properties.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE.key(), "true");
return properties;
}
private TypedProperties getPropertiesWithoutRecordKeyProp() {
TypedProperties properties = new TypedProperties();
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp");
return properties;
}
private TypedProperties getWrongRecordKeyFieldProps() {
TypedProperties properties = new TypedProperties();
properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_wrong_key");
return properties;
}
private TypedProperties getProps() {
TypedProperties properties = getCommonProps(true);
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp,ts_ms");
return properties;
}
@Test
public void testNullRecordKeyFields() {
Assertions.assertThrows(IllegalArgumentException.class, () -> new GlobalDeleteKeyGenerator(getPropertiesWithoutRecordKeyProp()));
}
@Test
public void testWrongRecordKeyField() {
GlobalDeleteKeyGenerator keyGenerator = new GlobalDeleteKeyGenerator(getWrongRecordKeyFieldProps());
Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord()));
Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.buildFieldPositionMapIfNeeded(KeyGeneratorTestUtilities.structType));
}
@Test
public void testHappyFlow() {
GlobalDeleteKeyGenerator keyGenerator = new GlobalDeleteKeyGenerator(getProps());
GenericRecord record = getRecord();
HoodieKey key = keyGenerator.getKey(record);
Assertions.assertEquals(key.getRecordKey(), "_row_key:key1,pii_col:pi");
Assertions.assertEquals(key.getPartitionPath(), "");
keyGenerator.buildFieldPositionMapIfNeeded(KeyGeneratorTestUtilities.structType);
Row row = KeyGeneratorTestUtilities.getRow(record);
Assertions.assertEquals(keyGenerator.getRecordKey(row), "_row_key:key1,pii_col:pi");
Assertions.assertEquals(keyGenerator.getPartitionPath(row), "");
}
}

View File

@@ -1,136 +0,0 @@
/*
* 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.keygen;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.exception.HoodieKeyException;
import org.apache.avro.generic.GenericRecord;
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
import org.apache.hudi.testutils.KeyGeneratorTestUtilities;
import org.apache.spark.sql.Row;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
import static junit.framework.TestCase.assertEquals;
public class TestNonpartitionedKeyGenerator extends KeyGeneratorTestUtilities {
private TypedProperties getCommonProps(boolean getComplexRecordKey) {
TypedProperties properties = new TypedProperties();
if (getComplexRecordKey) {
properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key, pii_col");
} else {
properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key");
}
properties.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE.key(), "true");
return properties;
}
private TypedProperties getPropertiesWithoutPartitionPathProp() {
return getCommonProps(false);
}
private TypedProperties getPropertiesWithPartitionPathProp() {
TypedProperties properties = getCommonProps(true);
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp,ts_ms");
return properties;
}
private TypedProperties getPropertiesWithoutRecordKeyProp() {
TypedProperties properties = new TypedProperties();
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp");
return properties;
}
private TypedProperties getWrongRecordKeyFieldProps() {
TypedProperties properties = new TypedProperties();
properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_wrong_key");
return properties;
}
@Test
public void testNullRecordKeyFields() {
Assertions.assertThrows(IllegalArgumentException.class, () -> new NonpartitionedKeyGenerator(getPropertiesWithoutRecordKeyProp()));
}
@Test
public void testNonNullPartitionPathFields() {
TypedProperties properties = getPropertiesWithPartitionPathProp();
NonpartitionedKeyGenerator keyGenerator = new NonpartitionedKeyGenerator(properties);
GenericRecord record = getRecord();
Row row = KeyGeneratorTestUtilities.getRow(record);
Assertions.assertEquals(properties.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key()), "timestamp,ts_ms");
Assertions.assertEquals(keyGenerator.getPartitionPath(row), "");
}
@Test
public void testNullPartitionPathFields() {
TypedProperties properties = getPropertiesWithoutPartitionPathProp();
NonpartitionedKeyGenerator keyGenerator = new NonpartitionedKeyGenerator(properties);
GenericRecord record = getRecord();
Row row = KeyGeneratorTestUtilities.getRow(record);
Assertions.assertEquals(keyGenerator.getPartitionPath(row), "");
}
@Test
public void testWrongRecordKeyField() {
NonpartitionedKeyGenerator keyGenerator = new NonpartitionedKeyGenerator(getWrongRecordKeyFieldProps());
Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord()));
Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.buildFieldPositionMapIfNeeded(KeyGeneratorTestUtilities.structType));
}
@Test
public void testSingleValueKeyGeneratorNonPartitioned() {
TypedProperties properties = new TypedProperties();
properties.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "timestamp");
properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "");
NonpartitionedKeyGenerator keyGenerator = new NonpartitionedKeyGenerator(properties);
assertEquals(keyGenerator.getRecordKeyFields().size(), 1);
assertEquals(keyGenerator.getPartitionPathFields().size(), 0);
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator();
GenericRecord record = dataGenerator.generateGenericRecords(1).get(0);
String rowKey = record.get("timestamp").toString();
HoodieKey hoodieKey = keyGenerator.getKey(record);
assertEquals(rowKey, hoodieKey.getRecordKey());
assertEquals("", hoodieKey.getPartitionPath());
}
@Test
public void testMultipleValueKeyGeneratorNonPartitioned1() {
TypedProperties properties = new TypedProperties();
properties.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "timestamp,driver");
properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "");
NonpartitionedKeyGenerator keyGenerator = new NonpartitionedKeyGenerator(properties);
assertEquals(keyGenerator.getRecordKeyFields().size(), 2);
assertEquals(keyGenerator.getPartitionPathFields().size(), 0);
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator();
GenericRecord record = dataGenerator.generateGenericRecords(1).get(0);
String rowKey =
"timestamp" + ComplexAvroKeyGenerator.DEFAULT_RECORD_KEY_SEPARATOR + record.get("timestamp").toString() + ","
+ "driver" + ComplexAvroKeyGenerator.DEFAULT_RECORD_KEY_SEPARATOR + record.get("driver").toString();
String partitionPath = "";
HoodieKey hoodieKey = keyGenerator.getKey(record);
assertEquals(rowKey, hoodieKey.getRecordKey());
assertEquals(partitionPath, hoodieKey.getPartitionPath());
}
}

View File

@@ -1,163 +0,0 @@
/*
* 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.keygen;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.exception.HoodieKeyException;
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
import org.apache.hudi.testutils.KeyGeneratorTestUtilities;
import org.apache.avro.generic.GenericRecord;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.catalyst.InternalRow;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.MethodSource;
import java.util.stream.Stream;
import static org.apache.hudi.keygen.KeyGenUtils.HUDI_DEFAULT_PARTITION_PATH;
public class TestSimpleKeyGenerator extends KeyGeneratorTestUtilities {
private TypedProperties getCommonProps() {
TypedProperties properties = new TypedProperties();
properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key");
properties.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE.key(), "true");
return properties;
}
private TypedProperties getPropertiesWithoutPartitionPathProp() {
return getCommonProps();
}
private TypedProperties getPropertiesWithoutRecordKeyProp() {
TypedProperties properties = new TypedProperties();
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp");
return properties;
}
private TypedProperties getWrongRecordKeyFieldProps() {
TypedProperties properties = new TypedProperties();
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp");
properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_wrong_key");
return properties;
}
private TypedProperties getWrongPartitionPathFieldProps() {
TypedProperties properties = new TypedProperties();
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "_wrong_partition_path");
properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key");
return properties;
}
private TypedProperties getComplexRecordKeyProp() {
TypedProperties properties = new TypedProperties();
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp");
properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key,pii_col");
return properties;
}
private TypedProperties getProps() {
TypedProperties properties = getCommonProps();
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp");
return properties;
}
private TypedProperties getPropsWithNestedPartitionPathField() {
TypedProperties properties = getCommonProps();
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "nested_col.prop1");
return properties;
}
@Test
public void testNullPartitionPathFields() {
Assertions.assertThrows(IllegalArgumentException.class, () -> new SimpleKeyGenerator(getPropertiesWithoutPartitionPathProp()));
}
@Test
public void testNullRecordKeyFields() {
Assertions.assertThrows(IllegalArgumentException.class, () -> new SimpleKeyGenerator(getPropertiesWithoutRecordKeyProp()));
}
@Test
public void testWrongRecordKeyField() {
SimpleKeyGenerator keyGenerator = new SimpleKeyGenerator(getWrongRecordKeyFieldProps());
Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord()));
Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.buildFieldPositionMapIfNeeded(KeyGeneratorTestUtilities.structType));
}
@Test
public void testWrongPartitionPathField() {
SimpleKeyGenerator keyGenerator = new SimpleKeyGenerator(getWrongPartitionPathFieldProps());
GenericRecord record = getRecord();
Assertions.assertEquals(keyGenerator.getPartitionPath(record), KeyGenUtils.HUDI_DEFAULT_PARTITION_PATH);
Assertions.assertEquals(keyGenerator.getPartitionPath(KeyGeneratorTestUtilities.getRow(record)),
KeyGenUtils.HUDI_DEFAULT_PARTITION_PATH);
}
@Test
public void testComplexRecordKeyField() {
SimpleKeyGenerator keyGenerator = new SimpleKeyGenerator(getComplexRecordKeyProp());
Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord()));
Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.buildFieldPositionMapIfNeeded(KeyGeneratorTestUtilities.structType));
}
@Test
public void testHappyFlow() {
SimpleKeyGenerator keyGenerator = new SimpleKeyGenerator(getProps());
GenericRecord record = getRecord();
HoodieKey key = keyGenerator.getKey(getRecord());
Assertions.assertEquals(key.getRecordKey(), "key1");
Assertions.assertEquals(key.getPartitionPath(), "timestamp=4357686");
Row row = KeyGeneratorTestUtilities.getRow(record);
Assertions.assertEquals(keyGenerator.getRecordKey(row), "key1");
Assertions.assertEquals(keyGenerator.getPartitionPath(row), "timestamp=4357686");
InternalRow internalRow = KeyGeneratorTestUtilities.getInternalRow(row);
Assertions.assertEquals(keyGenerator.getPartitionPath(internalRow, row.schema()), "timestamp=4357686");
}
private static Stream<GenericRecord> nestedColTestRecords() {
return Stream.of(null, getNestedColRecord(null, 10L),
getNestedColRecord("", 10L), getNestedColRecord("val1", 10L));
}
@ParameterizedTest
@MethodSource("nestedColTestRecords")
public void testNestedPartitionPathField(GenericRecord nestedColRecord) {
SimpleKeyGenerator keyGenerator = new SimpleKeyGenerator(getPropsWithNestedPartitionPathField());
GenericRecord record = getRecord(nestedColRecord);
String partitionPathFieldValue = null;
if (nestedColRecord != null) {
partitionPathFieldValue = (String) nestedColRecord.get("prop1");
}
String expectedPartitionPath = "nested_col.prop1="
+ (partitionPathFieldValue != null && !partitionPathFieldValue.isEmpty() ? partitionPathFieldValue : HUDI_DEFAULT_PARTITION_PATH);
HoodieKey key = keyGenerator.getKey(record);
Assertions.assertEquals("key1", key.getRecordKey());
Assertions.assertEquals(expectedPartitionPath, key.getPartitionPath());
Row row = KeyGeneratorTestUtilities.getRow(record);
Assertions.assertEquals("key1", keyGenerator.getRecordKey(row));
Assertions.assertEquals(expectedPartitionPath, keyGenerator.getPartitionPath(row));
}
}

View File

@@ -1,435 +0,0 @@
/*
* 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.keygen;
import org.apache.hudi.AvroConversionHelper;
import org.apache.hudi.AvroConversionUtils;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.testutils.SchemaTestUtil;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.hudi.exception.HoodieKeyGeneratorException;
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
import org.apache.hudi.testutils.KeyGeneratorTestUtilities;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema;
import org.apache.spark.sql.types.StructType;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import java.io.IOException;
import java.math.BigDecimal;
import scala.Function1;
import static org.junit.jupiter.api.Assertions.assertEquals;
public class TestTimestampBasedKeyGenerator {
private GenericRecord baseRecord;
private TypedProperties properties = new TypedProperties();
private Schema schema;
private StructType structType;
private Row baseRow;
private InternalRow internalRow;
@BeforeEach
public void initialize() throws IOException {
schema = SchemaTestUtil.getTimestampEvolvedSchema();
structType = AvroConversionUtils.convertAvroSchemaToStructType(schema);
baseRecord = SchemaTestUtil
.generateAvroRecordFromJson(schema, 1, "001", "f1");
baseRow = genericRecordToRow(baseRecord);
internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow);
properties.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "field1");
properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "createTime");
properties.setProperty(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE.key(), "false");
}
private TypedProperties getBaseKeyConfig(String timestampType, String dateFormat, String timezone, String scalarType) {
properties.setProperty(KeyGeneratorOptions.Config.TIMESTAMP_TYPE_FIELD_PROP, timestampType);
properties.setProperty(KeyGeneratorOptions.Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP, dateFormat);
properties.setProperty(KeyGeneratorOptions.Config.TIMESTAMP_TIMEZONE_FORMAT_PROP, timezone);
if (scalarType != null) {
properties.setProperty("hoodie.deltastreamer.keygen.timebased.timestamp.scalar.time.unit", scalarType);
}
return properties;
}
private Row genericRecordToRow(GenericRecord baseRecord) {
Function1<Object, Object> convertor = AvroConversionHelper.createConverterToRow(baseRecord.getSchema(), structType);
Row row = (Row) convertor.apply(baseRecord);
int fieldCount = structType.fieldNames().length;
Object[] values = new Object[fieldCount];
for (int i = 0; i < fieldCount; i++) {
values[i] = row.get(i);
}
return new GenericRowWithSchema(values, structType);
}
private TypedProperties getBaseKeyConfig(String timestampType, String inputFormatList, String inputFormatDelimiterRegex, String inputTimezone, String outputFormat, String outputTimezone) {
if (timestampType != null) {
properties.setProperty(KeyGeneratorOptions.Config.TIMESTAMP_TYPE_FIELD_PROP, timestampType);
}
if (inputFormatList != null) {
properties.setProperty(KeyGeneratorOptions.Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP, inputFormatList);
}
if (inputFormatDelimiterRegex != null) {
properties.setProperty(KeyGeneratorOptions.Config.TIMESTAMP_INPUT_DATE_FORMAT_LIST_DELIMITER_REGEX_PROP, inputFormatDelimiterRegex);
}
if (inputTimezone != null) {
properties.setProperty(KeyGeneratorOptions.Config.TIMESTAMP_INPUT_TIMEZONE_FORMAT_PROP, inputTimezone);
}
if (outputFormat != null) {
properties.setProperty(KeyGeneratorOptions.Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP, outputFormat);
}
if (outputTimezone != null) {
properties.setProperty(KeyGeneratorOptions.Config.TIMESTAMP_OUTPUT_TIMEZONE_FORMAT_PROP, outputTimezone);
}
return properties;
}
@Test
public void testTimestampBasedKeyGenerator() throws IOException {
// timezone is GMT+8:00
baseRecord.put("createTime", 1578283932000L);
properties = getBaseKeyConfig("EPOCHMILLISECONDS", "yyyy-MM-dd hh", "GMT+8:00", null);
TimestampBasedKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
assertEquals("2020-01-06 12", hk1.getPartitionPath());
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020-01-06 12", keyGen.getPartitionPath(baseRow));
internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow);
assertEquals("2020-01-06 12", keyGen.getPartitionPath(internalRow, baseRow.schema()));
// timezone is GMT+8:00, createTime is BigDecimal
baseRecord.put("createTime", new BigDecimal(1578283932000.00001));
properties = getBaseKeyConfig("EPOCHMILLISECONDS", "yyyy-MM-dd hh", "GMT+8:00", null);
keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey bigDecimalKey = keyGen.getKey(baseRecord);
assertEquals("2020-01-06 12", bigDecimalKey.getPartitionPath());
// test w/ Row
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020-01-06 12", keyGen.getPartitionPath(baseRow));
// timezone is GMT
properties = getBaseKeyConfig("EPOCHMILLISECONDS", "yyyy-MM-dd hh", "GMT", null);
keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk2 = keyGen.getKey(baseRecord);
assertEquals("2020-01-06 04", hk2.getPartitionPath());
// test w/ Row
assertEquals("2020-01-06 04", keyGen.getPartitionPath(baseRow));
// timestamp is DATE_STRING, timezone is GMT+8:00
baseRecord.put("createTime", "2020-01-06 12:12:12");
properties = getBaseKeyConfig("DATE_STRING", "yyyy-MM-dd hh", "GMT+8:00", null);
properties.setProperty("hoodie.deltastreamer.keygen.timebased.input.dateformat", "yyyy-MM-dd hh:mm:ss");
keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk3 = keyGen.getKey(baseRecord);
assertEquals("2020-01-06 12", hk3.getPartitionPath());
// test w/ Row
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020-01-06 12", keyGen.getPartitionPath(baseRow));
// timezone is GMT
properties = getBaseKeyConfig("DATE_STRING", "yyyy-MM-dd hh", "GMT", null);
keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk4 = keyGen.getKey(baseRecord);
assertEquals("2020-01-06 12", hk4.getPartitionPath());
// test w/ Row
assertEquals("2020-01-06 12", keyGen.getPartitionPath(baseRow));
// timezone is GMT+8:00, createTime is null
baseRecord.put("createTime", null);
properties = getBaseKeyConfig("EPOCHMILLISECONDS", "yyyy-MM-dd hh", "GMT+8:00", null);
keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk5 = keyGen.getKey(baseRecord);
assertEquals("1970-01-01 08", hk5.getPartitionPath());
// test w/ Row
baseRow = genericRecordToRow(baseRecord);
assertEquals("1970-01-01 08", keyGen.getPartitionPath(baseRow));
internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow);
assertEquals("1970-01-01 08", keyGen.getPartitionPath(internalRow, baseRow.schema()));
// timestamp is DATE_STRING, timezone is GMT, createTime is null
baseRecord.put("createTime", null);
properties = getBaseKeyConfig("DATE_STRING", "yyyy-MM-dd hh:mm:ss", "GMT", null);
properties.setProperty("hoodie.deltastreamer.keygen.timebased.input.dateformat", "yyyy-MM-dd hh:mm:ss");
keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk6 = keyGen.getKey(baseRecord);
assertEquals("1970-01-01 12:00:00", hk6.getPartitionPath());
// test w/ Row
baseRow = genericRecordToRow(baseRecord);
assertEquals("1970-01-01 12:00:00", keyGen.getPartitionPath(baseRow));
internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow);
assertEquals("1970-01-01 12:00:00", keyGen.getPartitionPath(internalRow, baseRow.schema()));
}
@Test
public void testScalar() throws IOException {
// timezone is GMT+8:00
baseRecord.put("createTime", 20000L);
// timezone is GMT
properties = getBaseKeyConfig("SCALAR", "yyyy-MM-dd hh", "GMT", "days");
TimestampBasedKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
assertEquals(hk1.getPartitionPath(), "2024-10-04 12");
// test w/ Row
baseRow = genericRecordToRow(baseRecord);
assertEquals("2024-10-04 12", keyGen.getPartitionPath(baseRow));
internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow);
assertEquals("2024-10-04 12", keyGen.getPartitionPath(internalRow, baseRow.schema()));
// timezone is GMT, createTime is null
baseRecord.put("createTime", null);
properties = getBaseKeyConfig("SCALAR", "yyyy-MM-dd hh", "GMT", "days");
keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk2 = keyGen.getKey(baseRecord);
assertEquals("1970-01-02 12", hk2.getPartitionPath());
// test w/ Row
baseRow = genericRecordToRow(baseRecord);
assertEquals("1970-01-02 12", keyGen.getPartitionPath(baseRow));
internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow);
assertEquals("1970-01-02 12", keyGen.getPartitionPath(internalRow, baseRow.schema()));
// timezone is GMT. number of days store integer in mysql
baseRecord.put("createTime", 18736);
properties = getBaseKeyConfig("SCALAR", "yyyy-MM-dd", "GMT", "DAYS");
keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey scalarSecondsKey = keyGen.getKey(baseRecord);
assertEquals("2021-04-19", scalarSecondsKey.getPartitionPath());
// test w/ Row
baseRow = genericRecordToRow(baseRecord);
assertEquals("2021-04-19", keyGen.getPartitionPath(baseRow));
}
@Test
public void testScalarWithLogicalType() throws IOException {
schema = SchemaTestUtil.getTimestampWithLogicalTypeSchema();
structType = AvroConversionUtils.convertAvroSchemaToStructType(schema);
baseRecord = SchemaTestUtil.generateAvroRecordFromJson(schema, 1, "001", "f1");
baseRecord.put("createTime", 1638513806000000L);
properties = getBaseKeyConfig("SCALAR", "yyyy/MM/dd", "GMT", "MICROSECONDS");
properties.setProperty(KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(), "true");
TimestampBasedKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
assertEquals("2021/12/03", hk1.getPartitionPath());
// test w/ Row
baseRow = genericRecordToRow(baseRecord);
assertEquals("2021/12/03", keyGen.getPartitionPath(baseRow));
internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow);
assertEquals("2021/12/03", keyGen.getPartitionPath(internalRow, baseRow.schema()));
// timezone is GMT, createTime is null
baseRecord.put("createTime", null);
properties = getBaseKeyConfig("SCALAR", "yyyy/MM/dd", "GMT", "MICROSECONDS");
properties.setProperty(KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(), "true");
keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk2 = keyGen.getKey(baseRecord);
assertEquals("1970/01/01", hk2.getPartitionPath());
// test w/ Row
baseRow = genericRecordToRow(baseRecord);
assertEquals("1970/01/01", keyGen.getPartitionPath(baseRow));
internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow);
assertEquals("1970/01/01", keyGen.getPartitionPath(internalRow, baseRow.schema()));
}
@Test
public void test_ExpectsMatch_SingleInputFormat_ISO8601WithMsZ_OutputTimezoneAsUTC() throws IOException {
baseRecord.put("createTime", "2020-04-01T13:01:33.428Z");
properties = this.getBaseKeyConfig(
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"GMT");
BuiltinKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
Assertions.assertEquals("2020040113", hk1.getPartitionPath());
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020040113", keyGen.getPartitionPath(baseRow));
}
@Test
public void test_ExpectsMatch_SingleInputFormats_ISO8601WithMsZ_OutputTimezoneAsInputDateTimeZone() throws IOException {
baseRecord.put("createTime", "2020-04-01T13:01:33.428Z");
properties = this.getBaseKeyConfig(
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"");
BuiltinKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
Assertions.assertEquals("2020040113", hk1.getPartitionPath());
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020040113", keyGen.getPartitionPath(baseRow));
}
@Test
public void test_ExpectsMatch_MultipleInputFormats_ISO8601WithMsZ_OutputTimezoneAsUTC() throws IOException {
baseRecord.put("createTime", "2020-04-01T13:01:33.428Z");
properties = this.getBaseKeyConfig(
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"UTC");
BuiltinKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
Assertions.assertEquals("2020040113", hk1.getPartitionPath());
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020040113", keyGen.getPartitionPath(baseRow));
}
@Test
public void test_ExpectsMatch_MultipleInputFormats_ISO8601NoMsZ_OutputTimezoneAsUTC() throws IOException {
baseRecord.put("createTime", "2020-04-01T13:01:33Z");
properties = this.getBaseKeyConfig(
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"UTC");
BuiltinKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
Assertions.assertEquals("2020040113", hk1.getPartitionPath());
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020040113", keyGen.getPartitionPath(baseRow));
}
@Test
public void test_ExpectsMatch_MultipleInputFormats_ISO8601NoMsWithOffset_OutputTimezoneAsUTC() throws IOException {
baseRecord.put("createTime", "2020-04-01T13:01:33-05:00");
properties = this.getBaseKeyConfig(
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"UTC");
BuiltinKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
Assertions.assertEquals("2020040118", hk1.getPartitionPath());
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020040118", keyGen.getPartitionPath(baseRow));
}
@Test
public void test_ExpectsMatch_MultipleInputFormats_ISO8601WithMsWithOffset_OutputTimezoneAsUTC() throws IOException {
baseRecord.put("createTime", "2020-04-01T13:01:33.123-05:00");
properties = this.getBaseKeyConfig(
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"UTC");
BuiltinKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
Assertions.assertEquals("2020040118", hk1.getPartitionPath());
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020040118", keyGen.getPartitionPath(baseRow));
}
@Test
public void test_ExpectsMatch_MultipleInputFormats_ISO8601WithMsZ_OutputTimezoneAsEST() throws IOException {
baseRecord.put("createTime", "2020-04-01T13:01:33.123Z");
properties = this.getBaseKeyConfig(
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"EST");
BuiltinKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
Assertions.assertEquals("2020040109", hk1.getPartitionPath());
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020040109", keyGen.getPartitionPath(baseRow));
}
@Test
public void test_Throws_MultipleInputFormats_InputDateNotMatchingFormats() throws IOException {
baseRecord.put("createTime", "2020-04-01 13:01:33.123-05:00");
properties = this.getBaseKeyConfig(
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ",
"",
"",
"yyyyMMddHH",
"UTC");
BuiltinKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
Assertions.assertThrows(HoodieKeyGeneratorException.class, () -> keyGen.getKey(baseRecord));
baseRow = genericRecordToRow(baseRecord);
Assertions.assertThrows(HoodieKeyGeneratorException.class, () -> keyGen.getPartitionPath(baseRow));
}
@Test
public void test_ExpectsMatch_MultipleInputFormats_ShortDate_OutputCustomDate() throws IOException {
baseRecord.put("createTime", "20200401");
properties = this.getBaseKeyConfig(
"DATE_STRING",
"yyyy-MM-dd'T'HH:mm:ssZ,yyyy-MM-dd'T'HH:mm:ss.SSSZ,yyyyMMdd",
"",
"UTC",
"MM/dd/yyyy",
"UTC");
BuiltinKeyGenerator keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk1 = keyGen.getKey(baseRecord);
Assertions.assertEquals("04/01/2020", hk1.getPartitionPath());
baseRow = genericRecordToRow(baseRecord);
assertEquals("04/01/2020", keyGen.getPartitionPath(baseRow));
}
}

View File

@@ -1,101 +0,0 @@
/*
* 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.keygen.factory;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieKeyGeneratorException;
import org.apache.hudi.keygen.ComplexKeyGenerator;
import org.apache.hudi.keygen.CustomKeyGenerator;
import org.apache.hudi.keygen.GlobalDeleteKeyGenerator;
import org.apache.hudi.keygen.KeyGenerator;
import org.apache.hudi.keygen.NonpartitionedKeyGenerator;
import org.apache.hudi.keygen.SimpleKeyGenerator;
import org.apache.hudi.keygen.TimestampBasedKeyGenerator;
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
import org.apache.hudi.keygen.constant.KeyGeneratorType;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.Arguments;
import org.junit.jupiter.params.provider.MethodSource;
import java.io.IOException;
import java.util.stream.Stream;
public class TestCreateKeyGeneratorByTypeWithFactory {
private TypedProperties props;
private static Stream<Arguments> configParams() {
String[] types = {KeyGeneratorType.SIMPLE.name(), KeyGeneratorType.TIMESTAMP.name(), KeyGeneratorType.COMPLEX.name(),
KeyGeneratorType.CUSTOM.name(), KeyGeneratorType.NON_PARTITION.name(), KeyGeneratorType.GLOBAL_DELETE.name()};
return Stream.of(types).map(Arguments::of);
}
@BeforeEach
public void init() {
props = new TypedProperties();
props.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key");
props.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE.key(), "true");
props.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp");
// for timestamp based key generator
props.put("hoodie.deltastreamer.keygen.timebased.timestamp.type", "DATE_STRING");
props.put("hoodie.deltastreamer.keygen.timebased.input.dateformat", "yyyy-MM-dd");
props.put("hoodie.deltastreamer.keygen.timebased.output.dateformat", "yyyyMMdd");
}
@AfterEach
public void teardown() {
props = null;
}
@ParameterizedTest
@MethodSource("configParams")
public void testKeyGeneratorTypes(String keyGenType) throws IOException {
props.put(HoodieWriteConfig.KEYGENERATOR_TYPE.key(), keyGenType);
KeyGeneratorType keyType = KeyGeneratorType.valueOf(keyGenType);
KeyGenerator keyGenerator = HoodieSparkKeyGeneratorFactory.createKeyGenerator(props);
switch (keyType) {
case SIMPLE:
Assertions.assertEquals(SimpleKeyGenerator.class.getName(), keyGenerator.getClass().getName());
return;
case COMPLEX:
Assertions.assertEquals(ComplexKeyGenerator.class.getName(), keyGenerator.getClass().getName());
return;
case TIMESTAMP:
Assertions.assertEquals(TimestampBasedKeyGenerator.class.getName(), keyGenerator.getClass().getName());
return;
case CUSTOM:
Assertions.assertEquals(CustomKeyGenerator.class.getName(), keyGenerator.getClass().getName());
return;
case NON_PARTITION:
Assertions.assertEquals(NonpartitionedKeyGenerator.class.getName(), keyGenerator.getClass().getName());
return;
case GLOBAL_DELETE:
Assertions.assertEquals(GlobalDeleteKeyGenerator.class.getName(), keyGenerator.getClass().getName());
return;
default:
throw new HoodieKeyGeneratorException("Unsupported keyGenerator Type " + keyGenType);
}
}
}

View File

@@ -1,82 +0,0 @@
/*
* 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.keygen.factory;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieKeyGeneratorException;
import org.apache.hudi.keygen.KeyGenerator;
import org.apache.hudi.keygen.SimpleKeyGenerator;
import org.apache.hudi.keygen.TestComplexKeyGenerator;
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
import org.apache.hudi.keygen.constant.KeyGeneratorType;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
import java.io.IOException;
import static org.junit.jupiter.api.Assertions.assertThrows;
/**
* This class assist test KeyGenerator configuration(class name and type) priority.
* <p>
* The functional test of KeyGenerator is left to other unit tests. {@link TestComplexKeyGenerator etc.}.
*/
public class TestHoodieSparkKeyGeneratorFactory {
@Test
public void testKeyGeneratorFactory() throws IOException {
TypedProperties props = getCommonProps();
// set KeyGenerator type only
props.put(HoodieWriteConfig.KEYGENERATOR_TYPE.key(), KeyGeneratorType.SIMPLE.name());
KeyGenerator keyGenerator = HoodieSparkKeyGeneratorFactory.createKeyGenerator(props);
Assertions.assertEquals(SimpleKeyGenerator.class.getName(), keyGenerator.getClass().getName());
// set KeyGenerator class only
props = getCommonProps();
props.put(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key(), SimpleKeyGenerator.class.getName());
KeyGenerator keyGenerator2 = HoodieSparkKeyGeneratorFactory.createKeyGenerator(props);
Assertions.assertEquals(SimpleKeyGenerator.class.getName(), keyGenerator2.getClass().getName());
// set both class name and keyGenerator type
props.put(HoodieWriteConfig.KEYGENERATOR_TYPE.key(), KeyGeneratorType.CUSTOM.name());
KeyGenerator keyGenerator3 = HoodieSparkKeyGeneratorFactory.createKeyGenerator(props);
// KEYGENERATOR_TYPE_PROP was overwritten by KEYGENERATOR_CLASS_PROP
Assertions.assertEquals(SimpleKeyGenerator.class.getName(), keyGenerator3.getClass().getName());
// set wrong class name
final TypedProperties props2 = getCommonProps();
props2.put(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key(), TestHoodieSparkKeyGeneratorFactory.class.getName());
assertThrows(IOException.class, () -> HoodieSparkKeyGeneratorFactory.createKeyGenerator(props2));
// set wrong keyGenerator type
final TypedProperties props3 = getCommonProps();
props3.put(HoodieWriteConfig.KEYGENERATOR_TYPE.key(), "wrong_type");
assertThrows(HoodieKeyGeneratorException.class, () -> HoodieSparkKeyGeneratorFactory.createKeyGenerator(props3));
}
private TypedProperties getCommonProps() {
TypedProperties properties = new TypedProperties();
properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key");
properties.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE.key(), "true");
properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "timestamp");
return properties;
}
}

View File

@@ -18,12 +18,10 @@
package org.apache.hudi.testutils;
import org.apache.hudi.AvroConversionHelper;
import org.apache.hudi.AvroConversionUtils;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericRecord;
import org.apache.hudi.AvroConversionUtils;
import org.apache.spark.package$;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.catalyst.InternalRow;
@@ -33,16 +31,15 @@ import org.apache.spark.sql.catalyst.encoders.RowEncoder;
import org.apache.spark.sql.catalyst.expressions.Attribute;
import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema;
import org.apache.spark.sql.types.StructType;
import scala.Function1;
import scala.collection.JavaConversions;
import scala.collection.JavaConverters;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
import java.util.List;
import java.util.stream.Collectors;
import scala.Function1;
import scala.collection.JavaConversions;
import scala.collection.JavaConverters;
public class KeyGeneratorTestUtilities {
public static final String NESTED_COL_SCHEMA = "{\"type\":\"record\", \"name\":\"nested_col\",\"fields\": ["
@@ -51,8 +48,7 @@ public class KeyGeneratorTestUtilities {
+ "{\"name\": \"timestamp\",\"type\": \"long\"},{\"name\": \"_row_key\", \"type\": \"string\"},"
+ "{\"name\": \"ts_ms\", \"type\": \"string\"},"
+ "{\"name\": \"pii_col\", \"type\": \"string\"},"
+ "{\"name\": \"nested_col\",\"type\": "
+ NESTED_COL_SCHEMA + "}"
+ "{\"name\": \"nested_col\",\"type\": [\"null\", " + NESTED_COL_SCHEMA + "]}"
+ "]}";
public static final String TEST_STRUCTNAME = "test_struct_name";
@@ -86,8 +82,8 @@ public class KeyGeneratorTestUtilities {
}
public static Row getRow(GenericRecord record, Schema schema, StructType structType) {
Function1<Object, Object> converterFn = AvroConversionHelper.createConverterToRow(schema, structType);
Row row = (Row) converterFn.apply(record);
Function1<GenericRecord, Row> converterFn = AvroConversionUtils.createConverterToRow(schema, structType);
Row row = converterFn.apply(record);
int fieldCount = structType.fieldNames().length;
Object[] values = new Object[fieldCount];
for (int i = 0; i < fieldCount; i++) {