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

@@ -19,7 +19,6 @@
package org.apache.hudi;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.api.java.function.ReduceFunction;
import org.apache.spark.sql.Dataset;
@@ -30,14 +29,13 @@ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
import org.apache.spark.sql.catalyst.encoders.RowEncoder;
import org.apache.spark.sql.catalyst.expressions.Attribute;
import org.apache.spark.sql.types.StructType;
import java.util.List;
import java.util.stream.Collectors;
import scala.Tuple2;
import scala.collection.JavaConversions;
import scala.collection.JavaConverters;
import java.util.List;
import java.util.stream.Collectors;
/**
* Helper class to assist in deduplicating Rows for BulkInsert with Rows.
*/
@@ -55,20 +53,13 @@ public class SparkRowWriteHelper {
}
public Dataset<Row> deduplicateRows(Dataset<Row> inputDf, String preCombineField, boolean isGlobalIndex) {
ExpressionEncoder encoder = getEncoder(inputDf.schema());
return inputDf.groupByKey(
(MapFunction<Row, String>) value ->
isGlobalIndex ? (value.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD)) :
(value.getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD) + "+" + value.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD)), Encoders.STRING())
.reduceGroups((ReduceFunction<Row>) (v1, v2) -> {
if (((Comparable) v1.getAs(preCombineField)).compareTo(((Comparable) v2.getAs(preCombineField))) >= 0) {
return v1;
} else {
return v2;
}
}
).map((MapFunction<Tuple2<String, Row>, Row>) value -> value._2, encoder);
return inputDf.groupByKey((MapFunction<Row, String>) value ->
isGlobalIndex
? (value.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD))
: (value.getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD) + "+" + value.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD)), Encoders.STRING())
.reduceGroups((ReduceFunction<Row>) (v1, v2) ->
((Comparable) v1.getAs(preCombineField)).compareTo(v2.getAs(preCombineField)) >= 0 ? v1 : v2)
.map((MapFunction<Tuple2<String, Row>, Row>) value -> value._2, getEncoder(inputDf.schema()));
}
private ExpressionEncoder getEncoder(StructType schema) {

View File

@@ -18,17 +18,30 @@
package org.apache.hudi
import org.apache.avro.Schema
import org.apache.avro.generic.GenericRecord
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.apache.hadoop.hbase.io.hfile.CacheConfig
import org.apache.hudi.common.config.SerializableConfiguration
import org.apache.hudi.common.fs.FSUtils
import org.apache.hudi.common.model.HoodieFileFormat
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
import org.apache.hudi.common.util.StringUtils
import org.apache.hudi.io.storage.HoodieHFileReader
import org.apache.hudi.metadata.HoodieTableMetadata
import org.apache.spark.internal.Logging
import org.apache.spark.sql.{SQLContext, SparkSession}
import org.apache.spark.sql.avro.SchemaConverters
import org.apache.spark.sql.sources.{BaseRelation, PrunedFilteredScan}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.execution.datasources.PartitionedFile
import org.apache.spark.sql.sources.{BaseRelation, Filter, PrunedFilteredScan}
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.{SQLContext, SparkSession}
import scala.collection.JavaConverters._
import scala.util.Try
case class HoodieTableSchema(structTypeSchema: StructType, avroSchemaStr: String)
/**
* Hoodie BaseRelation which extends [[PrunedFilteredScan]].
*/
@@ -41,15 +54,105 @@ abstract class HoodieBaseRelation(
protected val sparkSession: SparkSession = sqlContext.sparkSession
protected val tableAvroSchema: Schema = {
protected lazy val tableAvroSchema: Schema = {
val schemaUtil = new TableSchemaResolver(metaClient)
Try (schemaUtil.getTableAvroSchema).getOrElse(SchemaConverters.toAvroType(userSchema.get))
Try(schemaUtil.getTableAvroSchema).getOrElse(
// If there is no commit in the table, we can't get the schema
// t/h [[TableSchemaResolver]], fallback to the provided [[userSchema]] instead.
userSchema match {
case Some(s) => SchemaConverters.toAvroType(s)
case _ => throw new IllegalArgumentException("User-provided schema is required in case the table is empty")
}
)
}
protected val tableStructSchema: StructType = AvroConversionUtils.convertAvroSchemaToStructType(tableAvroSchema)
protected val partitionColumns: Array[String] = metaClient.getTableConfig.getPartitionFields.orElse(Array.empty)
override def schema: StructType = userSchema.getOrElse(tableStructSchema)
protected def getPrecombineFieldProperty: Option[String] =
Option(metaClient.getTableConfig.getPreCombineField)
.orElse(optParams.get(DataSourceWriteOptions.PRECOMBINE_FIELD.key)) match {
// NOTE: This is required to compensate for cases when empty string is used to stub
// property value to avoid it being set with the default value
// TODO(HUDI-3456) cleanup
case Some(f) if !StringUtils.isNullOrEmpty(f) => Some(f)
case _ => None
}
override def schema: StructType = tableStructSchema
}
object HoodieBaseRelation {
def isMetadataTable(metaClient: HoodieTableMetaClient) =
HoodieTableMetadata.isMetadataTable(metaClient.getBasePath)
/**
* Returns file-reader routine accepting [[PartitionedFile]] and returning an [[Iterator]]
* over [[InternalRow]]
*/
def createBaseFileReader(spark: SparkSession,
partitionSchema: StructType,
tableSchema: HoodieTableSchema,
requiredSchema: HoodieTableSchema,
filters: Seq[Filter],
options: Map[String, String],
hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = {
val hfileReader = createHFileReader(
spark = spark,
tableSchema = tableSchema,
requiredSchema = requiredSchema,
filters = filters,
options = options,
hadoopConf = hadoopConf
)
val parquetReader = HoodieDataSourceHelper.buildHoodieParquetReader(
sparkSession = spark,
dataSchema = tableSchema.structTypeSchema,
partitionSchema = partitionSchema,
requiredSchema = requiredSchema.structTypeSchema,
filters = filters,
options = options,
hadoopConf = hadoopConf
)
partitionedFile => {
val extension = FSUtils.getFileExtension(partitionedFile.filePath)
if (HoodieFileFormat.PARQUET.getFileExtension.equals(extension)) {
parquetReader.apply(partitionedFile)
} else if (HoodieFileFormat.HFILE.getFileExtension.equals(extension)) {
hfileReader.apply(partitionedFile)
} else {
throw new UnsupportedOperationException(s"Base file format not supported by Spark DataSource ($partitionedFile)")
}
}
}
private def createHFileReader(spark: SparkSession,
tableSchema: HoodieTableSchema,
requiredSchema: HoodieTableSchema,
filters: Seq[Filter],
options: Map[String, String],
hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = {
val hadoopConfBroadcast =
spark.sparkContext.broadcast(new SerializableConfiguration(hadoopConf))
partitionedFile => {
val hadoopConf = hadoopConfBroadcast.value.get()
val reader = new HoodieHFileReader[GenericRecord](hadoopConf, new Path(partitionedFile.filePath),
new CacheConfig(hadoopConf))
val requiredRowSchema = requiredSchema.structTypeSchema
// NOTE: Schema has to be parsed at this point, since Avro's [[Schema]] aren't serializable
// to be passed from driver to executor
val requiredAvroSchema = new Schema.Parser().parse(requiredSchema.avroSchemaStr)
val avroToRowConverter = AvroConversionUtils.createAvroToInternalRowConverter(requiredAvroSchema, requiredRowSchema)
reader.getRecordIterator(requiredAvroSchema).asScala
.map(record => {
avroToRowConverter.apply(record.asInstanceOf[GenericRecord]).get
})
}
}
}

View File

@@ -20,18 +20,16 @@ package org.apache.hudi
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.FileStatus
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{Expression, PredicateHelper, SpecificInternalRow, SubqueryExpression, UnsafeProjection}
import org.apache.spark.sql.execution.datasources.PartitionedFile
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
import org.apache.spark.sql.execution.datasources.{FilePartition, PartitionedFile}
import org.apache.spark.sql.sources.Filter
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.vectorized.ColumnarBatch
import scala.collection.JavaConverters._
import scala.collection.mutable.ArrayBuffer
object HoodieDataSourceHelper extends PredicateHelper {
@@ -77,14 +75,13 @@ object HoodieDataSourceHelper extends PredicateHelper {
* Wrapper `buildReaderWithPartitionValues` of [[ParquetFileFormat]]
* to deal with [[ColumnarBatch]] when enable parquet vectorized reader if necessary.
*/
def buildHoodieParquetReader(
sparkSession: SparkSession,
dataSchema: StructType,
partitionSchema: StructType,
requiredSchema: StructType,
filters: Seq[Filter],
options: Map[String, String],
hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = {
def buildHoodieParquetReader(sparkSession: SparkSession,
dataSchema: StructType,
partitionSchema: StructType,
requiredSchema: StructType,
filters: Seq[Filter],
options: Map[String, String],
hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = {
val readParquetFile: PartitionedFile => Iterator[Any] = new ParquetFileFormat().buildReaderWithPartitionValues(
sparkSession = sparkSession,
@@ -98,11 +95,10 @@ object HoodieDataSourceHelper extends PredicateHelper {
file: PartitionedFile => {
val iter = readParquetFile(file)
val rows = iter.flatMap(_ match {
iter.flatMap {
case r: InternalRow => Seq(r)
case b: ColumnarBatch => b.rowIterator().asScala
})
rows
}
}
}

View File

@@ -23,14 +23,19 @@ import org.apache.avro.generic.{GenericRecord, GenericRecordBuilder}
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.apache.hudi.HoodieDataSourceHelper._
import org.apache.hudi.HoodieMergeOnReadRDD.resolveAvroSchemaNullability
import org.apache.hudi.MergeOnReadSnapshotRelation.getFilePath
import org.apache.hudi.common.config.HoodieMetadataConfig
import org.apache.hudi.common.engine.HoodieLocalEngineContext
import org.apache.hudi.common.fs.FSUtils
import org.apache.hudi.common.fs.FSUtils.getRelativePartitionPath
import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner
import org.apache.hudi.config.HoodiePayloadConfig
import org.apache.hudi.exception.HoodieException
import org.apache.hudi.hadoop.config.HoodieRealtimeConfig
import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.HOODIE_RECORD_KEY_COL_POS
import org.apache.hudi.metadata.HoodieTableMetadata.getDataTableBasePathFromMetadataTable
import org.apache.hudi.metadata.{HoodieBackedTableMetadata, HoodieTableMetadata}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.avro.{HoodieAvroDeserializer, HoodieAvroSerializer}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.UnsafeProjection
import org.apache.spark.sql.execution.datasources.PartitionedFile
@@ -48,51 +53,38 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext,
@transient config: Configuration,
fullSchemaFileReader: PartitionedFile => Iterator[InternalRow],
requiredSchemaFileReader: PartitionedFile => Iterator[InternalRow],
tableState: HoodieMergeOnReadTableState)
tableState: HoodieMergeOnReadTableState,
tableSchema: HoodieTableSchema,
requiredSchema: HoodieTableSchema)
extends RDD[InternalRow](sc, Nil) {
private val confBroadcast = sc.broadcast(new SerializableWritable(config))
private val preCombineField = tableState.preCombineField
private val recordKeyFieldOpt = tableState.recordKeyFieldOpt
private val payloadProps = if (preCombineField.isDefined) {
HoodiePayloadConfig.newBuilder
.withPayloadOrderingField(preCombineField.get)
.build.getProps
} else {
new Properties()
}
private val requiredSchema = tableState.requiredStructSchema
private val requiredFieldPosition = HoodieSparkUtils.collectFieldIndexes(requiredSchema,
tableState.tableStructSchema
)
private val recordKeyField = tableState.recordKeyField
private val payloadProps = tableState.preCombineFieldOpt
.map(preCombineField =>
HoodiePayloadConfig.newBuilder
.withPayloadOrderingField(preCombineField)
.build
.getProps
)
.getOrElse(new Properties())
override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = {
val mergeOnReadPartition = split.asInstanceOf[HoodieMergeOnReadPartition]
val iter = mergeOnReadPartition.split match {
case dataFileOnlySplit if dataFileOnlySplit.logPaths.isEmpty =>
val rows = requiredSchemaFileReader(dataFileOnlySplit.dataFile.get)
extractRequiredSchema(rows, requiredSchema, requiredFieldPosition)
case dataFileOnlySplit if dataFileOnlySplit.logFiles.isEmpty =>
requiredSchemaFileReader(dataFileOnlySplit.dataFile.get)
case logFileOnlySplit if logFileOnlySplit.dataFile.isEmpty =>
logFileIterator(logFileOnlySplit, getConfig)
case skipMergeSplit if skipMergeSplit.mergeType
.equals(DataSourceReadOptions.REALTIME_SKIP_MERGE_OPT_VAL) =>
skipMergeFileIterator(
skipMergeSplit,
requiredSchemaFileReader(skipMergeSplit.dataFile.get),
getConfig
)
case payloadCombineSplit if payloadCombineSplit.mergeType
.equals(DataSourceReadOptions.REALTIME_PAYLOAD_COMBINE_OPT_VAL) =>
payloadCombineFileIterator(
payloadCombineSplit,
fullSchemaFileReader(payloadCombineSplit.dataFile.get),
getConfig
)
case skipMergeSplit if skipMergeSplit.mergeType.equals(DataSourceReadOptions.REALTIME_SKIP_MERGE_OPT_VAL) =>
skipMergeFileIterator(skipMergeSplit, requiredSchemaFileReader(skipMergeSplit.dataFile.get), getConfig)
case payloadCombineSplit
if payloadCombineSplit.mergeType.equals(DataSourceReadOptions.REALTIME_PAYLOAD_COMBINE_OPT_VAL) =>
payloadCombineFileIterator(payloadCombineSplit, fullSchemaFileReader(payloadCombineSplit.dataFile.get),
getConfig)
case _ => throw new HoodieException(s"Unable to select an Iterator to read the Hoodie MOR File Split for " +
s"file path: ${mergeOnReadPartition.split.dataFile.get.filePath}" +
s"log paths: ${mergeOnReadPartition.split.logPaths.toString}" +
s"log paths: ${mergeOnReadPartition.split.logFiles.toString}" +
s"hoodie table path: ${mergeOnReadPartition.split.tablePath}" +
s"spark partition Index: ${mergeOnReadPartition.index}" +
s"merge type: ${mergeOnReadPartition.split.mergeType}")
@@ -121,12 +113,15 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext,
private def logFileIterator(split: HoodieMergeOnReadFileSplit,
config: Configuration): Iterator[InternalRow] =
new Iterator[InternalRow] with Closeable {
private val tableAvroSchema = new Schema.Parser().parse(tableState.tableAvroSchema)
private val requiredAvroSchema = new Schema.Parser().parse(tableState.requiredAvroSchema)
new Iterator[InternalRow] with Closeable with SparkAdapterSupport {
private val tableAvroSchema = new Schema.Parser().parse(tableSchema.avroSchemaStr)
private val requiredAvroSchema = new Schema.Parser().parse(requiredSchema.avroSchemaStr)
private val requiredFieldPosition =
requiredSchema.structTypeSchema
.map(f => tableAvroSchema.getField(f.name).pos()).toList
private val recordBuilder = new GenericRecordBuilder(requiredAvroSchema)
private val deserializer = HoodieAvroDeserializer(requiredAvroSchema, tableState.requiredStructSchema)
private val unsafeProjection = UnsafeProjection.create(tableState.requiredStructSchema)
private val deserializer = sparkAdapter.createAvroDeserializer(requiredAvroSchema, requiredSchema.structTypeSchema)
private val unsafeProjection = UnsafeProjection.create(requiredSchema.structTypeSchema)
private var logScanner = HoodieMergeOnReadRDD.scanLog(split, tableAvroSchema, config)
private val logRecords = logScanner.getRecords
private val logRecordsKeyIterator = logRecords.keySet().iterator().asScala
@@ -141,9 +136,10 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext,
// delete record found, skipping
this.hasNext
} else {
val requiredAvroRecord = AvroConversionUtils
.buildAvroRecordBySchema(curAvroRecord.get(), requiredAvroSchema, requiredFieldPosition, recordBuilder)
recordToLoad = unsafeProjection(deserializer.deserializeData(requiredAvroRecord).asInstanceOf[InternalRow])
val requiredAvroRecord = AvroConversionUtils.buildAvroRecordBySchema(curAvroRecord.get(), requiredAvroSchema,
requiredFieldPosition, recordBuilder)
val rowOpt = deserializer.deserialize(requiredAvroRecord)
recordToLoad = unsafeProjection(rowOpt.get.asInstanceOf[InternalRow])
true
}
} else {
@@ -169,12 +165,15 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext,
private def skipMergeFileIterator(split: HoodieMergeOnReadFileSplit,
baseFileIterator: Iterator[InternalRow],
config: Configuration): Iterator[InternalRow] =
new Iterator[InternalRow] with Closeable {
private val tableAvroSchema = new Schema.Parser().parse(tableState.tableAvroSchema)
private val requiredAvroSchema = new Schema.Parser().parse(tableState.requiredAvroSchema)
new Iterator[InternalRow] with Closeable with SparkAdapterSupport {
private val tableAvroSchema = new Schema.Parser().parse(tableSchema.avroSchemaStr)
private val requiredAvroSchema = new Schema.Parser().parse(requiredSchema.avroSchemaStr)
private val requiredFieldPosition =
requiredSchema.structTypeSchema
.map(f => tableAvroSchema.getField(f.name).pos()).toList
private val recordBuilder = new GenericRecordBuilder(requiredAvroSchema)
private val deserializer = HoodieAvroDeserializer(requiredAvroSchema, tableState.requiredStructSchema)
private val unsafeProjection = UnsafeProjection.create(tableState.requiredStructSchema)
private val deserializer = sparkAdapter.createAvroDeserializer(requiredAvroSchema, requiredSchema.structTypeSchema)
private val unsafeProjection = UnsafeProjection.create(requiredSchema.structTypeSchema)
private var logScanner = HoodieMergeOnReadRDD.scanLog(split, tableAvroSchema, config)
private val logRecords = logScanner.getRecords
private val logRecordsKeyIterator = logRecords.keySet().iterator().asScala
@@ -185,7 +184,7 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext,
override def hasNext: Boolean = {
if (baseFileIterator.hasNext) {
val curRow = baseFileIterator.next()
recordToLoad = unsafeProjection(createInternalRowWithSchema(curRow, requiredSchema, requiredFieldPosition))
recordToLoad = unsafeProjection(curRow)
true
} else {
if (logRecordsKeyIterator.hasNext) {
@@ -195,9 +194,10 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext,
// delete record found, skipping
this.hasNext
} else {
val requiredAvroRecord = AvroConversionUtils
.buildAvroRecordBySchema(curAvroRecord.get(), requiredAvroSchema, requiredFieldPosition, recordBuilder)
recordToLoad = unsafeProjection(deserializer.deserializeData(requiredAvroRecord).asInstanceOf[InternalRow])
val requiredAvroRecord = AvroConversionUtils.buildAvroRecordBySchema(curAvroRecord.get(), requiredAvroSchema,
requiredFieldPosition, recordBuilder)
val rowOpt = deserializer.deserialize(requiredAvroRecord)
recordToLoad = unsafeProjection(rowOpt.get.asInstanceOf[InternalRow])
true
}
} else {
@@ -224,18 +224,22 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext,
private def payloadCombineFileIterator(split: HoodieMergeOnReadFileSplit,
baseFileIterator: Iterator[InternalRow],
config: Configuration): Iterator[InternalRow] =
new Iterator[InternalRow] with Closeable {
private val tableAvroSchema = new Schema.Parser().parse(tableState.tableAvroSchema)
private val requiredAvroSchema = new Schema.Parser().parse(tableState.requiredAvroSchema)
private val serializer = HoodieAvroSerializer(tableState.tableStructSchema, tableAvroSchema, false)
private val requiredDeserializer = HoodieAvroDeserializer(requiredAvroSchema, tableState.requiredStructSchema)
new Iterator[InternalRow] with Closeable with SparkAdapterSupport {
private val tableAvroSchema = new Schema.Parser().parse(tableSchema.avroSchemaStr)
private val requiredAvroSchema = new Schema.Parser().parse(requiredSchema.avroSchemaStr)
private val requiredFieldPosition =
requiredSchema.structTypeSchema
.map(f => tableAvroSchema.getField(f.name).pos()).toList
private val serializer = sparkAdapter.createAvroSerializer(tableSchema.structTypeSchema, tableAvroSchema,
resolveAvroSchemaNullability(tableAvroSchema))
private val requiredDeserializer = sparkAdapter.createAvroDeserializer(requiredAvroSchema, requiredSchema.structTypeSchema)
private val recordBuilder = new GenericRecordBuilder(requiredAvroSchema)
private val unsafeProjection = UnsafeProjection.create(tableState.requiredStructSchema)
private val unsafeProjection = UnsafeProjection.create(requiredSchema.structTypeSchema)
private var logScanner = HoodieMergeOnReadRDD.scanLog(split, tableAvroSchema, config)
private val logRecords = logScanner.getRecords
private val logRecordsKeyIterator = logRecords.keySet().iterator().asScala
private val keyToSkip = mutable.Set.empty[String]
private val recordKeyPosition = if (recordKeyFieldOpt.isEmpty) HOODIE_RECORD_KEY_COL_POS else tableState.tableStructSchema.fieldIndex(recordKeyFieldOpt.get)
private val recordKeyPosition = tableSchema.structTypeSchema.fieldIndex(recordKeyField)
private var recordToLoad: InternalRow = _
@@ -253,20 +257,15 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext,
this.hasNext
} else {
// load merged record as InternalRow with required schema
val requiredAvroRecord = AvroConversionUtils
.buildAvroRecordBySchema(
mergedAvroRecord.get(),
requiredAvroSchema,
requiredFieldPosition,
recordBuilder
)
recordToLoad = unsafeProjection(requiredDeserializer
.deserializeData(requiredAvroRecord).asInstanceOf[InternalRow])
val requiredAvroRecord = AvroConversionUtils.buildAvroRecordBySchema(mergedAvroRecord.get(), requiredAvroSchema,
requiredFieldPosition, recordBuilder)
val rowOpt = requiredDeserializer.deserialize(requiredAvroRecord)
recordToLoad = unsafeProjection(rowOpt.get.asInstanceOf[InternalRow])
true
}
} else {
// No merge needed, load current row with required schema
recordToLoad = unsafeProjection(createInternalRowWithSchema(curRow, requiredSchema, requiredFieldPosition))
recordToLoad = unsafeProjection(createInternalRowWithSchema(curRow, requiredSchema.structTypeSchema, requiredFieldPosition))
true
}
} else {
@@ -287,8 +286,8 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext,
requiredFieldPosition,
recordBuilder
)
recordToLoad = unsafeProjection(requiredDeserializer
.deserializeData(requiredAvroRecord).asInstanceOf[InternalRow])
val rowOpt = requiredDeserializer.deserialize(requiredAvroRecord)
recordToLoad = unsafeProjection(rowOpt.get.asInstanceOf[InternalRow])
true
}
}
@@ -312,8 +311,8 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext,
private def mergeRowWithLog(curRow: InternalRow, curKey: String) = {
val historyAvroRecord = serializer.serialize(curRow).asInstanceOf[GenericRecord]
logRecords.get(curKey).getData.combineAndGetUpdateValue(
historyAvroRecord, tableAvroSchema, payloadProps)
logRecords.get(curKey).getData
.combineAndGetUpdateValue(historyAvroRecord, tableAvroSchema, payloadProps)
}
}
}
@@ -323,32 +322,60 @@ private object HoodieMergeOnReadRDD {
def scanLog(split: HoodieMergeOnReadFileSplit, logSchema: Schema, config: Configuration): HoodieMergedLogRecordScanner = {
val fs = FSUtils.getFs(split.tablePath, config)
val partitionPath: String = if (split.logPaths.isEmpty || split.logPaths.get.asJava.isEmpty) {
null
val logFiles = split.logFiles.get
if (HoodieTableMetadata.isMetadataTable(split.tablePath)) {
val metadataConfig = HoodieMetadataConfig.newBuilder().enable(true).build()
val dataTableBasePath = getDataTableBasePathFromMetadataTable(split.tablePath)
val metadataTable = new HoodieBackedTableMetadata(
new HoodieLocalEngineContext(config), metadataConfig,
dataTableBasePath,
config.get(HoodieRealtimeConfig.SPILLABLE_MAP_BASE_PATH_PROP, HoodieRealtimeConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH))
// NOTE: In case of Metadata Table partition path equates to partition name (since there's just one level
// of indirection among MT partitions)
val relativePartitionPath = getRelativePartitionPath(new Path(split.tablePath), getPartitionPath(split))
metadataTable.getLogRecordScanner(logFiles.asJava, relativePartitionPath).getLeft
} else {
new Path(split.logPaths.get.asJava.get(0)).getParent.getName
val logRecordScannerBuilder = HoodieMergedLogRecordScanner.newBuilder()
.withFileSystem(fs)
.withBasePath(split.tablePath)
.withLogFilePaths(split.logFiles.get.map(logFile => getFilePath(logFile.getPath)).asJava)
.withReaderSchema(logSchema)
.withLatestInstantTime(split.latestCommit)
.withReadBlocksLazily(
Try(config.get(HoodieRealtimeConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP,
HoodieRealtimeConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED).toBoolean)
.getOrElse(false))
.withReverseReader(false)
.withBufferSize(
config.getInt(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP,
HoodieRealtimeConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE))
.withMaxMemorySizeInBytes(split.maxCompactionMemoryInBytes)
.withSpillableMapBasePath(
config.get(HoodieRealtimeConfig.SPILLABLE_MAP_BASE_PATH_PROP,
HoodieRealtimeConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH))
if (logFiles.nonEmpty) {
logRecordScannerBuilder.withPartition(getRelativePartitionPath(new Path(split.tablePath), logFiles.head.getPath.getParent))
}
logRecordScannerBuilder.build()
}
val logRecordScannerBuilder = HoodieMergedLogRecordScanner.newBuilder()
.withFileSystem(fs)
.withBasePath(split.tablePath)
.withLogFilePaths(split.logPaths.get.asJava)
.withReaderSchema(logSchema)
.withLatestInstantTime(split.latestCommit)
.withReadBlocksLazily(
Try(config.get(HoodieRealtimeConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP,
HoodieRealtimeConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED).toBoolean)
.getOrElse(false))
.withReverseReader(false)
.withBufferSize(
config.getInt(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP,
HoodieRealtimeConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE))
.withMaxMemorySizeInBytes(split.maxCompactionMemoryInBytes)
.withSpillableMapBasePath(
config.get(HoodieRealtimeConfig.SPILLABLE_MAP_BASE_PATH_PROP,
HoodieRealtimeConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH))
if (partitionPath != null) {
logRecordScannerBuilder.withPartition(partitionPath)
}
private def getPartitionPath(split: HoodieMergeOnReadFileSplit): Path = {
// Determine partition path as an immediate parent folder of either
// - The base file
// - Some log file
split.dataFile.map(baseFile => new Path(baseFile.filePath))
.getOrElse(split.logFiles.get.head.getPath)
.getParent
}
private def resolveAvroSchemaNullability(schema: Schema) = {
AvroConversionUtils.resolveAvroTypeNullability(schema) match {
case (nullable, _) => nullable
}
logRecordScannerBuilder.build()
}
}

View File

@@ -17,8 +17,10 @@
package org.apache.hudi
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{GlobPattern, Path}
import org.apache.hadoop.mapred.JobConf
import org.apache.hudi.HoodieBaseRelation.createBaseFileReader
import org.apache.hudi.common.model.HoodieRecord
import org.apache.hudi.common.table.HoodieTableMetaClient
import org.apache.hudi.common.table.view.HoodieTableFileSystemView
@@ -35,17 +37,17 @@ import org.apache.spark.sql.{Row, SQLContext}
import scala.collection.JavaConversions._
/**
* Experimental.
* Relation, that implements the Hoodie incremental view for Merge On Read table.
*
*/
* Experimental.
* Relation, that implements the Hoodie incremental view for Merge On Read table.
*
*/
class MergeOnReadIncrementalRelation(sqlContext: SQLContext,
val optParams: Map[String, String],
val userSchema: Option[StructType],
val metaClient: HoodieTableMetaClient)
extends HoodieBaseRelation(sqlContext, metaClient, optParams, userSchema) {
private val conf = sqlContext.sparkContext.hadoopConfiguration
private val conf = new Configuration(sqlContext.sparkContext.hadoopConfiguration)
private val jobConf = new JobConf(conf)
private val commitTimeline = metaClient.getCommitsAndCompactionTimeline.filterCompletedInstants()
@@ -75,84 +77,89 @@ class MergeOnReadIncrementalRelation(sqlContext: SQLContext,
private val fileIndex = if (commitsToReturn.isEmpty) List() else buildFileIndex()
private val preCombineField = {
val preCombineFieldFromTableConfig = metaClient.getTableConfig.getPreCombineField
if (preCombineFieldFromTableConfig != null) {
Some(preCombineFieldFromTableConfig)
} else {
// get preCombineFiled from the options if this is a old table which have not store
// the field to hoodie.properties
optParams.get(DataSourceReadOptions.READ_PRE_COMBINE_FIELD.key)
}
private val preCombineFieldOpt = getPrecombineFieldProperty
// Record filters making sure that only records w/in the requested bounds are being fetched as part of the
// scan collected by this relation
private lazy val incrementalSpanRecordsFilters: Seq[Filter] = {
val isNotNullFilter = IsNotNull(HoodieRecord.COMMIT_TIME_METADATA_FIELD)
val largerThanFilter = GreaterThanOrEqual(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitsToReturn.head.getTimestamp)
val lessThanFilter = LessThanOrEqual(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitsToReturn.last.getTimestamp)
Seq(isNotNullFilter, largerThanFilter, lessThanFilter)
}
private lazy val mandatoryColumns = {
// NOTE: This columns are required for Incremental flow to be able to handle the rows properly, even in
// cases when no columns are requested to be fetched (for ex, when using {@code count()} API)
Seq(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD) ++
preCombineFieldOpt.map(Seq(_)).getOrElse(Seq())
}
override def needConversion: Boolean = false
override def unhandledFilters(filters: Array[Filter]): Array[Filter] = {
if (fileIndex.isEmpty) {
filters
} else {
val isNotNullFilter = IsNotNull(HoodieRecord.COMMIT_TIME_METADATA_FIELD)
val largerThanFilter = GreaterThanOrEqual(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitsToReturn.head.getTimestamp)
val lessThanFilter = LessThanOrEqual(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitsToReturn.last.getTimestamp)
filters :+ isNotNullFilter :+ largerThanFilter :+ lessThanFilter
}
}
override def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = {
if (fileIndex.isEmpty) {
sqlContext.sparkContext.emptyRDD[Row]
} else {
logDebug(s"buildScan requiredColumns = ${requiredColumns.mkString(",")}")
logDebug(s"buildScan filters = ${filters.mkString(",")}")
// config to ensure the push down filter for parquet will be applied.
sqlContext.sparkSession.sessionState.conf.setConfString("spark.sql.parquet.filterPushdown", "true")
sqlContext.sparkSession.sessionState.conf.setConfString("spark.sql.parquet.recordLevelFilter.enabled", "true")
sqlContext.sparkSession.sessionState.conf.setConfString("spark.sql.parquet.enableVectorizedReader", "false")
val pushDownFilter = {
val isNotNullFilter = IsNotNull(HoodieRecord.COMMIT_TIME_METADATA_FIELD)
val largerThanFilter = GreaterThanOrEqual(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitsToReturn.head.getTimestamp)
val lessThanFilter = LessThanOrEqual(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitsToReturn.last.getTimestamp)
filters :+ isNotNullFilter :+ largerThanFilter :+ lessThanFilter
}
val fetchedColumns: Array[String] = appendMandatoryColumns(requiredColumns)
val (requiredAvroSchema, requiredStructSchema) =
HoodieSparkUtils.getRequiredSchema(tableAvroSchema, requiredColumns)
HoodieSparkUtils.getRequiredSchema(tableAvroSchema, fetchedColumns)
val hoodieTableState = HoodieMergeOnReadTableState(
tableStructSchema,
requiredStructSchema,
tableAvroSchema.toString,
requiredAvroSchema.toString,
fileIndex,
preCombineField,
Option.empty
)
val fullSchemaParquetReader = HoodieDataSourceHelper.buildHoodieParquetReader(
sparkSession = sqlContext.sparkSession,
dataSchema = tableStructSchema,
partitionSchema = StructType(Nil),
requiredSchema = tableStructSchema,
filters = pushDownFilter,
val partitionSchema = StructType(Nil)
val tableSchema = HoodieTableSchema(tableStructSchema, tableAvroSchema.toString)
val requiredSchema = HoodieTableSchema(requiredStructSchema, requiredAvroSchema.toString)
val fullSchemaParquetReader = createBaseFileReader(
spark = sqlContext.sparkSession,
partitionSchema = partitionSchema,
tableSchema = tableSchema,
requiredSchema = tableSchema,
// This file-reader is used to read base file records, subsequently merging them with the records
// stored in delta-log files. As such, we have to read _all_ records from the base file, while avoiding
// applying any user-defined filtering _before_ we complete combining them w/ delta-log records (to make sure that
// we combine them correctly)
//
// The only filtering applicable here is the filtering to make sure we're only fetching records that
// fall into incremental span of the timeline being queried
filters = incrementalSpanRecordsFilters,
options = optParams,
hadoopConf = sqlContext.sparkSession.sessionState.newHadoopConf()
// NOTE: We have to fork the Hadoop Config here as Spark will be modifying it
// to configure Parquet reader appropriately
hadoopConf = new Configuration(conf)
)
val requiredSchemaParquetReader = HoodieDataSourceHelper.buildHoodieParquetReader(
sparkSession = sqlContext.sparkSession,
dataSchema = tableStructSchema,
partitionSchema = StructType(Nil),
requiredSchema = tableStructSchema,
filters = pushDownFilter,
val requiredSchemaParquetReader = createBaseFileReader(
spark = sqlContext.sparkSession,
partitionSchema = partitionSchema,
tableSchema = tableSchema,
requiredSchema = requiredSchema,
filters = filters ++ incrementalSpanRecordsFilters,
options = optParams,
hadoopConf = sqlContext.sparkSession.sessionState.newHadoopConf()
// NOTE: We have to fork the Hadoop Config here as Spark will be modifying it
// to configure Parquet reader appropriately
hadoopConf = new Configuration(conf)
)
val hoodieTableState = HoodieMergeOnReadTableState(fileIndex, HoodieRecord.RECORD_KEY_METADATA_FIELD, preCombineFieldOpt)
// TODO implement incremental span record filtering w/in RDD to make sure returned iterator is appropriately
// filtered, since file-reader might not be capable to perform filtering
val rdd = new HoodieMergeOnReadRDD(
sqlContext.sparkContext,
jobConf,
fullSchemaParquetReader,
requiredSchemaParquetReader,
hoodieTableState
hoodieTableState,
tableSchema,
requiredSchema
)
rdd.asInstanceOf[RDD[Row]]
}
@@ -206,10 +213,9 @@ class MergeOnReadIncrementalRelation(sqlContext: SQLContext,
}
val logPath = if (f.getLatestFileSlice.isPresent) {
//If log path doesn't exist, we still include an empty path to avoid using
// If log path doesn't exist, we still include an empty path to avoid using
// the default parquet reader to ensure the push down filter will be applied.
Option(f.getLatestFileSlice.get().getLogFiles.iterator().toList
.map(logfile => logfile.getPath.toString))
Option(f.getLatestFileSlice.get().getLogFiles.iterator().toList)
}
else {
Option.empty
@@ -219,4 +225,9 @@ class MergeOnReadIncrementalRelation(sqlContext: SQLContext,
latestCommit, metaClient.getBasePath, maxCompactionMemoryInBytes, mergeType)
})
}
private def appendMandatoryColumns(requestedColumns: Array[String]): Array[String] = {
val missing = mandatoryColumns.filter(col => !requestedColumns.contains(col))
requestedColumns ++ missing
}
}

View File

@@ -18,43 +18,37 @@
package org.apache.hudi
import org.apache.hudi.common.model.HoodieLogFile
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.apache.hadoop.mapred.JobConf
import org.apache.hudi.HoodieBaseRelation.{createBaseFileReader, isMetadataTable}
import org.apache.hudi.common.model.{HoodieLogFile, HoodieRecord}
import org.apache.hudi.common.table.HoodieTableMetaClient
import org.apache.hudi.common.table.view.HoodieTableFileSystemView
import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils
import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils.getMaxCompactionMemoryInBytes
import org.apache.hadoop.fs.Path
import org.apache.hadoop.mapred.JobConf
import org.apache.spark.internal.Logging
import org.apache.hudi.metadata.HoodieMetadataPayload
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.avro.SchemaConverters
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.Expression
import org.apache.spark.sql.execution.datasources.{FileStatusCache, PartitionedFile}
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
import org.apache.spark.sql.hudi.HoodieSqlCommonUtils
import org.apache.spark.sql.{Row, SQLContext}
import org.apache.spark.sql.sources.{BaseRelation, Filter, PrunedFilteredScan}
import org.apache.spark.sql.sources.Filter
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.{Row, SQLContext}
import scala.collection.JavaConverters._
case class HoodieMergeOnReadFileSplit(dataFile: Option[PartitionedFile],
logPaths: Option[List[String]],
logFiles: Option[List[HoodieLogFile]],
latestCommit: String,
tablePath: String,
maxCompactionMemoryInBytes: Long,
mergeType: String)
case class HoodieMergeOnReadTableState(tableStructSchema: StructType,
requiredStructSchema: StructType,
tableAvroSchema: String,
requiredAvroSchema: String,
hoodieRealtimeFileSplits: List[HoodieMergeOnReadFileSplit],
preCombineField: Option[String],
recordKeyFieldOpt: Option[String])
case class HoodieMergeOnReadTableState(hoodieRealtimeFileSplits: List[HoodieMergeOnReadFileSplit],
recordKeyField: String,
preCombineFieldOpt: Option[String])
class MergeOnReadSnapshotRelation(sqlContext: SQLContext,
optParams: Map[String, String],
@@ -63,7 +57,7 @@ class MergeOnReadSnapshotRelation(sqlContext: SQLContext,
val metaClient: HoodieTableMetaClient)
extends HoodieBaseRelation(sqlContext, metaClient, optParams, userSchema) {
private val conf = sqlContext.sparkContext.hadoopConfiguration
private val conf = new Configuration(sqlContext.sparkContext.hadoopConfiguration)
private val jobConf = new JobConf(conf)
private val mergeType = optParams.getOrElse(
@@ -72,19 +66,21 @@ class MergeOnReadSnapshotRelation(sqlContext: SQLContext,
private val maxCompactionMemoryInBytes = getMaxCompactionMemoryInBytes(jobConf)
private val preCombineField = {
val preCombineFieldFromTableConfig = metaClient.getTableConfig.getPreCombineField
if (preCombineFieldFromTableConfig != null) {
Some(preCombineFieldFromTableConfig)
} else {
// get preCombineFiled from the options if this is a old table which have not store
// the field to hoodie.properties
optParams.get(DataSourceReadOptions.READ_PRE_COMBINE_FIELD.key)
}
// If meta fields are enabled, always prefer key from the meta field as opposed to user-specified one
// NOTE: This is historical behavior which is preserved as is
private val recordKeyField = {
if (metaClient.getTableConfig.populateMetaFields()) HoodieRecord.RECORD_KEY_METADATA_FIELD
else metaClient.getTableConfig.getRecordKeyFieldProp
}
private var recordKeyFieldOpt = Option.empty[String]
if (!metaClient.getTableConfig.populateMetaFields()) {
recordKeyFieldOpt = Option(metaClient.getTableConfig.getRecordKeyFieldProp)
private val preCombineFieldOpt = getPrecombineFieldProperty
private lazy val mandatoryColumns = {
if (isMetadataTable(metaClient)) {
Seq(HoodieMetadataPayload.KEY_FIELD_NAME, HoodieMetadataPayload.SCHEMA_FIELD_NAME_TYPE)
} else {
Seq(recordKeyField) ++ preCombineFieldOpt.map(Seq(_)).getOrElse(Seq())
}
}
override def needConversion: Boolean = false
@@ -96,45 +92,56 @@ class MergeOnReadSnapshotRelation(sqlContext: SQLContext,
log.debug(s" buildScan requiredColumns = ${requiredColumns.mkString(",")}")
log.debug(s" buildScan filters = ${filters.mkString(",")}")
// NOTE: In case list of requested columns doesn't contain the Primary Key one, we
// have to add it explicitly so that
// - Merging could be performed correctly
// - In case 0 columns are to be fetched (for ex, when doing {@code count()} on Spark's [[Dataset]],
// Spark still fetches all the rows to execute the query correctly
//
// It's okay to return columns that have not been requested by the caller, as those nevertheless will be
// filtered out upstream
val fetchedColumns: Array[String] = appendMandatoryColumns(requiredColumns)
val (requiredAvroSchema, requiredStructSchema) =
HoodieSparkUtils.getRequiredSchema(tableAvroSchema, requiredColumns)
HoodieSparkUtils.getRequiredSchema(tableAvroSchema, fetchedColumns)
val fileIndex = buildFileIndex(filters)
val hoodieTableState = HoodieMergeOnReadTableState(
tableStructSchema,
requiredStructSchema,
tableAvroSchema.toString,
requiredAvroSchema.toString,
fileIndex,
preCombineField,
recordKeyFieldOpt
)
val fullSchemaParquetReader = HoodieDataSourceHelper.buildHoodieParquetReader(
sparkSession = sqlContext.sparkSession,
dataSchema = tableStructSchema,
partitionSchema = StructType(Nil),
requiredSchema = tableStructSchema,
val partitionSchema = StructType(Nil)
val tableSchema = HoodieTableSchema(tableStructSchema, tableAvroSchema.toString)
val requiredSchema = HoodieTableSchema(requiredStructSchema, requiredAvroSchema.toString)
val fullSchemaParquetReader = createBaseFileReader(
spark = sqlContext.sparkSession,
partitionSchema = partitionSchema,
tableSchema = tableSchema,
requiredSchema = tableSchema,
// This file-reader is used to read base file records, subsequently merging them with the records
// stored in delta-log files. As such, we have to read _all_ records from the base file, while avoiding
// applying any filtering _before_ we complete combining them w/ delta-log records (to make sure that
// we combine them correctly)
filters = Seq.empty,
options = optParams,
hadoopConf = sqlContext.sparkSession.sessionState.newHadoopConf()
// NOTE: We have to fork the Hadoop Config here as Spark will be modifying it
// to configure Parquet reader appropriately
hadoopConf = new Configuration(conf)
)
val requiredSchemaParquetReader = HoodieDataSourceHelper.buildHoodieParquetReader(
sparkSession = sqlContext.sparkSession,
dataSchema = tableStructSchema,
partitionSchema = StructType(Nil),
requiredSchema = tableStructSchema,
val requiredSchemaParquetReader = createBaseFileReader(
spark = sqlContext.sparkSession,
partitionSchema = partitionSchema,
tableSchema = tableSchema,
requiredSchema = requiredSchema,
filters = filters,
options = optParams,
hadoopConf = sqlContext.sparkSession.sessionState.newHadoopConf()
// NOTE: We have to fork the Hadoop Config here as Spark will be modifying it
// to configure Parquet reader appropriately
hadoopConf = new Configuration(conf)
)
val rdd = new HoodieMergeOnReadRDD(
sqlContext.sparkContext,
jobConf,
fullSchemaParquetReader,
requiredSchemaParquetReader,
hoodieTableState
)
val tableState = HoodieMergeOnReadTableState(fileIndex, recordKeyField, preCombineFieldOpt)
val rdd = new HoodieMergeOnReadRDD(sqlContext.sparkContext, jobConf, fullSchemaParquetReader,
requiredSchemaParquetReader, tableState, tableSchema, requiredSchema)
rdd.asInstanceOf[RDD[Row]]
}
@@ -214,8 +221,7 @@ class MergeOnReadSnapshotRelation(sqlContext: SQLContext,
Option.empty
}
val logPaths = fileSlice.getLogFiles.sorted(HoodieLogFile.getLogFileComparator).iterator().asScala
.map(logFile => MergeOnReadSnapshotRelation.getFilePath(logFile.getPath)).toList
val logPaths = fileSlice.getLogFiles.sorted(HoodieLogFile.getLogFileComparator).iterator().asScala.toList
val logPathsOptional = if (logPaths.isEmpty) Option.empty else Option(logPaths)
HoodieMergeOnReadFileSplit(partitionedFile, logPathsOptional, queryInstant, metaClient.getBasePath,
@@ -225,6 +231,11 @@ class MergeOnReadSnapshotRelation(sqlContext: SQLContext,
}
}
}
private def appendMandatoryColumns(requestedColumns: Array[String]): Array[String] = {
val missing = mandatoryColumns.filter(col => !requestedColumns.contains(col))
requestedColumns ++ missing
}
}
object MergeOnReadSnapshotRelation {

View File

@@ -20,9 +20,10 @@ package org.apache.spark.sql.avro
import org.apache.avro.Schema
import org.apache.spark.sql.types.DataType
/**
* As AvroSerializer cannot be access out of the spark.sql.avro package since spark 3.1, we define
* this class to be accessed by other class.
*/
case class HoodieAvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: Boolean)
extends AvroSerializer(rootCatalystType, rootAvroType, nullable)
class HoodieAvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: Boolean)
extends HoodieAvroSerializerTrait {
val avroSerializer = new AvroSerializer(rootCatalystType, rootAvroType, nullable)
override def serialize(catalystData: Any): Any = avroSerializer.serialize(catalystData)
}

View File

@@ -25,17 +25,15 @@ import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
import org.apache.hudi.common.util.ValidationUtils
import org.apache.hudi.keygen.ComplexKeyGenerator
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory
import org.apache.spark.internal.Logging
import org.apache.spark.sql.{AnalysisException, SparkSession}
import org.apache.spark.sql.avro.SchemaConverters
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.hudi.HoodieOptionConfig
import org.apache.spark.sql.hudi.HoodieSqlCommonUtils._
import org.apache.spark.sql.types.{StructField, StructType}
import org.apache.spark.sql.{AnalysisException, SparkSession}
import java.util.{Locale, Properties}
import scala.collection.JavaConverters._
import scala.collection.mutable

View File

@@ -42,17 +42,21 @@ trait ProvidesHoodieConfig extends Logging {
val tableConfig = hoodieCatalogTable.tableConfig
val tableId = hoodieCatalogTable.table.identifier
// NOTE: Here we fallback to "" to make sure that null value is not overridden with
// default value ("ts")
// TODO(HUDI-3456) clean up
val preCombineField = Option(tableConfig.getPreCombineField).getOrElse("")
require(hoodieCatalogTable.primaryKeys.nonEmpty,
s"There are no primary key in table ${hoodieCatalogTable.table.identifier}, cannot execute update operator")
val enableHive = isEnableHive(sparkSession)
withSparkConf(sparkSession, catalogProperties) {
Map(
Map.apply(
"path" -> hoodieCatalogTable.tableLocation,
RECORDKEY_FIELD.key -> hoodieCatalogTable.primaryKeys.mkString(","),
PRECOMBINE_FIELD.key -> preCombineField,
TBL_NAME.key -> hoodieCatalogTable.tableName,
PRECOMBINE_FIELD.key -> preCombineField,
HIVE_STYLE_PARTITIONING.key -> tableConfig.getHiveStylePartitioningEnable,
URL_ENCODE_PARTITIONING.key -> tableConfig.getUrlEncodePartitioning,
KEYGENERATOR_CLASS_NAME.key -> classOf[SqlKeyGenerator].getCanonicalName,
@@ -70,6 +74,7 @@ trait ProvidesHoodieConfig extends Logging {
HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key -> "200",
SqlKeyGenerator.PARTITION_SCHEMA -> hoodieCatalogTable.partitionSchema.toDDL
)
.filter { case(_, v) => v != null }
}
}
@@ -98,8 +103,12 @@ trait ProvidesHoodieConfig extends Logging {
val options = hoodieCatalogTable.catalogProperties ++ tableConfig.getProps.asScala.toMap ++ extraOptions
val parameters = withSparkConf(sparkSession, options)()
val preCombineColumn = hoodieCatalogTable.preCombineKey.getOrElse("")
val partitionFields = hoodieCatalogTable.partitionFields.mkString(",")
val partitionFieldsStr = hoodieCatalogTable.partitionFields.mkString(",")
// NOTE: Here we fallback to "" to make sure that null value is not overridden with
// default value ("ts")
// TODO(HUDI-3456) clean up
val preCombineField = hoodieCatalogTable.preCombineKey.getOrElse("")
val hiveStylePartitioningEnable = Option(tableConfig.getHiveStylePartitioningEnable).getOrElse("true")
val urlEncodePartitioning = Option(tableConfig.getUrlEncodePartitioning).getOrElse("false")
@@ -115,7 +124,7 @@ trait ProvidesHoodieConfig extends Logging {
DataSourceWriteOptions.SQL_INSERT_MODE.defaultValue()))
val isNonStrictMode = insertMode == InsertMode.NON_STRICT
val isPartitionedTable = hoodieCatalogTable.partitionFields.nonEmpty
val hasPrecombineColumn = preCombineColumn.nonEmpty
val hasPrecombineColumn = hoodieCatalogTable.preCombineKey.nonEmpty
val operation =
(enableBulkInsert, isOverwrite, dropDuplicate, isNonStrictMode, isPartitionedTable) match {
case (true, _, _, false, _) =>
@@ -147,37 +156,41 @@ trait ProvidesHoodieConfig extends Logging {
} else {
classOf[OverwriteWithLatestAvroPayload].getCanonicalName
}
logInfo(s"insert statement use write operation type: $operation, payloadClass: $payloadClassName")
logInfo(s"Insert statement use write operation type: $operation, payloadClass: $payloadClassName")
val enableHive = isEnableHive(sparkSession)
withSparkConf(sparkSession, options) {
Map(
"path" -> path,
TABLE_TYPE.key -> tableType,
TBL_NAME.key -> hoodieCatalogTable.tableName,
PRECOMBINE_FIELD.key -> preCombineColumn,
OPERATION.key -> operation,
HIVE_STYLE_PARTITIONING.key -> hiveStylePartitioningEnable,
URL_ENCODE_PARTITIONING.key -> urlEncodePartitioning,
KEYGENERATOR_CLASS_NAME.key -> classOf[SqlKeyGenerator].getCanonicalName,
SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME -> keyGeneratorClassName,
RECORDKEY_FIELD.key -> hoodieCatalogTable.primaryKeys.mkString(","),
PARTITIONPATH_FIELD.key -> partitionFields,
PRECOMBINE_FIELD.key -> preCombineField,
PARTITIONPATH_FIELD.key -> partitionFieldsStr,
PAYLOAD_CLASS_NAME.key -> payloadClassName,
ENABLE_ROW_WRITER.key -> enableBulkInsert.toString,
HoodieWriteConfig.COMBINE_BEFORE_INSERT.key -> String.valueOf(hasPrecombineColumn),
HIVE_PARTITION_FIELDS.key -> partitionFieldsStr,
META_SYNC_ENABLED.key -> enableHive.toString,
HIVE_SYNC_MODE.key -> HiveSyncMode.HMS.name(),
HIVE_USE_JDBC.key -> "false",
HIVE_DATABASE.key -> hoodieCatalogTable.table.identifier.database.getOrElse("default"),
HIVE_TABLE.key -> hoodieCatalogTable.table.identifier.table,
HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true",
HIVE_PARTITION_FIELDS.key -> partitionFields,
HIVE_PARTITION_EXTRACTOR_CLASS.key -> classOf[MultiPartKeysValueExtractor].getCanonicalName,
HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key -> "200",
HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key -> "200",
SqlKeyGenerator.PARTITION_SCHEMA -> hoodieCatalogTable.partitionSchema.toDDL
)
.filter { case (_, v) => v != null }
}
}
}

View File

@@ -17,7 +17,6 @@
package org.apache.spark.sql.hudi.command.payload
import java.util.UUID
import org.apache.avro.generic.{GenericRecord, IndexedRecord}
import org.apache.hudi.sql.IExpressionEvaluator
import org.apache.spark.executor.InputMetrics
@@ -37,6 +36,8 @@ import org.apache.spark.{TaskContext, TaskKilledException}
import org.codehaus.commons.compiler.CompileException
import org.codehaus.janino.{ClassBodyEvaluator, InternalCompilerException}
import java.util.UUID
/**
* Do CodeGen for expression based on IndexedRecord.
* The mainly difference with the spark's CodeGen for expression is that

View File

@@ -17,14 +17,9 @@
package org.apache.spark.sql.hudi.command.payload
import java.util.{Base64, Properties}
import java.util.concurrent.Callable
import com.google.common.cache.CacheBuilder
import org.apache.avro.Schema
import org.apache.avro.generic.{GenericData, GenericRecord, IndexedRecord}
import org.apache.hudi.AvroConversionUtils
import org.apache.hudi.DataSourceWriteOptions._
import org.apache.hudi.avro.HoodieAvroUtils
@@ -34,13 +29,14 @@ import org.apache.hudi.common.util.{ValidationUtils, Option => HOption}
import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.io.HoodieWriteHandle
import org.apache.hudi.sql.IExpressionEvaluator
import org.apache.spark.sql.avro.{AvroSerializer, HoodieAvroSerializer, SchemaConverters}
import org.apache.spark.sql.avro.{AvroSerializer, SchemaConverters}
import org.apache.spark.sql.catalyst.expressions.Expression
import org.apache.spark.sql.hudi.SerDeUtils
import org.apache.spark.sql.hudi.command.payload.ExpressionPayload.getEvaluator
import org.apache.spark.sql.types.{StructField, StructType}
import java.util.concurrent.Callable
import java.util.{Base64, Properties}
import scala.collection.JavaConverters._
import scala.collection.mutable.ArrayBuffer
@@ -315,7 +311,7 @@ object ExpressionPayload {
val conditionEvaluator = ExpressionCodeGen.doCodeGen(Seq(condition), conditionSerializer)
val assignSqlType = AvroConversionUtils.convertAvroSchemaToStructType(writeSchema)
val assignSerializer = new HoodieAvroSerializer(assignSqlType, writeSchema, false)
val assignSerializer = new AvroSerializer(assignSqlType, writeSchema, false)
val assignmentEvaluator = ExpressionCodeGen.doCodeGen(assignments, assignSerializer)
conditionEvaluator -> assignmentEvaluator
}

View File

@@ -17,22 +17,19 @@
package org.apache.spark.sql.hudi.command.payload
import org.apache.avro.generic.IndexedRecord
import org.apache.avro.Schema
import org.apache.hudi.AvroConversionUtils
import org.apache.spark.sql.avro.HoodieAvroDeserializer
import org.apache.avro.generic.IndexedRecord
import org.apache.hudi.{AvroConversionUtils, SparkAdapterSupport}
import org.apache.spark.sql.catalyst.InternalRow
/**
* A sql typed record which will convert the avro field to sql typed value.
*/
class SqlTypedRecord(val record: IndexedRecord) extends IndexedRecord {
class SqlTypedRecord(val record: IndexedRecord) extends IndexedRecord with SparkAdapterSupport {
private lazy val sqlType = AvroConversionUtils.convertAvroSchemaToStructType(getSchema)
private lazy val avroDeserializer = HoodieAvroDeserializer(record.getSchema, sqlType)
private lazy val sqlRow = avroDeserializer.deserializeData(record).asInstanceOf[InternalRow]
private lazy val avroDeserializer = sparkAdapter.createAvroDeserializer(record.getSchema, sqlType)
private lazy val sqlRow = avroDeserializer.deserialize(record).get.asInstanceOf[InternalRow]
override def put(i: Int, v: Any): Unit = {
record.put(i, v)

View File

@@ -443,13 +443,18 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Hoodie
val partitionColumns = tableConfig.getPartitionFieldProp.split(",").map(_.toLowerCase)
val partitionSchema = StructType(tableSchema.filter(f => partitionColumns.contains(f.name)))
// NOTE: Here we fallback to "" to make sure that null value is not overridden with
// default value ("ts")
// TODO(HUDI-3456) clean up
val preCombineField = hoodieCatalogTable.preCombineKey.getOrElse("")
// Enable the hive sync by default if spark have enable the hive metastore.
val enableHive = isEnableHive(sparkSession)
withSparkConf(sparkSession, hoodieCatalogTable.catalogProperties) {
Map(
"path" -> path,
RECORDKEY_FIELD.key -> tableConfig.getRecordKeyFieldProp,
PRECOMBINE_FIELD.key -> hoodieCatalogTable.preCombineKey.getOrElse(""),
PRECOMBINE_FIELD.key -> preCombineField,
TBL_NAME.key -> hoodieCatalogTable.tableName,
PARTITIONPATH_FIELD.key -> tableConfig.getPartitionFieldProp,
PAYLOAD_CLASS_NAME.key -> classOf[ExpressionPayload].getCanonicalName,
@@ -470,6 +475,7 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Hoodie
HoodieWriteConfig.DELETE_PARALLELISM_VALUE.key -> "200",
SqlKeyGenerator.PARTITION_SCHEMA -> partitionSchema.toDDL
)
.filter { case (_, v) => v != null }
}
}
}

View File

@@ -0,0 +1,178 @@
/*
* 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.avro.generic.GenericRecord;
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.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

@@ -0,0 +1,385 @@
/*
* 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.avro.generic.GenericRecord;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieKey;
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

@@ -0,0 +1,86 @@
/*
* 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.avro.generic.GenericRecord;
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.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

@@ -0,0 +1,135 @@
/*
* 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.avro.generic.GenericRecord;
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.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

@@ -0,0 +1,162 @@
/*
* 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.avro.generic.GenericRecord;
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.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

@@ -0,0 +1,449 @@
/*
* 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.avro.Conversions;
import org.apache.avro.LogicalTypes;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericFixed;
import org.apache.avro.generic.GenericRecord;
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.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 scala.Function1;
import scala.Tuple2;
import java.io.IOException;
import java.math.BigDecimal;
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 Row genericRecordToRow(GenericRecord baseRecord) {
Function1<GenericRecord, Row> convertor = AvroConversionUtils.createConverterToRow(baseRecord.getSchema(), structType);
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 partitionPathField, String timestampType, String dateFormat, String timezone, String scalarType) {
TypedProperties properties = new TypedProperties(this.properties);
properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), partitionPathField);
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 TypedProperties getBaseKeyConfig(String partitionPathField,
String timestampType,
String inputFormatList,
String inputFormatDelimiterRegex,
String inputTimezone,
String outputFormat,
String outputTimezone) {
TypedProperties properties = new TypedProperties(this.properties);
properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), partitionPathField);
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("createTime", "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
BigDecimal decimal = new BigDecimal("1578283932000.0001");
Conversions.DecimalConversion conversion = new Conversions.DecimalConversion();
Tuple2<Object, Schema> resolvedNullableSchema = AvroConversionUtils.resolveAvroTypeNullability(schema.getField("createTimeDecimal").schema());
GenericFixed avroDecimal = conversion.toFixed(decimal, resolvedNullableSchema._2, LogicalTypes.decimal(20, 4));
baseRecord.put("createTimeDecimal", avroDecimal);
properties = getBaseKeyConfig("createTimeDecimal", "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());
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020-01-06 12", keyGen.getPartitionPath(baseRow));
// timezone is GMT
properties = getBaseKeyConfig("createTime", "EPOCHMILLISECONDS", "yyyy-MM-dd hh", "GMT", null);
keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk2 = keyGen.getKey(baseRecord);
assertEquals("2020-01-06 04", hk2.getPartitionPath());
assertEquals("2020-01-06 04", keyGen.getPartitionPath(baseRow));
// timestamp is DATE_STRING, timezone is GMT+8:00
baseRecord.put("createTimeString", "2020-01-06 12:12:12");
properties = getBaseKeyConfig("createTimeString", "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());
baseRow = genericRecordToRow(baseRecord);
assertEquals("2020-01-06 12", keyGen.getPartitionPath(baseRow));
// timezone is GMT
properties = getBaseKeyConfig("createTimeString", "DATE_STRING", "yyyy-MM-dd hh", "GMT", null);
keyGen = new TimestampBasedKeyGenerator(properties);
HoodieKey hk4 = keyGen.getKey(baseRecord);
assertEquals("2020-01-06 12", hk4.getPartitionPath());
assertEquals("2020-01-06 12", keyGen.getPartitionPath(baseRow));
// timezone is GMT+8:00, createTime is null
baseRecord.put("createTime", null);
properties = getBaseKeyConfig("createTime", "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());
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("createTimeString", null);
properties = getBaseKeyConfig("createTime", "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());
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("createTime", "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("createTime", "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", 18736L);
properties = getBaseKeyConfig("createTime", "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("createTime", "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("createTime", "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("createTimeString", "2020-04-01T13:01:33.428Z");
properties = this.getBaseKeyConfig(
"createTimeString",
"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("createTimeString", "2020-04-01T13:01:33.428Z");
properties = this.getBaseKeyConfig(
"createTimeString",
"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("createTimeString", "2020-04-01T13:01:33.428Z");
properties = this.getBaseKeyConfig(
"createTimeString",
"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("createTimeString", "2020-04-01T13:01:33Z");
properties = this.getBaseKeyConfig(
"createTimeString",
"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("createTimeString", "2020-04-01T13:01:33-05:00");
properties = this.getBaseKeyConfig(
"createTimeString",
"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("createTimeString", "2020-04-01T13:01:33.123-05:00");
properties = this.getBaseKeyConfig(
"createTimeString",
"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("createTimeString", "2020-04-01T13:01:33.123Z");
properties = this.getBaseKeyConfig(
"createTimeString",
"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("createTimeString", "2020-04-01 13:01:33.123-05:00");
properties = this.getBaseKeyConfig(
"createTimeString",
"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("createTimeString", "20200401");
properties = this.getBaseKeyConfig(
"createTimeString",
"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

@@ -0,0 +1,101 @@
/*
* 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

@@ -0,0 +1,82 @@
/*
* 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,13 +18,13 @@
package org.apache.hudi
import java.time.LocalDate
import org.apache.avro.Schema
import org.apache.avro.generic.GenericData
import org.apache.spark.sql.catalyst.expressions.GenericRow
import org.scalatest.{FunSuite, Matchers}
import java.time.LocalDate
class TestAvroConversionHelper extends FunSuite with Matchers {
val dateSchema = s"""
@@ -42,7 +42,7 @@ class TestAvroConversionHelper extends FunSuite with Matchers {
test("Logical type: date") {
val schema = new Schema.Parser().parse(dateSchema)
val convertor = AvroConversionHelper.createConverterToRow(schema, AvroConversionUtils.convertAvroSchemaToStructType(schema))
val convertor = AvroConversionUtils.createConverterToRow(schema, AvroConversionUtils.convertAvroSchemaToStructType(schema))
val dateOutputData = dateInputData.map(x => {
val record = new GenericData.Record(schema) {{ put("date", x) }}

View File

@@ -243,7 +243,7 @@ class TestDataSourceDefaults {
val partitionPathProp: String = props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD.key)
val STRUCT_NAME: String = "hoodieRowTopLevelField"
val NAMESPACE: String = "hoodieRow"
var converterFn: Function1[Any, Any] = _
var converterFn: Function1[Row, GenericRecord] = _
override def getKey(record: GenericRecord): HoodieKey = {
new HoodieKey(HoodieAvroUtils.getNestedFieldValAsString(record, recordKeyProp, true, false),
@@ -251,13 +251,13 @@ class TestDataSourceDefaults {
}
override def getRecordKey(row: Row): String = {
if (null == converterFn) converterFn = AvroConversionHelper.createConverterToAvro(row.schema, STRUCT_NAME, NAMESPACE)
if (null == converterFn) converterFn = AvroConversionUtils.createConverterToAvro(row.schema, STRUCT_NAME, NAMESPACE)
val genericRecord = converterFn.apply(row).asInstanceOf[GenericRecord]
getKey(genericRecord).getRecordKey
}
override def getPartitionPath(row: Row): String = {
if (null == converterFn) converterFn = AvroConversionHelper.createConverterToAvro(row.schema, STRUCT_NAME, NAMESPACE)
if (null == converterFn) converterFn = AvroConversionUtils.createConverterToAvro(row.schema, STRUCT_NAME, NAMESPACE)
val genericRecord = converterFn.apply(row).asInstanceOf[GenericRecord]
getKey(genericRecord).getPartitionPath
}

View File

@@ -32,13 +32,13 @@ import org.apache.hudi.functional.TestBootstrap
import org.apache.hudi.hive.HiveSyncConfig
import org.apache.hudi.keygen.{ComplexKeyGenerator, NonpartitionedKeyGenerator, SimpleKeyGenerator}
import org.apache.hudi.testutils.DataSourceTestUtils
import org.apache.spark.{SparkConf, SparkContext}
import org.apache.spark.api.java.JavaSparkContext
import org.apache.spark.sql._
import org.apache.spark.sql.functions.{expr, lit}
import org.apache.spark.sql.hudi.HoodieSparkSessionExtension
import org.apache.spark.sql.hudi.command.SqlKeyGenerator
import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
import org.apache.spark.{SparkConf, SparkContext}
import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue, fail}
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
import org.junit.jupiter.params.ParameterizedTest
@@ -827,33 +827,32 @@ class TestHoodieSparkSqlWriter {
/**
* Test case for non partition table with metatable support.
*/
@Test
def testNonPartitionTableWithMetatableSupport(): Unit = {
List(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL).foreach { tableType =>
val options = Map(DataSourceWriteOptions.TABLE_TYPE.key -> tableType,
DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "col3",
DataSourceWriteOptions.RECORDKEY_FIELD.key -> "keyid",
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "",
DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key -> "org.apache.hudi.keygen.NonpartitionedKeyGenerator",
HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
"hoodie.insert.shuffle.parallelism" -> "1",
"hoodie.metadata.enable" -> "true")
val df = spark.range(0, 10).toDF("keyid")
.withColumn("col3", expr("keyid"))
.withColumn("age", expr("keyid + 1000"))
df.write.format("hudi")
.options(options.updated(DataSourceWriteOptions.OPERATION.key, "insert"))
.mode(SaveMode.Overwrite).save(tempBasePath)
// upsert same record again
val df_update = spark.range(0, 10).toDF("keyid")
.withColumn("col3", expr("keyid"))
.withColumn("age", expr("keyid + 2000"))
df_update.write.format("hudi")
.options(options.updated(DataSourceWriteOptions.OPERATION.key, "upsert"))
.mode(SaveMode.Append).save(tempBasePath)
assert(spark.read.format("hudi").load(tempBasePath).count() == 10)
assert(spark.read.format("hudi").load(tempBasePath).where("age >= 2000").count() == 10)
}
@ParameterizedTest
@EnumSource(value = classOf[HoodieTableType])
def testNonPartitionTableWithMetatableSupport(tableType: HoodieTableType): Unit = {
val options = Map(DataSourceWriteOptions.TABLE_TYPE.key -> tableType.name,
DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "col3",
DataSourceWriteOptions.RECORDKEY_FIELD.key -> "keyid",
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "",
DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key -> "org.apache.hudi.keygen.NonpartitionedKeyGenerator",
HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
"hoodie.insert.shuffle.parallelism" -> "1",
"hoodie.metadata.enable" -> "true")
val df = spark.range(0, 10).toDF("keyid")
.withColumn("col3", expr("keyid"))
.withColumn("age", expr("keyid + 1000"))
df.write.format("hudi")
.options(options.updated(DataSourceWriteOptions.OPERATION.key, "insert"))
.mode(SaveMode.Overwrite).save(tempBasePath)
// upsert same record again
val df_update = spark.range(0, 10).toDF("keyid")
.withColumn("col3", expr("keyid"))
.withColumn("age", expr("keyid + 2000"))
df_update.write.format("hudi")
.options(options.updated(DataSourceWriteOptions.OPERATION.key, "upsert"))
.mode(SaveMode.Append).save(tempBasePath)
assert(spark.read.format("hudi").load(tempBasePath).count() == 10)
assert(spark.read.format("hudi").load(tempBasePath).where("age >= 2000").count() == 10)
}
/**

View File

@@ -20,22 +20,18 @@ package org.apache.hudi
import org.apache.avro.Schema
import org.apache.avro.generic.GenericRecord
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.apache.hudi.exception.SchemaCompatibilityException
import org.apache.hudi.testutils.DataSourceTestUtils
import org.apache.spark.sql.types.{StructType, TimestampType}
import org.apache.spark.sql.{Row, SparkSession}
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Test
import org.junit.jupiter.api.io.TempDir
import java.io.File
import java.nio.file.Paths
import scala.collection.JavaConverters
class TestHoodieSparkUtils {
@@ -232,8 +228,9 @@ class TestHoodieSparkUtils {
fail("createRdd should fail, because records don't have a column which is not nullable in the passed in schema")
} catch {
case e: Exception =>
e.getCause.asInstanceOf[NullPointerException]
assertTrue(e.getMessage.contains("null of string in field new_nested_col of"))
val cause = e.getCause
assertTrue(cause.isInstanceOf[SchemaCompatibilityException])
assertTrue(e.getMessage.contains("Unable to validate the rewritten record {\"innerKey\": \"innerKey1_2\", \"innerValue\": 2} against schema"))
}
spark.stop()
}

View File

@@ -17,11 +17,11 @@
package org.apache.hudi.functional
import org.apache.avro.generic.GenericRecord
import org.apache.hadoop.fs.Path
import org.apache.hudi.DataSourceWriteOptions._
import org.apache.hudi.common.config.HoodieMetadataConfig
import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, HoodieTableType}
import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, HoodieRecord, HoodieRecordPayload, HoodieTableType}
import org.apache.hudi.common.table.HoodieTableMetaClient
import org.apache.hudi.common.testutils.HoodieTestDataGenerator
import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
@@ -30,18 +30,18 @@ import org.apache.hudi.index.HoodieIndex.IndexType
import org.apache.hudi.keygen.NonpartitionedKeyGenerator
import org.apache.hudi.keygen.constant.KeyGeneratorOptions.Config
import org.apache.hudi.testutils.{DataSourceTestUtils, HoodieClientTestBase}
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers, HoodieSparkUtils}
import org.apache.hudi.{AvroConversionUtils, DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers, HoodieSparkUtils}
import org.apache.log4j.LogManager
import org.apache.spark.rdd.RDD
import org.apache.spark.sql._
import org.apache.spark.sql.functions._
import org.apache.spark.sql.types.BooleanType
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.CsvSource
import java.util
import scala.collection.JavaConversions._
import scala.collection.JavaConverters._
@@ -349,11 +349,15 @@ class TestMORDataSource extends HoodieClientTestBase {
// First Operation:
// Producing parquet files to three default partitions.
// SNAPSHOT view on MOR table with parquet files only.
// Overriding the partition-path field
val opts = commonOpts + (DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition_path")
val hoodieRecords1 = dataGen.generateInserts("001", 100)
val records1 = recordsToStrings(hoodieRecords1).toList
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
val inputDF1 = toDataset(hoodieRecords1)
inputDF1.write.format("org.apache.hudi")
.options(commonOpts)
.options(opts)
.option("hoodie.compact.inline", "false") // else fails due to compaction & deltacommit instant times being same
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
.option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
@@ -376,11 +380,10 @@ class TestMORDataSource extends HoodieClientTestBase {
// Second Operation:
// Upsert 50 update records
// Snopshot view should read 100 records
val records2 = recordsToStrings(dataGen.generateUniqueUpdates("002", 50))
.toList
val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2))
val records2 = dataGen.generateUniqueUpdates("002", 50)
val inputDF2 = toDataset(records2)
inputDF2.write.format("org.apache.hudi")
.options(commonOpts)
.options(opts)
.mode(SaveMode.Append)
.save(basePath)
val hudiSnapshotDF2 = spark.read.format("org.apache.hudi")
@@ -424,17 +427,31 @@ class TestMORDataSource extends HoodieClientTestBase {
verifyShow(hudiIncDF2)
verifyShow(hudiIncDF1Skipmerge)
val record3 = recordsToStrings(dataGen.generateUpdatesWithTS("003", hoodieRecords1, -1))
spark.read.json(spark.sparkContext.parallelize(record3, 2))
.write.format("org.apache.hudi").options(commonOpts)
val record3 = dataGen.generateUpdatesWithTS("003", hoodieRecords1, -1)
val inputDF3 = toDataset(record3)
inputDF3.write.format("org.apache.hudi").options(opts)
.mode(SaveMode.Append).save(basePath)
val hudiSnapshotDF3 = spark.read.format("org.apache.hudi")
.option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
.load(basePath + "/*/*/*/*")
verifyShow(hudiSnapshotDF3);
assertEquals(100, hudiSnapshotDF3.count())
assertEquals(0, hudiSnapshotDF3.filter("rider = 'rider-003'").count())
}
private def toDataset(records: util.List[HoodieRecord[_]]) = {
val avroRecords = records.map(_.getData
.asInstanceOf[HoodieRecordPayload[_]]
.getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA)
.get
.asInstanceOf[GenericRecord])
val rdd: RDD[GenericRecord] = spark.sparkContext.parallelize(avroRecords, 2)
AvroConversionUtils.createDataFrame(rdd, HoodieTestDataGenerator.AVRO_SCHEMA.toString, spark)
}
@Test
def testVectorizedReader() {
spark.conf.set("spark.sql.parquet.enableVectorizedReader", true)
@@ -553,15 +570,10 @@ class TestMORDataSource extends HoodieClientTestBase {
.orderBy(desc("_hoodie_commit_time"))
.head()
assertEquals(sampleRow.getDouble(0), sampleRow.get(0))
assertEquals(sampleRow.getLong(1), sampleRow.get(1))
assertEquals(sampleRow.getDate(1), sampleRow.get(1))
assertEquals(sampleRow.getString(2), sampleRow.get(2))
assertEquals(sampleRow.getSeq(3), sampleRow.get(3))
if (HoodieSparkUtils.gteqSpark3_2) {
// Since Spark3.2, the `nation` column is parsed as String, not Struct.
assertEquals(sampleRow.getString(4), sampleRow.get(4))
} else {
assertEquals(sampleRow.getStruct(4), sampleRow.get(4))
}
assertEquals(sampleRow.getAs[Array[Byte]](4), sampleRow.get(4))
}
def verifyShow(df: DataFrame): Unit = {

View File

@@ -0,0 +1,98 @@
/*
* 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.functional
import org.apache.hudi.DataSourceWriteOptions
import org.apache.hudi.common.config.HoodieMetadataConfig
import org.apache.hudi.common.testutils.HoodieTestDataGenerator
import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.testutils.SparkClientFunctionalTestHarness
import org.apache.spark.sql.SaveMode
import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.{Tag, Test}
import scala.collection.JavaConverters._
@Tag("functional")
class TestMetadataTableWithSparkDataSource extends SparkClientFunctionalTestHarness {
val hudi = "org.apache.hudi"
var commonOpts = Map(
"hoodie.insert.shuffle.parallelism" -> "4",
"hoodie.upsert.shuffle.parallelism" -> "4",
"hoodie.bulkinsert.shuffle.parallelism" -> "2",
"hoodie.delete.shuffle.parallelism" -> "1",
DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
)
@Test
def testReadability(): Unit = {
val dataGen = new HoodieTestDataGenerator()
val opts: Map[String, String] = commonOpts ++ Map(
HoodieMetadataConfig.ENABLE.key -> "true",
HoodieMetadataConfig.COMPACT_NUM_DELTA_COMMITS.key -> "1"
)
// Insert records
val newRecords = dataGen.generateInserts("001", 100)
val newRecordsDF = parseRecords(recordsToStrings(newRecords).asScala)
newRecordsDF.write.format(hudi)
.options(opts)
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
.mode(SaveMode.Append)
.save(basePath)
// Update records
val updatedRecords = dataGen.generateUpdates("002", newRecords)
val updatedRecordsDF = parseRecords(recordsToStrings(updatedRecords).asScala)
updatedRecordsDF.write.format(hudi)
.options(opts)
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
.mode(SaveMode.Append)
.save(basePath)
val metadataDF = spark.read.format(hudi).load(s"$basePath/.hoodie/metadata")
// Smoke test
metadataDF.show()
// Query w/ 0 requested columns should be working fine
assertEquals(4, metadataDF.count())
val expectedKeys = Seq("2015/03/16", "2015/03/17", "2016/03/15", "__all_partitions__")
val keys = metadataDF.select("key")
.collect()
.map(_.getString(0))
.toSeq
.sorted
assertEquals(expectedKeys, keys)
}
private def parseRecords(records: Seq[String]) = {
spark.read.json(spark.sparkContext.parallelize(records, 2))
}
}

View File

@@ -199,6 +199,15 @@
<groupId>org.apache.spark</groupId>
<artifactId>spark-sql_${scala.binary.version}</artifactId>
<version>${spark2.version}</version>
<scope>provided</scope>
<optional>true</optional>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-avro_${scala.binary.version}</artifactId>
<version>${spark2.version}</version>
<scope>provided</scope>
<optional>true</optional>
</dependency>

View File

@@ -17,9 +17,10 @@
package org.apache.spark.sql.adapter
import org.apache.avro.Schema
import org.apache.hudi.Spark2RowSerDe
import org.apache.hudi.client.utils.SparkRowSerDe
import org.apache.spark.sql.avro.{HoodieAvroDeserializerTrait, HoodieAvroSerializerTrait, Spark2HoodieAvroDeserializer, HoodieAvroSerializer}
import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.catalyst.expressions.{Expression, Like}
@@ -31,6 +32,7 @@ import org.apache.spark.sql.execution.datasources.{FilePartition, PartitionedFil
import org.apache.spark.sql.hudi.SparkAdapter
import org.apache.spark.sql.hudi.parser.HoodieSpark2ExtendedSqlParser
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.DataType
import org.apache.spark.sql.{Row, SparkSession}
import scala.collection.mutable.ArrayBuffer
@@ -40,6 +42,12 @@ import scala.collection.mutable.ArrayBuffer
*/
class Spark2Adapter extends SparkAdapter {
def createAvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: Boolean): HoodieAvroSerializerTrait =
new HoodieAvroSerializer(rootCatalystType, rootAvroType, nullable)
def createAvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType): HoodieAvroDeserializerTrait =
new Spark2HoodieAvroDeserializer(rootAvroType, rootCatalystType)
override def createSparkRowSerDe(encoder: ExpressionEncoder[Row]): SparkRowSerDe = {
new Spark2RowSerDe(encoder)
}

View File

@@ -0,0 +1,398 @@
/*
* 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
import org.apache.avro.Conversions.DecimalConversion
import org.apache.avro.LogicalTypes.{TimestampMicros, TimestampMillis}
import org.apache.avro.Schema.Type._
import org.apache.avro.generic._
import org.apache.avro.util.Utf8
import org.apache.avro.{LogicalTypes, Schema, SchemaBuilder}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{SpecificInternalRow, UnsafeArrayData}
import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, DateTimeUtils, GenericArrayData}
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String
import java.math.BigDecimal
import java.nio.ByteBuffer
import scala.collection.JavaConverters._
import scala.collection.mutable.ArrayBuffer
/**
* A deserializer to deserialize data in avro format to data in catalyst format.
*
* NOTE: This is a version of {@code AvroDeserializer} impl from Spark 2.4.4 w/ the fix for SPARK-30267
* applied on top of it
*/
class PatchedAvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType) {
private lazy val decimalConversions = new DecimalConversion()
private val converter: Any => Any = rootCatalystType match {
// A shortcut for empty schema.
case st: StructType if st.isEmpty =>
(data: Any) => InternalRow.empty
case st: StructType =>
val resultRow = new SpecificInternalRow(st.map(_.dataType))
val fieldUpdater = new RowUpdater(resultRow)
val writer = getRecordWriter(rootAvroType, st, Nil)
(data: Any) => {
val record = data.asInstanceOf[GenericRecord]
writer(fieldUpdater, record)
resultRow
}
case _ =>
val tmpRow = new SpecificInternalRow(Seq(rootCatalystType))
val fieldUpdater = new RowUpdater(tmpRow)
val writer = newWriter(rootAvroType, rootCatalystType, Nil)
(data: Any) => {
writer(fieldUpdater, 0, data)
tmpRow.get(0, rootCatalystType)
}
}
def deserialize(data: Any): Any = converter(data)
/**
* Creates a writer to write avro values to Catalyst values at the given ordinal with the given
* updater.
*/
private def newWriter(
avroType: Schema,
catalystType: DataType,
path: List[String]): (CatalystDataUpdater, Int, Any) => Unit =
(avroType.getType, catalystType) match {
case (NULL, NullType) => (updater, ordinal, _) =>
updater.setNullAt(ordinal)
// TODO: we can avoid boxing if future version of avro provide primitive accessors.
case (BOOLEAN, BooleanType) => (updater, ordinal, value) =>
updater.setBoolean(ordinal, value.asInstanceOf[Boolean])
case (INT, IntegerType) => (updater, ordinal, value) =>
updater.setInt(ordinal, value.asInstanceOf[Int])
case (INT, DateType) => (updater, ordinal, value) =>
updater.setInt(ordinal, value.asInstanceOf[Int])
case (LONG, LongType) => (updater, ordinal, value) =>
updater.setLong(ordinal, value.asInstanceOf[Long])
case (LONG, TimestampType) => avroType.getLogicalType match {
case _: TimestampMillis => (updater, ordinal, value) =>
updater.setLong(ordinal, value.asInstanceOf[Long] * 1000)
case _: TimestampMicros => (updater, ordinal, value) =>
updater.setLong(ordinal, value.asInstanceOf[Long])
case null => (updater, ordinal, value) =>
// For backward compatibility, if the Avro type is Long and it is not logical type,
// the value is processed as timestamp type with millisecond precision.
updater.setLong(ordinal, value.asInstanceOf[Long] * 1000)
case other => throw new IncompatibleSchemaException(
s"Cannot convert Avro logical type ${other} to Catalyst Timestamp type.")
}
// Before we upgrade Avro to 1.8 for logical type support, spark-avro converts Long to Date.
// For backward compatibility, we still keep this conversion.
case (LONG, DateType) => (updater, ordinal, value) =>
updater.setInt(ordinal, (value.asInstanceOf[Long] / DateTimeUtils.MILLIS_PER_DAY).toInt)
case (FLOAT, FloatType) => (updater, ordinal, value) =>
updater.setFloat(ordinal, value.asInstanceOf[Float])
case (DOUBLE, DoubleType) => (updater, ordinal, value) =>
updater.setDouble(ordinal, value.asInstanceOf[Double])
case (STRING, StringType) => (updater, ordinal, value) =>
val str = value match {
case s: String => UTF8String.fromString(s)
case s: Utf8 =>
val bytes = new Array[Byte](s.getByteLength)
System.arraycopy(s.getBytes, 0, bytes, 0, s.getByteLength)
UTF8String.fromBytes(bytes)
}
updater.set(ordinal, str)
case (ENUM, StringType) => (updater, ordinal, value) =>
updater.set(ordinal, UTF8String.fromString(value.toString))
case (FIXED, BinaryType) => (updater, ordinal, value) =>
updater.set(ordinal, value.asInstanceOf[GenericFixed].bytes().clone())
case (BYTES, BinaryType) => (updater, ordinal, value) =>
val bytes = value match {
case b: ByteBuffer =>
val bytes = new Array[Byte](b.remaining)
b.get(bytes)
bytes
case b: Array[Byte] => b
case other => throw new RuntimeException(s"$other is not a valid avro binary.")
}
updater.set(ordinal, bytes)
case (FIXED, d: DecimalType) => (updater, ordinal, value) =>
val bigDecimal = decimalConversions.fromFixed(value.asInstanceOf[GenericFixed], avroType,
LogicalTypes.decimal(d.precision, d.scale))
val decimal = createDecimal(bigDecimal, d.precision, d.scale)
updater.setDecimal(ordinal, decimal)
case (BYTES, d: DecimalType) => (updater, ordinal, value) =>
val bigDecimal = decimalConversions.fromBytes(value.asInstanceOf[ByteBuffer], avroType,
LogicalTypes.decimal(d.precision, d.scale))
val decimal = createDecimal(bigDecimal, d.precision, d.scale)
updater.setDecimal(ordinal, decimal)
case (RECORD, st: StructType) =>
val writeRecord = getRecordWriter(avroType, st, path)
(updater, ordinal, value) =>
val row = new SpecificInternalRow(st)
writeRecord(new RowUpdater(row), value.asInstanceOf[GenericRecord])
updater.set(ordinal, row)
case (ARRAY, ArrayType(elementType, containsNull)) =>
val elementWriter = newWriter(avroType.getElementType, elementType, path)
val elementPath = path :+ "element"
(updater, ordinal, value) =>
val collection = value.asInstanceOf[java.util.Collection[Any]]
val len = collection.size()
val result = createArrayData(elementType, len)
val elementUpdater = new ArrayDataUpdater(result)
var i = 0
val iter = collection.iterator()
while (iter.hasNext) {
val element = iter.next()
if (element == null) {
if (!containsNull) {
throw new RuntimeException(
s"Array value at path '${elementPath.mkString(".")}' is not allowed to be null")
} else {
elementUpdater.setNullAt(i)
}
} else {
elementWriter(elementUpdater, i, element)
}
i += 1
}
updater.set(ordinal, result)
case (MAP, MapType(keyType, valueType, valueContainsNull)) if keyType == StringType =>
val keyWriter = newWriter(SchemaBuilder.builder().stringType(), StringType, path)
val valueWriter = newWriter(avroType.getValueType, valueType, path)
(updater, ordinal, value) =>
val map = value.asInstanceOf[java.util.Map[AnyRef, AnyRef]]
val keyArray = createArrayData(keyType, map.size())
val keyUpdater = new ArrayDataUpdater(keyArray)
val valueArray = createArrayData(valueType, map.size())
val valueUpdater = new ArrayDataUpdater(valueArray)
val iter = map.entrySet().iterator()
var i = 0
while (iter.hasNext) {
val entry = iter.next()
assert(entry.getKey != null)
keyWriter(keyUpdater, i, entry.getKey)
if (entry.getValue == null) {
if (!valueContainsNull) {
throw new RuntimeException(s"Map value at path ${path.mkString(".")} is not " +
"allowed to be null")
} else {
valueUpdater.setNullAt(i)
}
} else {
valueWriter(valueUpdater, i, entry.getValue)
}
i += 1
}
updater.set(ordinal, new ArrayBasedMapData(keyArray, valueArray))
case (UNION, _) =>
val allTypes = avroType.getTypes.asScala
val nonNullTypes = allTypes.filter(_.getType != NULL)
val nonNullAvroType = Schema.createUnion(nonNullTypes.asJava)
if (nonNullTypes.nonEmpty) {
if (nonNullTypes.length == 1) {
newWriter(nonNullTypes.head, catalystType, path)
} else {
nonNullTypes.map(_.getType) match {
case Seq(a, b) if Set(a, b) == Set(INT, LONG) && catalystType == LongType =>
(updater, ordinal, value) => value match {
case null => updater.setNullAt(ordinal)
case l: java.lang.Long => updater.setLong(ordinal, l)
case i: java.lang.Integer => updater.setLong(ordinal, i.longValue())
}
case Seq(a, b) if Set(a, b) == Set(FLOAT, DOUBLE) && catalystType == DoubleType =>
(updater, ordinal, value) => value match {
case null => updater.setNullAt(ordinal)
case d: java.lang.Double => updater.setDouble(ordinal, d)
case f: java.lang.Float => updater.setDouble(ordinal, f.doubleValue())
}
case _ =>
catalystType match {
case st: StructType if st.length == nonNullTypes.size =>
val fieldWriters = nonNullTypes.zip(st.fields).map {
case (schema, field) => newWriter(schema, field.dataType, path :+ field.name)
}.toArray
(updater, ordinal, value) => {
val row = new SpecificInternalRow(st)
val fieldUpdater = new RowUpdater(row)
val i = GenericData.get().resolveUnion(nonNullAvroType, value)
fieldWriters(i)(fieldUpdater, i, value)
updater.set(ordinal, row)
}
case _ =>
throw new IncompatibleSchemaException(
s"Cannot convert Avro to catalyst because schema at path " +
s"${path.mkString(".")} is not compatible " +
s"(avroType = $avroType, sqlType = $catalystType).\n" +
s"Source Avro schema: $rootAvroType.\n" +
s"Target Catalyst type: $rootCatalystType")
}
}
}
} else {
(updater, ordinal, value) => updater.setNullAt(ordinal)
}
case _ =>
throw new IncompatibleSchemaException(
s"Cannot convert Avro to catalyst because schema at path ${path.mkString(".")} " +
s"is not compatible (avroType = $avroType, sqlType = $catalystType).\n" +
s"Source Avro schema: $rootAvroType.\n" +
s"Target Catalyst type: $rootCatalystType")
}
// TODO: move the following method in Decimal object on creating Decimal from BigDecimal?
private def createDecimal(decimal: 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)
}
}
private def getRecordWriter(
avroType: Schema,
sqlType: StructType,
path: List[String]): (CatalystDataUpdater, GenericRecord) => Unit = {
val validFieldIndexes = ArrayBuffer.empty[Int]
val fieldWriters = ArrayBuffer.empty[(CatalystDataUpdater, Any) => Unit]
val length = sqlType.length
var i = 0
while (i < length) {
val sqlField = sqlType.fields(i)
val avroField = avroType.getField(sqlField.name)
if (avroField != null) {
validFieldIndexes += avroField.pos()
val baseWriter = newWriter(avroField.schema(), sqlField.dataType, path :+ sqlField.name)
val ordinal = i
val fieldWriter = (fieldUpdater: CatalystDataUpdater, value: Any) => {
if (value == null) {
fieldUpdater.setNullAt(ordinal)
} else {
baseWriter(fieldUpdater, ordinal, value)
}
}
fieldWriters += fieldWriter
} else if (!sqlField.nullable) {
throw new IncompatibleSchemaException(
s"""
|Cannot find non-nullable field ${path.mkString(".")}.${sqlField.name} in Avro schema.
|Source Avro schema: $rootAvroType.
|Target Catalyst type: $rootCatalystType.
""".stripMargin)
}
i += 1
}
(fieldUpdater, record) => {
var i = 0
while (i < validFieldIndexes.length) {
fieldWriters(i)(fieldUpdater, record.get(validFieldIndexes(i)))
i += 1
}
}
}
private def createArrayData(elementType: DataType, length: Int): ArrayData = elementType match {
case BooleanType => UnsafeArrayData.fromPrimitiveArray(new Array[Boolean](length))
case ByteType => UnsafeArrayData.fromPrimitiveArray(new Array[Byte](length))
case ShortType => UnsafeArrayData.fromPrimitiveArray(new Array[Short](length))
case IntegerType => UnsafeArrayData.fromPrimitiveArray(new Array[Int](length))
case LongType => UnsafeArrayData.fromPrimitiveArray(new Array[Long](length))
case FloatType => UnsafeArrayData.fromPrimitiveArray(new Array[Float](length))
case DoubleType => UnsafeArrayData.fromPrimitiveArray(new Array[Double](length))
case _ => new GenericArrayData(new Array[Any](length))
}
/**
* A base interface for updating values inside catalyst data structure like `InternalRow` and
* `ArrayData`.
*/
sealed trait CatalystDataUpdater {
def set(ordinal: Int, value: Any): Unit
def setNullAt(ordinal: Int): Unit = set(ordinal, null)
def setBoolean(ordinal: Int, value: Boolean): Unit = set(ordinal, value)
def setByte(ordinal: Int, value: Byte): Unit = set(ordinal, value)
def setShort(ordinal: Int, value: Short): Unit = set(ordinal, value)
def setInt(ordinal: Int, value: Int): Unit = set(ordinal, value)
def setLong(ordinal: Int, value: Long): Unit = set(ordinal, value)
def setDouble(ordinal: Int, value: Double): Unit = set(ordinal, value)
def setFloat(ordinal: Int, value: Float): Unit = set(ordinal, value)
def setDecimal(ordinal: Int, value: Decimal): Unit = set(ordinal, value)
}
final class RowUpdater(row: InternalRow) extends CatalystDataUpdater {
override def set(ordinal: Int, value: Any): Unit = row.update(ordinal, value)
override def setNullAt(ordinal: Int): Unit = row.setNullAt(ordinal)
override def setBoolean(ordinal: Int, value: Boolean): Unit = row.setBoolean(ordinal, value)
override def setByte(ordinal: Int, value: Byte): Unit = row.setByte(ordinal, value)
override def setShort(ordinal: Int, value: Short): Unit = row.setShort(ordinal, value)
override def setInt(ordinal: Int, value: Int): Unit = row.setInt(ordinal, value)
override def setLong(ordinal: Int, value: Long): Unit = row.setLong(ordinal, value)
override def setDouble(ordinal: Int, value: Double): Unit = row.setDouble(ordinal, value)
override def setFloat(ordinal: Int, value: Float): Unit = row.setFloat(ordinal, value)
override def setDecimal(ordinal: Int, value: Decimal): Unit =
row.setDecimal(ordinal, value, value.precision)
}
final class ArrayDataUpdater(array: ArrayData) extends CatalystDataUpdater {
override def set(ordinal: Int, value: Any): Unit = array.update(ordinal, value)
override def setNullAt(ordinal: Int): Unit = array.setNullAt(ordinal)
override def setBoolean(ordinal: Int, value: Boolean): Unit = array.setBoolean(ordinal, value)
override def setByte(ordinal: Int, value: Byte): Unit = array.setByte(ordinal, value)
override def setShort(ordinal: Int, value: Short): Unit = array.setShort(ordinal, value)
override def setInt(ordinal: Int, value: Int): Unit = array.setInt(ordinal, value)
override def setLong(ordinal: Int, value: Long): Unit = array.setLong(ordinal, value)
override def setDouble(ordinal: Int, value: Double): Unit = array.setDouble(ordinal, value)
override def setFloat(ordinal: Int, value: Float): Unit = array.setFloat(ordinal, value)
override def setDecimal(ordinal: Int, value: Decimal): Unit = array.update(ordinal, value)
}
}

View File

@@ -0,0 +1,33 @@
/*
* 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
import org.apache.avro.Schema
import org.apache.spark.sql.types.DataType
/**
* This is Spark 2 implementation for the [[HoodieAvroDeserializerTrait]] leveraging [[PatchedAvroDeserializer]],
* which is just copied over version of [[AvroDeserializer]] from Spark 2.4.4 w/ SPARK-30267 being back-ported to it
*/
class Spark2HoodieAvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType)
extends HoodieAvroDeserializerTrait {
private val avroDeserializer = new PatchedAvroDeserializer(rootAvroType, rootCatalystType)
def doDeserialize(data: Any): Any = avroDeserializer.deserialize(data)
}

View File

@@ -168,6 +168,15 @@
<groupId>org.apache.spark</groupId>
<artifactId>spark-sql_2.12</artifactId>
<version>${spark3.version}</version>
<scope>provided</scope>
<optional>true</optional>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-avro_2.12</artifactId>
<version>${spark3.version}</version>
<scope>provided</scope>
<optional>true</optional>
</dependency>

View File

@@ -17,10 +17,11 @@
package org.apache.spark.sql.adapter
import org.apache.avro.Schema
import org.apache.hudi.Spark3RowSerDe
import org.apache.hudi.client.utils.SparkRowSerDe
import org.apache.hudi.spark3.internal.ReflectUtil
import org.apache.spark.sql.{Row, SparkSession}
import org.apache.spark.sql.avro.{HoodieAvroDeserializerTrait, HoodieAvroSerializerTrait, Spark3HoodieAvroDeserializer, HoodieAvroSerializer}
import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.catalyst.expressions.{Expression, Like}
@@ -30,18 +31,24 @@ import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoStatement, Join, J
import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier}
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
import org.apache.spark.sql.connector.catalog.Table
import org.apache.spark.sql.execution.datasources._
import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
import org.apache.spark.sql.execution.datasources.{FilePartition, LogicalRelation, PartitionedFile, Spark3ParsePartitionUtil, SparkParsePartitionUtil}
import org.apache.spark.sql.hudi.SparkAdapter
import org.apache.spark.sql.internal.SQLConf
import scala.collection.JavaConverters.mapAsScalaMapConverter
import org.apache.spark.sql.types.DataType
import org.apache.spark.sql.{Row, SparkSession}
/**
* The adapter for spark3.
*/
class Spark3Adapter extends SparkAdapter {
def createAvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: Boolean): HoodieAvroSerializerTrait =
new HoodieAvroSerializer(rootCatalystType, rootAvroType, nullable)
def createAvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType): HoodieAvroDeserializerTrait =
new Spark3HoodieAvroDeserializer(rootAvroType, rootCatalystType)
override def createSparkRowSerDe(encoder: ExpressionEncoder[Row]): SparkRowSerDe = {
new Spark3RowSerDe(encoder)
}

View File

@@ -18,20 +18,15 @@
package org.apache.spark.sql.avro
import org.apache.avro.Schema
import org.apache.hudi.HoodieSparkUtils
import org.apache.spark.sql.types.DataType
/**
* This is to be compatible with the type returned by Spark 3.1
* and other spark versions for AvroDeserializer
*/
case class HoodieAvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType) {
class Spark3HoodieAvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType)
extends HoodieAvroDeserializerTrait {
// SPARK-34404: As of Spark3.2, there is no AvroDeserializer's constructor with Schema and DataType arguments.
// So use the reflection to get AvroDeserializer instance.
private val avroDeserializer = if (HoodieSparkUtils.isSpark3_2) {
// SPARK-34404: As of Spark3.2, there is no AvroDeserializer's constructor with Schema and DataType arguments.
// So use the reflection to get AvroDeserializer instance.
val constructor = classOf[AvroDeserializer].getConstructor(classOf[Schema], classOf[DataType], classOf[String])
constructor.newInstance(rootAvroType, rootCatalystType, "EXCEPTION")
} else {
@@ -39,10 +34,5 @@ case class HoodieAvroDeserializer(rootAvroType: Schema, rootCatalystType: DataTy
constructor.newInstance(rootAvroType, rootCatalystType)
}
def deserializeData(data: Any): Any = {
avroDeserializer.deserialize(data) match {
case Some(r) => r // As of spark 3.1, this will return data wrapped with Option, so we fetch the data.
case o => o // for other spark version, return the data directly.
}
}
def doDeserialize(data: Any): Any = avroDeserializer.deserialize(data)
}

View File

@@ -158,6 +158,7 @@
<groupId>org.apache.spark</groupId>
<artifactId>spark-sql_2.12</artifactId>
<version>${spark3.version}</version>
<scope>provided</scope>
<optional>true</optional>
</dependency>