diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/BaseSparkInternalRowReaderContext.java b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/BaseSparkInternalRowReaderContext.java index af5e1c75b252..528cab69dc4b 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/BaseSparkInternalRowReaderContext.java +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/BaseSparkInternalRowReaderContext.java @@ -37,12 +37,14 @@ import org.apache.spark.sql.HoodieInternalRowUtils; import org.apache.spark.sql.HoodieUnsafeRowUtils; import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.catalyst.expressions.UnsafeProjection; +import org.apache.spark.sql.catalyst.expressions.UnsafeRow; import org.apache.spark.sql.types.StructType; import java.util.Map; import java.util.function.UnaryOperator; +import scala.Function1; + import static org.apache.hudi.common.model.HoodieRecord.RECORD_KEY_METADATA_FIELD; import static org.apache.hudi.common.model.HoodieRecordMerger.DEFAULT_MERGER_STRATEGY_UUID; import static org.apache.spark.sql.HoodieInternalRowUtils.getCachedSchema; @@ -137,8 +139,14 @@ private Object getFieldValueFromInternalRow(InternalRow row, Schema recordSchema } @Override - public UnaryOperator projectRecord(Schema from, Schema to) { - UnsafeProjection projection = HoodieInternalRowUtils.generateUnsafeProjectionAlias(getCachedSchema(from), getCachedSchema(to)); - return projection::apply; + public UnaryOperator projectRecord(Schema from, Schema to, Map renamedColumns) { + Function1 unsafeRowWriter = + HoodieInternalRowUtils.getCachedUnsafeRowWriter(getCachedSchema(from), getCachedSchema(to), renamedColumns); + return row -> (InternalRow) unsafeRowWriter.apply(row); + + } + + protected UnaryOperator getIdentityProjection() { + return row -> row; } } diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala index c3d49ca47a89..5cbca31a6f90 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala @@ -19,48 +19,59 @@ package org.apache.hudi +import org.apache.avro.Schema +import org.apache.avro.generic.IndexedRecord +import org.apache.hadoop.conf.Configuration +import org.apache.hudi.SparkFileFormatInternalRowReaderContext.getAppliedRequiredSchema +import org.apache.hudi.avro.AvroSchemaUtils import org.apache.hudi.common.engine.HoodieReaderContext import org.apache.hudi.common.fs.FSUtils +import org.apache.hudi.common.table.read.HoodiePositionBasedFileGroupRecordBuffer.ROW_INDEX_TEMPORARY_COLUMN_NAME import org.apache.hudi.common.util.ValidationUtils.checkState -import org.apache.hudi.common.util.collection.{ClosableIterator, CloseableMappingIterator} +import org.apache.hudi.common.util.collection.{CachingIterator, ClosableIterator, CloseableMappingIterator} import org.apache.hudi.io.storage.{HoodieSparkFileReaderFactory, HoodieSparkParquetReader} -import org.apache.hudi.storage.{HoodieStorage, StoragePath} +import org.apache.hudi.storage.{HoodieStorage, StorageConfiguration, StoragePath} import org.apache.hudi.util.CloseableInternalRowIterator - -import org.apache.avro.Schema -import org.apache.avro.generic.IndexedRecord import org.apache.spark.sql.HoodieInternalRowUtils import org.apache.spark.sql.avro.HoodieAvroDeserializer import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{JoinedRow, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.execution.datasources.PartitionedFile -import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, SparkParquetReader} +import org.apache.spark.sql.hudi.SparkAdapter +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.types.{LongType, MetadataBuilder, StructField, StructType} import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch} import scala.collection.mutable /** - * Implementation of {@link HoodieReaderContext} to read {@link InternalRow}s with - * {@link ParquetFileFormat} on Spark. + * Implementation of [[HoodieReaderContext]] to read [[InternalRow]]s with + * [[ParquetFileFormat]] on Spark. * * This uses Spark parquet reader to read parquet data files or parquet log blocks. * - * @param readermaps our intention is to build the reader inside of getFileRecordIterator, but since it is called from - * the executor, we will need to port a bunch of the code from ParquetFileFormat for each spark version - * for now, we pass in a map of the different readers we expect to create + * @param parquetFileReader A reader that transforms a [[PartitionedFile]] to an iterator of + * [[InternalRow]]. This is required for reading the base file and + * not required for reading a file group with only log files. + * @param recordKeyColumn column name for the recordkey + * @param filters spark filters that might be pushed down into the reader */ -class SparkFileFormatInternalRowReaderContext(readerMaps: mutable.Map[Long, PartitionedFile => Iterator[InternalRow]]) extends BaseSparkInternalRowReaderContext { - lazy val sparkAdapter = SparkAdapterSupport.sparkAdapter - val deserializerMap: mutable.Map[Schema, HoodieAvroDeserializer] = mutable.Map() - - override def getFileRecordIterator(filePath: StoragePath, start: Long, length: Long, dataSchema: Schema, requiredSchema: Schema, storage: HoodieStorage): ClosableIterator[InternalRow] = { - // partition value is empty because the spark parquet reader will append the partition columns to - // each row if they are given. That is the only usage of the partition values in the reader. - val fileInfo = sparkAdapter.getSparkPartitionedFileUtils - .createPartitionedFile(InternalRow.empty, filePath, start, length) +class SparkFileFormatInternalRowReaderContext(parquetFileReader: SparkParquetReader, + recordKeyColumn: String, + filters: Seq[Filter]) extends BaseSparkInternalRowReaderContext { + lazy val sparkAdapter: SparkAdapter = SparkAdapterSupport.sparkAdapter + lazy val recordKeyFilters: Seq[Filter] = filters.filter(f => f.references.exists(c => c.equalsIgnoreCase(recordKeyColumn))) + private val deserializerMap: mutable.Map[Schema, HoodieAvroDeserializer] = mutable.Map() + + override def getFileRecordIterator(filePath: StoragePath, + start: Long, + length: Long, + dataSchema: Schema, + requiredSchema: Schema, + storage: HoodieStorage): ClosableIterator[InternalRow] = { + val structType: StructType = HoodieInternalRowUtils.getCachedSchema(requiredSchema) if (FSUtils.isLogFile(filePath)) { - val structType: StructType = HoodieInternalRowUtils.getCachedSchema(requiredSchema) val projection: UnsafeProjection = HoodieInternalRowUtils.getCachedUnsafeProjection(structType, structType) new CloseableMappingIterator[InternalRow, UnsafeRow]( new HoodieSparkFileReaderFactory(storage).newParquetFileReader(filePath) @@ -73,16 +84,27 @@ class SparkFileFormatInternalRowReaderContext(readerMaps: mutable.Map[Long, Part } }).asInstanceOf[ClosableIterator[InternalRow]] } else { - val schemaPairHashKey = generateSchemaPairHashKey(dataSchema, requiredSchema) - if (!readerMaps.contains(schemaPairHashKey)) { - throw new IllegalStateException("schemas don't hash to a known reader") - } - new CloseableInternalRowIterator(readerMaps(schemaPairHashKey).apply(fileInfo)) + // partition value is empty because the spark parquet reader will append the partition columns to + // each row if they are given. That is the only usage of the partition values in the reader. + val fileInfo = sparkAdapter.getSparkPartitionedFileUtils + .createPartitionedFile(InternalRow.empty, filePath, start, length) + val (readSchema, readFilters) = getSchemaAndFiltersForRead(structType) + new CloseableInternalRowIterator(parquetFileReader.read(fileInfo, + readSchema, StructType(Seq.empty), readFilters, storage.getConf.asInstanceOf[StorageConfiguration[Configuration]])) } } - private def generateSchemaPairHashKey(dataSchema: Schema, requestedSchema: Schema): Long = { - dataSchema.hashCode() + requestedSchema.hashCode() + private def getSchemaAndFiltersForRead(structType: StructType): (StructType, Seq[Filter]) = { + (getHasLogFiles, getNeedsBootstrapMerge, getUseRecordPosition) match { + case (false, false, _) => + (structType, filters) + case (false, true, true) => + (getAppliedRequiredSchema(structType), filters) + case (true, _, true) => + (getAppliedRequiredSchema(structType), recordKeyFilters) + case (_, _, _) => + (structType, Seq.empty) + } } /** @@ -101,46 +123,147 @@ class SparkFileFormatInternalRowReaderContext(readerMaps: mutable.Map[Long, Part } override def mergeBootstrapReaders(skeletonFileIterator: ClosableIterator[InternalRow], - dataFileIterator: ClosableIterator[InternalRow]): ClosableIterator[InternalRow] = { - doBootstrapMerge(skeletonFileIterator.asInstanceOf[ClosableIterator[Any]], - dataFileIterator.asInstanceOf[ClosableIterator[Any]]) + skeletonRequiredSchema: Schema, + dataFileIterator: ClosableIterator[InternalRow], + dataRequiredSchema: Schema): ClosableIterator[InternalRow] = { + doBootstrapMerge(skeletonFileIterator.asInstanceOf[ClosableIterator[Any]], skeletonRequiredSchema, + dataFileIterator.asInstanceOf[ClosableIterator[Any]], dataRequiredSchema) } - protected def doBootstrapMerge(skeletonFileIterator: ClosableIterator[Any], dataFileIterator: ClosableIterator[Any]): ClosableIterator[InternalRow] = { - new ClosableIterator[Any] { - val combinedRow = new JoinedRow() - - override def hasNext: Boolean = { - //If the iterators are out of sync it is probably due to filter pushdown - checkState(dataFileIterator.hasNext == skeletonFileIterator.hasNext, - "Bootstrap data-file iterator and skeleton-file iterator have to be in-sync!") - dataFileIterator.hasNext && skeletonFileIterator.hasNext + private def doBootstrapMerge(skeletonFileIterator: ClosableIterator[Any], + skeletonRequiredSchema: Schema, + dataFileIterator: ClosableIterator[Any], + dataRequiredSchema: Schema): ClosableIterator[InternalRow] = { + if (getUseRecordPosition) { + assert(AvroSchemaUtils.containsFieldInSchema(skeletonRequiredSchema, ROW_INDEX_TEMPORARY_COLUMN_NAME)) + assert(AvroSchemaUtils.containsFieldInSchema(dataRequiredSchema, ROW_INDEX_TEMPORARY_COLUMN_NAME)) + val javaSet = new java.util.HashSet[String]() + javaSet.add(ROW_INDEX_TEMPORARY_COLUMN_NAME) + val skeletonProjection = projectRecord(skeletonRequiredSchema, + AvroSchemaUtils.removeFieldsFromSchema(skeletonRequiredSchema, javaSet)) + //If we have log files, we will want to do position based merging with those as well, + //so leave the row index column at the end + val dataProjection = if (getHasLogFiles) { + getIdentityProjection + } else { + projectRecord(dataRequiredSchema, + AvroSchemaUtils.removeFieldsFromSchema(dataRequiredSchema, javaSet)) } - override def next(): Any = { - (skeletonFileIterator.next(), dataFileIterator.next()) match { - case (s: ColumnarBatch, d: ColumnarBatch) => - val numCols = s.numCols() + d.numCols() - val vecs: Array[ColumnVector] = new Array[ColumnVector](numCols) - for (i <- 0 until numCols) { - if (i < s.numCols()) { - vecs(i) = s.column(i) + //Always use internal row for positional merge because + //we need to iterate row by row when merging + new CachingIterator[InternalRow] { + val combinedRow = new JoinedRow() + + //position column will always be at the end of the row + private def getPos(row: InternalRow): Long = { + row.getLong(row.numFields-1) + } + + private def getNextSkeleton: (InternalRow, Long) = { + val nextSkeletonRow = skeletonFileIterator.next().asInstanceOf[InternalRow] + (nextSkeletonRow, getPos(nextSkeletonRow)) + } + + private def getNextData: (InternalRow, Long) = { + val nextSkeletonRow = skeletonFileIterator.next().asInstanceOf[InternalRow] + (nextSkeletonRow, getPos(nextSkeletonRow)) + } + + override def close(): Unit = { + skeletonFileIterator.close() + dataFileIterator.close() + } + + override protected def doHasNext(): Boolean = { + if (!dataFileIterator.hasNext || !skeletonFileIterator.hasNext) { + false + } else { + var nextSkeleton = getNextSkeleton + var nextData = getNextData + while (nextSkeleton._2 != nextData._2) { + if (nextSkeleton._2 > nextData._2) { + if (!dataFileIterator.hasNext) { + return false + } else { + nextData = getNextData + } } else { - vecs(i) = d.column(i - s.numCols()) + if (!skeletonFileIterator.hasNext) { + return false + } else { + nextSkeleton = getNextSkeleton + } } } - assert(s.numRows() == d.numRows()) - sparkAdapter.makeColumnarBatch(vecs, s.numRows()) - case (_: ColumnarBatch, _: InternalRow) => throw new IllegalStateException("InternalRow ColumnVector mismatch") - case (_: InternalRow, _: ColumnarBatch) => throw new IllegalStateException("InternalRow ColumnVector mismatch") - case (s: InternalRow, d: InternalRow) => combinedRow(s, d) + nextRecord = combinedRow(skeletonProjection.apply(nextSkeleton._1), dataProjection.apply(nextData._1)) + true + } } } + } else { + new ClosableIterator[Any] { + val combinedRow = new JoinedRow() - override def close(): Unit = { - skeletonFileIterator.close() - dataFileIterator.close() - } - }.asInstanceOf[ClosableIterator[InternalRow]] + override def hasNext: Boolean = { + //If the iterators are out of sync it is probably due to filter pushdown + checkState(dataFileIterator.hasNext == skeletonFileIterator.hasNext, + "Bootstrap data-file iterator and skeleton-file iterator have to be in-sync!") + dataFileIterator.hasNext && skeletonFileIterator.hasNext + } + + override def next(): Any = { + (skeletonFileIterator.next(), dataFileIterator.next()) match { + case (s: ColumnarBatch, d: ColumnarBatch) => + //This will not be used until [HUDI-7693] is implemented + val numCols = s.numCols() + d.numCols() + val vecs: Array[ColumnVector] = new Array[ColumnVector](numCols) + for (i <- 0 until numCols) { + if (i < s.numCols()) { + vecs(i) = s.column(i) + } else { + vecs(i) = d.column(i - s.numCols()) + } + } + assert(s.numRows() == d.numRows()) + sparkAdapter.makeColumnarBatch(vecs, s.numRows()) + case (_: ColumnarBatch, _: InternalRow) => throw new IllegalStateException("InternalRow ColumnVector mismatch") + case (_: InternalRow, _: ColumnarBatch) => throw new IllegalStateException("InternalRow ColumnVector mismatch") + case (s: InternalRow, d: InternalRow) => combinedRow(s, d) + } + } + + override def close(): Unit = { + skeletonFileIterator.close() + dataFileIterator.close() + } + }.asInstanceOf[ClosableIterator[InternalRow]] + } } } + +object SparkFileFormatInternalRowReaderContext { + // From "namedExpressions.scala": Used to construct to record position field metadata. + private val FILE_SOURCE_GENERATED_METADATA_COL_ATTR_KEY = "__file_source_generated_metadata_col" + private val FILE_SOURCE_METADATA_COL_ATTR_KEY = "__file_source_metadata_col" + private val METADATA_COL_ATTR_KEY = "__metadata_col" + + def getRecordKeyRelatedFilters(filters: Seq[Filter], recordKeyColumn: String): Seq[Filter] = { + filters.filter(f => f.references.exists(c => c.equalsIgnoreCase(recordKeyColumn))) + } + + def getAppliedRequiredSchema(requiredSchema: StructType): StructType = { + val metadata = new MetadataBuilder() + .putString(METADATA_COL_ATTR_KEY, ROW_INDEX_TEMPORARY_COLUMN_NAME) + .putBoolean(FILE_SOURCE_METADATA_COL_ATTR_KEY, value = true) + .putString(FILE_SOURCE_GENERATED_METADATA_COL_ATTR_KEY, ROW_INDEX_TEMPORARY_COLUMN_NAME) + .build() + val rowIndexField = StructField(ROW_INDEX_TEMPORARY_COLUMN_NAME, LongType, nullable = false, metadata) + StructType(requiredSchema.fields.filterNot(isIndexTempColumn) :+ rowIndexField) + } + + private def isIndexTempColumn(field: StructField): Boolean = { + field.name.equals(ROW_INDEX_TEMPORARY_COLUMN_NAME) + } + +} \ No newline at end of file diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieInternalRowUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieInternalRowUtils.scala index 58230965e133..f3eb2214ea22 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieInternalRowUtils.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieInternalRowUtils.scala @@ -75,13 +75,6 @@ object HoodieInternalRowUtils { .getOrElseUpdate((from, to), generateUnsafeProjection(from, to)) } - /** - * due to scala2.11 and HoodieCatalystExpressionUtils is both an object and trait, - * we can't directly call generateUnsafeProjection from java code - */ - def generateUnsafeProjectionAlias(from: StructType, to: StructType): UnsafeProjection = { - generateUnsafeProjection(from, to) - } /** * Provides cached instance of [[UnsafeRowWriter]] transforming provided [[InternalRow]]s from * one [[StructType]] and into another [[StructType]] diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkParquetReader.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkParquetReader.scala index 920e4cb0e0b1..4ed4b0cd1d2f 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkParquetReader.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkParquetReader.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.datasources.parquet import org.apache.hadoop.conf.Configuration +import org.apache.hudi.storage.StorageConfiguration import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.sources.Filter @@ -33,12 +34,12 @@ trait SparkParquetReader extends Serializable { * @param requiredSchema desired output schema of the data * @param partitionSchema schema of the partition columns. Partition values will be appended to the end of every row * @param filters filters for data skipping. Not guaranteed to be used; the spark plan will also apply the filters. - * @param sharedConf the hadoop conf + * @param storageConf the hadoop conf * @return iterator of rows read from the file output type says [[InternalRow]] but could be [[ColumnarBatch]] */ def read(file: PartitionedFile, requiredSchema: StructType, partitionSchema: StructType, filters: Seq[Filter], - sharedConf: Configuration): Iterator[InternalRow] + storageConf: StorageConfiguration[Configuration]): Iterator[InternalRow] } diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaUtils.java b/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaUtils.java index ba747a63cbc0..da3ab4824b8b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaUtils.java @@ -231,7 +231,13 @@ private static Option findNestedField(Schema schema, String[] fiel if (!nestedPart.isPresent()) { return Option.empty(); } - return nestedPart; + boolean isUnion = false; + if (foundSchema.getType().equals(Schema.Type.UNION)) { + isUnion = true; + foundSchema = resolveNullableSchema(foundSchema); + } + Schema newSchema = Schema.createRecord(foundSchema.getName(), foundSchema.getDoc(), foundSchema.getNamespace(), false, Collections.singletonList(nestedPart.get())); + return Option.of(new Schema.Field(foundField.name(), isUnion ? createNullableSchema(newSchema) : newSchema, foundField.doc(), foundField.defaultVal())); } public static Schema appendFieldsToSchemaDedupNested(Schema schema, List newFields) { @@ -291,6 +297,16 @@ private static Schema appendFieldsToSchemaBase(Schema schema, List return newSchema; } + public static Schema removeFieldsFromSchema(Schema schema, Set fieldNames) { + List fields = schema.getFields().stream() + .filter(field -> !fieldNames.contains(field.name())) + .map(field -> new Schema.Field(field.name(), field.schema(), field.doc(), field.defaultVal())) + .collect(Collectors.toList()); + Schema newSchema = Schema.createRecord(schema.getName(), schema.getDoc(), schema.getNamespace(), schema.isError()); + newSchema.setFields(fields); + return newSchema; + } + /** * Passed in {@code Union} schema and will try to resolve the field with the {@code fieldSchemaFullName} * w/in the union returning its corresponding schema diff --git a/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java b/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java index 78f58db7eb5f..577478b6ae62 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java @@ -22,6 +22,7 @@ import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordMerger; +import org.apache.hudi.common.table.read.HoodieFileGroupReaderSchemaHandler; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.ClosableIterator; import org.apache.hudi.storage.HoodieStorage; @@ -32,6 +33,7 @@ import org.apache.avro.generic.IndexedRecord; import java.io.IOException; +import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.function.UnaryOperator; @@ -48,6 +50,88 @@ * and {@code RowData} in Flink. */ public abstract class HoodieReaderContext { + + private HoodieFileGroupReaderSchemaHandler schemaHandler = null; + private String tablePath = null; + private String latestCommitTime = null; + private HoodieRecordMerger recordMerger = null; + private Boolean hasLogFiles = null; + private Boolean hasBootstrapBaseFile = null; + private Boolean needsBootstrapMerge = null; + private Boolean useRecordPosition = null; + + // Getter and Setter for schemaHandler + public HoodieFileGroupReaderSchemaHandler getSchemaHandler() { + return schemaHandler; + } + + public void setSchemaHandler(HoodieFileGroupReaderSchemaHandler schemaHandler) { + this.schemaHandler = schemaHandler; + } + + public String getTablePath() { + if (tablePath == null) { + throw new IllegalStateException("Table path not set in reader context."); + } + return tablePath; + } + + public void setTablePath(String tablePath) { + this.tablePath = tablePath; + } + + public String getLatestCommitTime() { + return latestCommitTime; + } + + public void setLatestCommitTime(String latestCommitTime) { + this.latestCommitTime = latestCommitTime; + } + + public HoodieRecordMerger getRecordMerger() { + return recordMerger; + } + + public void setRecordMerger(HoodieRecordMerger recordMerger) { + this.recordMerger = recordMerger; + } + + // Getter and Setter for hasLogFiles + public boolean getHasLogFiles() { + return hasLogFiles; + } + + public void setHasLogFiles(boolean hasLogFiles) { + this.hasLogFiles = hasLogFiles; + } + + // Getter and Setter for hasBootstrapBaseFile + public boolean getHasBootstrapBaseFile() { + return hasBootstrapBaseFile; + } + + public void setHasBootstrapBaseFile(boolean hasBootstrapBaseFile) { + this.hasBootstrapBaseFile = hasBootstrapBaseFile; + } + + // Getter and Setter for needsBootstrapMerge + public boolean getNeedsBootstrapMerge() { + return needsBootstrapMerge; + } + + public void setNeedsBootstrapMerge(boolean needsBootstrapMerge) { + this.needsBootstrapMerge = needsBootstrapMerge; + } + + // Getter and Setter for useRecordPosition + public boolean getUseRecordPosition() { + return useRecordPosition; + } + + public void setUseRecordPosition(boolean useRecordPosition) { + this.useRecordPosition = useRecordPosition; + } + // These internal key names are only used in memory for record metadata and merging, // and should not be persisted to storage. public static final String INTERNAL_META_RECORD_KEY = "_0"; @@ -197,20 +281,29 @@ public Map updateSchemaAndResetOrderingValInMetadata(Map mergeBootstrapReaders(ClosableIterator skeletonFileIterator, ClosableIterator dataFileIterator); + public abstract ClosableIterator mergeBootstrapReaders(ClosableIterator skeletonFileIterator, + Schema skeletonRequiredSchema, + ClosableIterator dataFileIterator, + Schema dataRequiredSchema); /** * Creates a function that will reorder records of schema "from" to schema of "to" * all fields in "to" must be in "from", but not all fields in "from" must be in "to" * - * @param from the schema of records to be passed into UnaryOperator - * @param to the schema of records produced by UnaryOperator + * @param from the schema of records to be passed into UnaryOperator + * @param to the schema of records produced by UnaryOperator + * @param renamedColumns map of renamed columns where the key is the new name from the query and + * the value is the old name that exists in the file * @return a function that takes in a record and returns the record with reordered columns */ - public abstract UnaryOperator projectRecord(Schema from, Schema to); + public abstract UnaryOperator projectRecord(Schema from, Schema to, Map renamedColumns); + + public final UnaryOperator projectRecord(Schema from, Schema to) { + return projectRecord(from, to, Collections.emptyMap()); + } /** * Extracts the record position value from the record itself. diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/BaseHoodieLogRecordReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/BaseHoodieLogRecordReader.java index 30e659057ce5..d58c54a929f6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/BaseHoodieLogRecordReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/BaseHoodieLogRecordReader.java @@ -140,22 +140,21 @@ public abstract class BaseHoodieLogRecordReader { protected HoodieFileGroupRecordBuffer recordBuffer; protected BaseHoodieLogRecordReader(HoodieReaderContext readerContext, - HoodieStorage storage, String basePath, List logFilePaths, - Schema readerSchema, String latestInstantTime, + HoodieStorage storage, + List logFilePaths, boolean reverseReader, int bufferSize, Option instantRange, boolean withOperationField, boolean forceFullScan, Option partitionNameOverride, - InternalSchema internalSchema, Option keyFieldOverride, boolean enableOptimizedLogBlocksScan, HoodieRecordMerger recordMerger, HoodieFileGroupRecordBuffer recordBuffer) { this.readerContext = readerContext; - this.readerSchema = readerSchema; - this.latestInstantTime = latestInstantTime; + this.readerSchema = readerContext.getSchemaHandler().getRequiredSchema(); + this.latestInstantTime = readerContext.getLatestCommitTime(); this.hoodieTableMetaClient = HoodieTableMetaClient.builder() .setStorage(storage) - .setBasePath(basePath).build(); + .setBasePath(readerContext.getTablePath()).build(); // load class from the payload fully qualified class name HoodieTableConfig tableConfig = this.hoodieTableMetaClient.getTableConfig(); this.payloadClassFQN = tableConfig.getPayloadClass(); @@ -175,7 +174,7 @@ protected BaseHoodieLogRecordReader(HoodieReaderContext readerContext, this.instantRange = instantRange; this.withOperationField = withOperationField; this.forceFullScan = forceFullScan; - this.internalSchema = internalSchema == null ? InternalSchema.getEmptyInternalSchema() : internalSchema; + this.internalSchema = readerContext.getSchemaHandler().getInternalSchema(); this.enableOptimizedLogBlocksScan = enableOptimizedLogBlocksScan; if (keyFieldOverride.isPresent()) { @@ -845,16 +844,8 @@ public abstract static class Builder { public abstract Builder withStorage(HoodieStorage storage); - public abstract Builder withBasePath(String basePath); - public abstract Builder withLogFiles(List hoodieLogFiles); - public abstract Builder withReaderSchema(Schema schema); - - public abstract Builder withInternalSchema(InternalSchema internalSchema); - - public abstract Builder withLatestInstantTime(String latestInstantTime); - public abstract Builder withReverseReader(boolean reverseReader); public abstract Builder withBufferSize(int bufferSize); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordReader.java index b8c9048added..f6d0ce3a0b51 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordReader.java @@ -31,11 +31,9 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; -import org.apache.hudi.internal.schema.InternalSchema; import org.apache.hudi.storage.HoodieStorage; import org.apache.hudi.storage.StoragePath; -import org.apache.avro.Schema; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -70,19 +68,16 @@ public class HoodieMergedLogRecordReader extends BaseHoodieLogRecordReader @SuppressWarnings("unchecked") private HoodieMergedLogRecordReader(HoodieReaderContext readerContext, - HoodieStorage storage, String basePath, List logFilePaths, Schema readerSchema, - String latestInstantTime, - boolean reverseReader, int bufferSize, Option instantRange, + HoodieStorage storage, List logFilePaths, boolean reverseReader, + int bufferSize, Option instantRange, boolean withOperationField, boolean forceFullScan, Option partitionName, - InternalSchema internalSchema, Option keyFieldOverride, boolean enableOptimizedLogBlocksScan, HoodieRecordMerger recordMerger, HoodieFileGroupRecordBuffer recordBuffer) { - super(readerContext, storage, basePath, logFilePaths, readerSchema, latestInstantTime, reverseReader, bufferSize, - instantRange, withOperationField, forceFullScan, partitionName, internalSchema, keyFieldOverride, enableOptimizedLogBlocksScan, - recordMerger, recordBuffer); + super(readerContext, storage, logFilePaths, reverseReader, bufferSize, instantRange, withOperationField, + forceFullScan, partitionName, keyFieldOverride, enableOptimizedLogBlocksScan, recordMerger, recordBuffer); this.scannedPrefixes = new HashSet<>(); if (forceFullScan) { @@ -218,11 +213,7 @@ public void close() { public static class Builder extends BaseHoodieLogRecordReader.Builder { private HoodieReaderContext readerContext; private HoodieStorage storage; - private String basePath; private List logFilePaths; - private Schema readerSchema; - private InternalSchema internalSchema = InternalSchema.getEmptyInternalSchema(); - private String latestInstantTime; private boolean reverseReader; private int bufferSize; // specific configurations @@ -252,12 +243,6 @@ public Builder withStorage(HoodieStorage storage) { return this; } - @Override - public Builder withBasePath(String basePath) { - this.basePath = basePath; - return this; - } - @Override public Builder withLogFiles(List hoodieLogFiles) { this.logFilePaths = hoodieLogFiles.stream() @@ -267,18 +252,6 @@ public Builder withLogFiles(List hoodieLogFiles) { return this; } - @Override - public Builder withReaderSchema(Schema schema) { - this.readerSchema = schema; - return this; - } - - @Override - public Builder withLatestInstantTime(String latestInstantTime) { - this.latestInstantTime = latestInstantTime; - return this; - } - @Override public Builder withReverseReader(boolean reverseReader) { this.reverseReader = reverseReader; @@ -297,12 +270,6 @@ public Builder withInstantRange(Option instantRange) { return this; } - @Override - public Builder withInternalSchema(InternalSchema internalSchema) { - this.internalSchema = internalSchema; - return this; - } - public Builder withOperationField(boolean withOperationField) { this.withOperationField = withOperationField; return this; @@ -343,19 +310,19 @@ public Builder withRecordBuffer(HoodieFileGroupRecordBuffer recordBuffer) @Override public HoodieMergedLogRecordReader build() { + ValidationUtils.checkArgument(recordMerger != null); + ValidationUtils.checkArgument(recordBuffer != null); + ValidationUtils.checkArgument(readerContext != null); if (this.partitionName == null && CollectionUtils.nonEmpty(this.logFilePaths)) { this.partitionName = getRelativePartitionPath( - new StoragePath(basePath), new StoragePath(this.logFilePaths.get(0)).getParent()); + new StoragePath(readerContext.getTablePath()), new StoragePath(this.logFilePaths.get(0)).getParent()); } - ValidationUtils.checkArgument(recordMerger != null); - ValidationUtils.checkArgument(recordBuffer != null); return new HoodieMergedLogRecordReader<>( - readerContext, storage, basePath, logFilePaths, readerSchema, - latestInstantTime, reverseReader, - bufferSize, instantRange, + readerContext, storage, logFilePaths, + reverseReader, bufferSize, instantRange, withOperationField, forceFullScan, - Option.ofNullable(partitionName), internalSchema, + Option.ofNullable(partitionName), Option.ofNullable(keyFieldOverride), enableOptimizedLogBlocksScan, recordMerger, recordBuffer); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java index 70ddb5abff2f..0d5750e29928 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java @@ -24,20 +24,26 @@ import org.apache.hudi.common.model.DeleteRecord; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordMerger; +import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.KeySpec; import org.apache.hudi.common.table.log.block.HoodieDataBlock; import org.apache.hudi.common.table.log.block.HoodieLogBlock; import org.apache.hudi.common.util.DefaultSizeEstimator; import org.apache.hudi.common.util.HoodieRecordSizeEstimator; +import org.apache.hudi.common.util.InternalSchemaCache; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.collection.ClosableIterator; +import org.apache.hudi.common.util.collection.CloseableMappingIterator; import org.apache.hudi.common.util.collection.ExternalSpillableMap; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieCorruptedDataException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieKeyException; import org.apache.hudi.exception.HoodieValidationException; +import org.apache.hudi.internal.schema.InternalSchema; +import org.apache.hudi.internal.schema.action.InternalSchemaMerger; +import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter; import org.apache.avro.Schema; import org.roaringbitmap.longlong.Roaring64NavigableMap; @@ -45,17 +51,19 @@ import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; +import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.function.Function; import static org.apache.hudi.common.engine.HoodieReaderContext.INTERNAL_META_SCHEMA; +import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType.INSTANT_TIME; public abstract class HoodieBaseFileGroupRecordBuffer implements HoodieFileGroupRecordBuffer { protected final HoodieReaderContext readerContext; protected final Schema readerSchema; - protected final Schema baseFileSchema; protected final Option partitionNameOverrideOpt; protected final Option partitionPathFieldOpt; protected final HoodieRecordMerger recordMerger; @@ -65,10 +73,11 @@ public abstract class HoodieBaseFileGroupRecordBuffer implements HoodieFileGr protected Iterator, Map>> logRecordIterator; protected T nextRecord; protected boolean enablePartialMerging = false; + protected InternalSchema internalSchema; + protected HoodieTableMetaClient hoodieTableMetaClient; public HoodieBaseFileGroupRecordBuffer(HoodieReaderContext readerContext, - Schema readerSchema, - Schema baseFileSchema, + HoodieTableMetaClient hoodieTableMetaClient, Option partitionNameOverrideOpt, Option partitionPathFieldOpt, HoodieRecordMerger recordMerger, @@ -78,12 +87,13 @@ public HoodieBaseFileGroupRecordBuffer(HoodieReaderContext readerContext, ExternalSpillableMap.DiskMapType diskMapType, boolean isBitCaskDiskMapCompressionEnabled) { this.readerContext = readerContext; - this.readerSchema = readerSchema; - this.baseFileSchema = baseFileSchema; + this.readerSchema = readerContext.getSchemaHandler().getRequiredSchema(); this.partitionNameOverrideOpt = partitionNameOverrideOpt; this.partitionPathFieldOpt = partitionPathFieldOpt; this.recordMerger = recordMerger; this.payloadProps = payloadProps; + this.internalSchema = readerContext.getSchemaHandler().getInternalSchema(); + this.hoodieTableMetaClient = hoodieTableMetaClient; try { // Store merged records for all versions for this log file, set the in-memory footprint to maxInMemoryMapSize this.records = new ExternalSpillableMap<>(maxMemorySizeInBytes, spillableMapBasePath, new DefaultSizeEstimator<>(), @@ -173,7 +183,7 @@ protected Option>> doProcessNextDataRecord(T record, if (!combinedRecordAndSchemaOpt.isPresent()) { return Option.empty(); } - + Pair combinedRecordAndSchema = combinedRecordAndSchemaOpt.get(); HoodieRecord combinedRecord = combinedRecordAndSchema.getLeft(); @@ -232,7 +242,6 @@ protected Option doProcessNextDeletedRecord(DeleteRecord deleteRec * @param dataBlock * @param keySpecOpt * @return - * @throws IOException */ protected Pair, Schema> getRecordsIterator(HoodieDataBlock dataBlock, Option keySpecOpt) { ClosableIterator blockRecordsIterator; @@ -242,7 +251,34 @@ protected Pair, Schema> getRecordsIterator(HoodieDataBlock d } else { blockRecordsIterator = dataBlock.getEngineRecordIterator(readerContext); } - return Pair.of(blockRecordsIterator, dataBlock.getSchema()); + Pair, Schema> schemaTransformerWithEvolvedSchema = getSchemaTransformerWithEvolvedSchema(dataBlock); + return Pair.of(new CloseableMappingIterator<>( + blockRecordsIterator, schemaTransformerWithEvolvedSchema.getLeft()), schemaTransformerWithEvolvedSchema.getRight()); + } + + /** + * Get final Read Schema for support evolution. + * step1: find the fileSchema for current dataBlock. + * step2: determine whether fileSchema is compatible with the final read internalSchema. + * step3: merge fileSchema and read internalSchema to produce final read schema. + * + * @param dataBlock current processed block + * @return final read schema. + */ + protected Option, Schema>> composeEvolvedSchemaTransformer( + HoodieDataBlock dataBlock) { + if (internalSchema.isEmptySchema()) { + return Option.empty(); + } + + long currentInstantTime = Long.parseLong(dataBlock.getLogBlockHeader().get(INSTANT_TIME)); + InternalSchema fileSchema = InternalSchemaCache.searchSchemaAndCache(currentInstantTime, + hoodieTableMetaClient, false); + Pair> mergedInternalSchema = new InternalSchemaMerger(fileSchema, internalSchema, + true, false, false).mergeSchemaGetRenamed(); + Schema mergedAvroSchema = AvroInternalSchemaConverter.convert(mergedInternalSchema.getLeft(), readerSchema.getFullName()); + assert mergedAvroSchema.equals(readerSchema); + return Option.of(Pair.of(readerContext.projectRecord(dataBlock.getSchema(), mergedAvroSchema, mergedInternalSchema.getRight()), mergedAvroSchema)); } /** @@ -256,7 +292,7 @@ protected Pair, Schema> getRecordsIterator(HoodieDataBlock d * @throws IOException */ protected Option merge(Option older, Map olderInfoMap, - Option newer, Map newerInfoMap) throws IOException { + Option newer, Map newerInfoMap) throws IOException { if (!older.isPresent()) { return newer; } @@ -275,6 +311,9 @@ protected Option merge(Option older, Map olderInfoMap, if (mergedRecord.isPresent() && !mergedRecord.get().getLeft().isDelete(mergedRecord.get().getRight(), payloadProps)) { + if (!mergedRecord.get().getRight().equals(readerSchema)) { + return Option.ofNullable((T) mergedRecord.get().getLeft().rewriteRecordWithNewSchema(mergedRecord.get().getRight(), null, readerSchema).getData()); + } return Option.ofNullable((T) mergedRecord.get().getLeft().getData()); } return Option.empty(); @@ -285,8 +324,8 @@ protected Option merge(Option older, Map olderInfoMap, * 1. A set of pre-specified keys exists. * 2. The key of the record is not contained in the set. */ - protected boolean shouldSkip(T record, String keyFieldName, boolean isFullKey, Set keys) { - String recordKey = readerContext.getValue(record, readerSchema, keyFieldName).toString(); + protected boolean shouldSkip(T record, String keyFieldName, boolean isFullKey, Set keys, Schema dataBlockSchema) { + String recordKey = readerContext.getValue(record, dataBlockSchema, keyFieldName).toString(); // Can not extract the record key, throw. if (recordKey == null || recordKey.isEmpty()) { throw new HoodieKeyException("Can not extract the key for a record"); @@ -333,4 +372,51 @@ protected static List extractRecordPositions(HoodieLogBlock logBlock) thro return blockPositions; } + + protected boolean hasNextBaseRecord(T baseRecord, Pair, Map> logRecordInfo) throws IOException { + Map metadata = readerContext.generateMetadataForRecord( + baseRecord, readerSchema); + + Option resultRecord = logRecordInfo != null + ? merge(Option.of(baseRecord), metadata, logRecordInfo.getLeft(), logRecordInfo.getRight()) + : merge(Option.empty(), Collections.emptyMap(), Option.of(baseRecord), metadata); + if (resultRecord.isPresent()) { + nextRecord = readerContext.seal(resultRecord.get()); + return true; + } + return false; + } + + protected boolean hasNextLogRecord() throws IOException { + if (logRecordIterator == null) { + logRecordIterator = records.values().iterator(); + } + + while (logRecordIterator.hasNext()) { + Pair, Map> nextRecordInfo = logRecordIterator.next(); + Option resultRecord; + resultRecord = merge(Option.empty(), Collections.emptyMap(), + nextRecordInfo.getLeft(), nextRecordInfo.getRight()); + if (resultRecord.isPresent()) { + nextRecord = readerContext.seal(resultRecord.get()); + return true; + } + } + return false; + } + + protected Pair, Schema> getSchemaTransformerWithEvolvedSchema(HoodieDataBlock dataBlock) { + Option, Schema>> schemaEvolutionTransformerOpt = + composeEvolvedSchemaTransformer(dataBlock); + + // In case when schema has been evolved original persisted records will have to be + // transformed to adhere to the new schema + Function transformer = + schemaEvolutionTransformerOpt.map(Pair::getLeft) + .orElse(Function.identity()); + + Schema evolvedSchema = schemaEvolutionTransformerOpt.map(Pair::getRight) + .orElseGet(dataBlock::getSchema); + return Pair.of(transformer, evolvedSchema); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReader.java index 0f9e93309146..9db271aee29a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReader.java @@ -26,9 +26,9 @@ import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieLogFile; -import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordMerger; import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.HoodieMergedLogRecordReader; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.CachingIterator; @@ -37,6 +37,7 @@ import org.apache.hudi.common.util.collection.ExternalSpillableMap; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.internal.schema.InternalSchema; import org.apache.hudi.storage.HoodieStorage; import org.apache.hudi.storage.StoragePath; @@ -44,16 +45,10 @@ import java.io.Closeable; import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; import java.util.List; -import java.util.Map; import java.util.function.UnaryOperator; import java.util.stream.Collectors; -import java.util.stream.Stream; -import static org.apache.hudi.avro.AvroSchemaUtils.appendFieldsToSchema; -import static org.apache.hudi.avro.AvroSchemaUtils.findNestedField; import static org.apache.hudi.common.fs.FSUtils.getRelativePartitionPath; import static org.apache.hudi.common.util.ConfigUtils.getIntWithAltKeys; @@ -78,20 +73,8 @@ public final class HoodieFileGroupReader implements Closeable { private final long length; // Core structure to store and process records. private final HoodieFileGroupRecordBuffer recordBuffer; - private final HoodieFileGroupReaderState readerState = new HoodieFileGroupReaderState(); private ClosableIterator baseFileIterator; - private HoodieRecordMerger recordMerger; - - private final Schema dataSchema; - - // requestedSchema: the schema that the caller requests - private final Schema requestedSchema; - - // requiredSchema: the requestedSchema with any additional columns required for merging etc - private final Schema requiredSchema; - - private final HoodieTableConfig hoodieTableConfig; - + private final HoodieRecordMerger recordMerger; private final Option> outputConverter; public HoodieFileGroupReader(HoodieReaderContext readerContext, @@ -101,6 +84,8 @@ public HoodieFileGroupReader(HoodieReaderContext readerContext, FileSlice fileSlice, Schema dataSchema, Schema requestedSchema, + Option internalSchemaOpt, + HoodieTableMetaClient hoodieTableMetaClient, TypedProperties props, HoodieTableConfig tableConfig, long start, @@ -118,29 +103,23 @@ public HoodieFileGroupReader(HoodieReaderContext readerContext, this.start = start; this.length = length; this.recordMerger = readerContext.getRecordMerger(tableConfig.getRecordMergerStrategy()); - this.readerState.tablePath = tablePath; - this.readerState.latestCommitTime = latestCommitTime; - this.dataSchema = dataSchema; - this.requestedSchema = requestedSchema; - this.hoodieTableConfig = tableConfig; - this.requiredSchema = generateRequiredSchema(); - if (!requestedSchema.equals(requiredSchema)) { - this.outputConverter = Option.of(readerContext.projectRecord(requiredSchema, requestedSchema)); - } else { - this.outputConverter = Option.empty(); - } - this.readerState.baseFileAvroSchema = requiredSchema; - this.readerState.logRecordAvroSchema = requiredSchema; - this.readerState.mergeProps.putAll(props); + readerContext.setRecordMerger(this.recordMerger); + readerContext.setTablePath(tablePath); + readerContext.setLatestCommitTime(latestCommitTime); + readerContext.setUseRecordPosition(shouldUseRecordPosition); + readerContext.setHasLogFiles(!this.logFiles.isEmpty()); + readerContext.setHasBootstrapBaseFile(hoodieBaseFileOption.isPresent() && hoodieBaseFileOption.get().getBootstrapBaseFile().isPresent()); + readerContext.setSchemaHandler(shouldUseRecordPosition + ? new HoodiePositionBasedSchemaHandler<>(readerContext, dataSchema, requestedSchema, internalSchemaOpt, tableConfig) + : new HoodieFileGroupReaderSchemaHandler<>(readerContext, dataSchema, requestedSchema, internalSchemaOpt, tableConfig)); + this.outputConverter = readerContext.getSchemaHandler().getOutputConverter(); this.recordBuffer = this.logFiles.isEmpty() ? null : shouldUseRecordPosition - ? new HoodiePositionBasedFileGroupRecordBuffer<>( - readerContext, requiredSchema, requiredSchema, Option.empty(), Option.empty(), - recordMerger, props, maxMemorySizeInBytes, spillableMapBasePath, diskMapType, isBitCaskDiskMapCompressionEnabled) - : new HoodieKeyBasedFileGroupRecordBuffer<>( - readerContext, requiredSchema, requiredSchema, Option.empty(), Option.empty(), - recordMerger, props, maxMemorySizeInBytes, spillableMapBasePath, diskMapType, isBitCaskDiskMapCompressionEnabled); + ? new HoodiePositionBasedFileGroupRecordBuffer<>(readerContext, hoodieTableMetaClient, Option.empty(), + Option.empty(), recordMerger, props, maxMemorySizeInBytes, spillableMapBasePath, diskMapType, isBitCaskDiskMapCompressionEnabled) + : new HoodieKeyBasedFileGroupRecordBuffer<>(readerContext, hoodieTableMetaClient, Option.empty(), + Option.empty(), recordMerger, props, maxMemorySizeInBytes, spillableMapBasePath, diskMapType, isBitCaskDiskMapCompressionEnabled); } /** @@ -168,96 +147,40 @@ private ClosableIterator makeBaseFileIterator() throws IOException { } return readerContext.getFileRecordIterator( - baseFile.getStoragePath(), start, - length, - dataSchema, requiredSchema, storage); - } - - private Schema generateRequiredSchema() { - //might need to change this if other queries than mor have mandatory fields - if (logFiles.isEmpty()) { - return requestedSchema; - } - - List addedFields = new ArrayList<>(); - for (String field : recordMerger.getMandatoryFieldsForMerging(hoodieTableConfig)) { - if (requestedSchema.getField(field) == null) { - Option foundFieldOpt = findNestedField(dataSchema, field); - if (!foundFieldOpt.isPresent()) { - throw new IllegalArgumentException("Field: " + field + " does not exist in the table schema"); - } - Schema.Field foundField = foundFieldOpt.get(); - addedFields.add(foundField); - } - } - - if (addedFields.isEmpty()) { - return maybeReorderForBootstrap(requestedSchema); - } - - return maybeReorderForBootstrap(appendFieldsToSchema(requestedSchema, addedFields)); - } - - private Schema maybeReorderForBootstrap(Schema input) { - if (this.hoodieBaseFileOption.isPresent() && this.hoodieBaseFileOption.get().getBootstrapBaseFile().isPresent()) { - Pair, List> requiredFields = getDataAndMetaCols(input); - if (!(requiredFields.getLeft().isEmpty() || requiredFields.getRight().isEmpty())) { - return createSchemaFromFields(Stream.concat(requiredFields.getLeft().stream(), requiredFields.getRight().stream()) - .collect(Collectors.toList())); - } - } - return input; - } - - private static Pair, List> getDataAndMetaCols(Schema schema) { - Map> fieldsByMeta = schema.getFields().stream() - .collect(Collectors.partitioningBy(f -> HoodieRecord.HOODIE_META_COLUMNS_WITH_OPERATION.contains(f.name()))); - return Pair.of(fieldsByMeta.getOrDefault(true, Collections.emptyList()), - fieldsByMeta.getOrDefault(false, Collections.emptyList())); - } - - private Schema createSchemaFromFields(List fields) { - //fields have positions set, so we need to remove them due to avro setFields implementation - for (int i = 0; i < fields.size(); i++) { - Schema.Field curr = fields.get(i); - fields.set(i, new Schema.Field(curr.name(), curr.schema(), curr.doc(), curr.defaultVal())); - } - Schema newSchema = Schema.createRecord(dataSchema.getName(), dataSchema.getDoc(), dataSchema.getNamespace(), dataSchema.isError()); - newSchema.setFields(fields); - return newSchema; + baseFile.getStoragePath(), start, length, + readerContext.getSchemaHandler().getDataSchema(), + readerContext.getSchemaHandler().getRequiredSchema(), storage); } private ClosableIterator makeBootstrapBaseFileIterator(HoodieBaseFile baseFile) throws IOException { BaseFile dataFile = baseFile.getBootstrapBaseFile().get(); - Pair, List> requiredFields = - getDataAndMetaCols(requiredSchema); - Pair, List> allFields = getDataAndMetaCols(dataSchema); - - Option> dataFileIterator = - requiredFields.getRight().isEmpty() ? Option.empty() : - Option.of(readerContext.getFileRecordIterator( - dataFile.getStoragePath(), 0, - dataFile.getFileLen(), - createSchemaFromFields(allFields.getRight()), - createSchemaFromFields(requiredFields.getRight()), storage)); - - Option> skeletonFileIterator = - requiredFields.getLeft().isEmpty() ? Option.empty() : - Option.of(readerContext.getFileRecordIterator( - baseFile.getStoragePath(), 0, - baseFile.getFileLen(), - createSchemaFromFields(allFields.getLeft()), - createSchemaFromFields(requiredFields.getLeft()), storage)); + Pair,List> requiredFields = readerContext.getSchemaHandler().getBootstrapRequiredFields(); + Pair,List> allFields = readerContext.getSchemaHandler().getBootstrapDataFields(); + Option,Schema>> dataFileIterator = + makeBootstrapBaseFileIteratorHelper(requiredFields.getRight(), allFields.getRight(), dataFile); + Option,Schema>> skeletonFileIterator = + makeBootstrapBaseFileIteratorHelper(requiredFields.getLeft(), allFields.getLeft(), baseFile); if (!dataFileIterator.isPresent() && !skeletonFileIterator.isPresent()) { throw new IllegalStateException("should not be here if only partition cols are required"); } else if (!dataFileIterator.isPresent()) { - return skeletonFileIterator.get(); + return skeletonFileIterator.get().getLeft(); } else if (!skeletonFileIterator.isPresent()) { - return dataFileIterator.get(); + return dataFileIterator.get().getLeft(); } else { - return readerContext.mergeBootstrapReaders(skeletonFileIterator.get(), - dataFileIterator.get()); + return readerContext.mergeBootstrapReaders(skeletonFileIterator.get().getLeft(), skeletonFileIterator.get().getRight(), + dataFileIterator.get().getLeft(), dataFileIterator.get().getRight()); + } + } + + private Option,Schema>> makeBootstrapBaseFileIteratorHelper(List requiredFields, + List allFields, + BaseFile file) throws IOException { + if (requiredFields.isEmpty()) { + return Option.empty(); } + Schema requiredSchema = readerContext.getSchemaHandler().createSchemaFromFields(requiredFields); + return Option.of(Pair.of(readerContext.getFileRecordIterator(file.getStoragePath(), 0, file.getFileLen(), + readerContext.getSchemaHandler().createSchemaFromFields(allFields), requiredSchema, storage), requiredSchema)); } /** @@ -284,18 +207,15 @@ public T next() { } private void scanLogFiles() { - String path = readerState.tablePath; + String path = readerContext.getTablePath(); HoodieMergedLogRecordReader logRecordReader = HoodieMergedLogRecordReader.newBuilder() .withHoodieReaderContext(readerContext) .withStorage(storage) - .withBasePath(readerState.tablePath) .withLogFiles(logFiles) - .withLatestInstantTime(readerState.latestCommitTime) - .withReaderSchema(readerState.logRecordAvroSchema) .withReverseReader(false) .withBufferSize(getIntWithAltKeys(props, HoodieMemoryConfig.MAX_DFS_STREAM_BUFFER_SIZE)) .withPartition(getRelativePartitionPath( - new StoragePath(readerState.tablePath), logFiles.get(0).getPath().getParent())) + new StoragePath(path), logFiles.get(0).getPath().getParent())) .withRecordMerger(recordMerger) .withRecordBuffer(recordBuffer) .build(); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderSchemaHandler.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderSchemaHandler.java new file mode 100644 index 000000000000..1a4c68e00202 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderSchemaHandler.java @@ -0,0 +1,185 @@ +/* + * 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.common.table.read; + +import org.apache.hudi.common.engine.HoodieReaderContext; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordMerger; +import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.internal.schema.InternalSchema; +import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter; + +import org.apache.avro.Schema; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.function.UnaryOperator; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.hudi.avro.AvroSchemaUtils.appendFieldsToSchemaDedupNested; +import static org.apache.hudi.avro.AvroSchemaUtils.findNestedField; + +/** + * This class is responsible for handling the schema for the file group reader. + */ +public class HoodieFileGroupReaderSchemaHandler { + + protected final Schema dataSchema; + + // requestedSchema: the schema that the caller requests + protected final Schema requestedSchema; + + // requiredSchema: the requestedSchema with any additional columns required for merging etc + protected final Schema requiredSchema; + + protected final InternalSchema internalSchema; + + + protected final HoodieTableConfig hoodieTableConfig; + + protected final HoodieReaderContext readerContext; + + protected final HoodieRecordMerger recordMerger; + + protected final boolean hasBootstrapBaseFile; + protected boolean needsBootstrapMerge; + + protected final boolean needsMORMerge; + + public HoodieFileGroupReaderSchemaHandler(HoodieReaderContext readerContext, + Schema dataSchema, + Schema requestedSchema, + Option internalSchemaOpt, + HoodieTableConfig hoodieTableConfig) { + this.readerContext = readerContext; + this.hasBootstrapBaseFile = readerContext.getHasBootstrapBaseFile(); + this.needsMORMerge = readerContext.getHasLogFiles(); + this.recordMerger = readerContext.getRecordMerger(); + this.dataSchema = dataSchema; + this.requestedSchema = requestedSchema; + this.hoodieTableConfig = hoodieTableConfig; + this.requiredSchema = prepareSchema(); + this.internalSchema = pruneInternalSchema(requiredSchema, internalSchemaOpt); + readerContext.setNeedsBootstrapMerge(this.needsBootstrapMerge); + } + + public Schema getDataSchema() { + return this.dataSchema; + } + + public Schema getRequestedSchema() { + return this.requestedSchema; + } + + public Schema getRequiredSchema() { + return this.requiredSchema; + } + + public InternalSchema getInternalSchema() { + return this.internalSchema; + } + + public Option> getOutputConverter() { + if (!requestedSchema.equals(requiredSchema)) { + return Option.of(readerContext.projectRecord(requiredSchema, requestedSchema)); + } + return Option.empty(); + } + + private static InternalSchema pruneInternalSchema(Schema requiredSchema, Option internalSchemaOption) { + if (!internalSchemaOption.isPresent()) { + return InternalSchema.getEmptyInternalSchema(); + } + InternalSchema notPruned = internalSchemaOption.get(); + if (notPruned == null || notPruned.isEmptySchema()) { + return InternalSchema.getEmptyInternalSchema(); + } + + return AvroInternalSchemaConverter.pruneAvroSchemaToInternalSchema(requiredSchema, notPruned); + } + + private Schema generateRequiredSchema() { + //might need to change this if other queries than mor have mandatory fields + if (!needsMORMerge) { + return requestedSchema; + } + + List addedFields = new ArrayList<>(); + for (String field : recordMerger.getMandatoryFieldsForMerging(hoodieTableConfig)) { + if (!findNestedField(requestedSchema, field).isPresent()) { + Option foundFieldOpt = findNestedField(dataSchema, field); + if (!foundFieldOpt.isPresent()) { + throw new IllegalArgumentException("Field: " + field + " does not exist in the table schema"); + } + Schema.Field foundField = foundFieldOpt.get(); + addedFields.add(foundField); + } + } + + if (addedFields.isEmpty()) { + return requestedSchema; + } + + return appendFieldsToSchemaDedupNested(requestedSchema, addedFields); + } + + protected Schema prepareSchema() { + Schema preReorderRequiredSchema = generateRequiredSchema(); + Pair, List> requiredFields = getDataAndMetaCols(preReorderRequiredSchema); + this.needsBootstrapMerge = hasBootstrapBaseFile && !requiredFields.getLeft().isEmpty() && !requiredFields.getRight().isEmpty(); + return needsBootstrapMerge + ? createSchemaFromFields(Stream.concat(requiredFields.getLeft().stream(), requiredFields.getRight().stream()).collect(Collectors.toList())) + : preReorderRequiredSchema; + } + + public Pair,List> getBootstrapRequiredFields() { + return getDataAndMetaCols(requiredSchema); + } + + public Pair,List> getBootstrapDataFields() { + return getDataAndMetaCols(dataSchema); + } + + private static Pair, List> getDataAndMetaCols(Schema schema) { + Map> fieldsByMeta = schema.getFields().stream() + //if there are no data fields, then we don't want to think the temp col is a data col + .filter(f -> !Objects.equals(f.name(), HoodiePositionBasedFileGroupRecordBuffer.ROW_INDEX_TEMPORARY_COLUMN_NAME)) + .collect(Collectors.partitioningBy(f -> HoodieRecord.HOODIE_META_COLUMNS_WITH_OPERATION.contains(f.name()))); + return Pair.of(fieldsByMeta.getOrDefault(true, Collections.emptyList()), + fieldsByMeta.getOrDefault(false, Collections.emptyList())); + } + + public Schema createSchemaFromFields(List fields) { + //fields have positions set, so we need to remove them due to avro setFields implementation + for (int i = 0; i < fields.size(); i++) { + Schema.Field curr = fields.get(i); + fields.set(i, new Schema.Field(curr.name(), curr.schema(), curr.doc(), curr.defaultVal())); + } + Schema newSchema = Schema.createRecord(dataSchema.getName(), dataSchema.getDoc(), dataSchema.getNamespace(), dataSchema.isError()); + newSchema.setFields(fields); + return newSchema; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderState.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderState.java deleted file mode 100644 index e50713bb40a0..000000000000 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderState.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.hudi.common.table.read; - -import org.apache.hudi.common.config.TypedProperties; - -import org.apache.avro.Schema; - -/** - * A class holding the state that is needed by {@code HoodieFileGroupReader}, - * e.g., schema, merging strategy, etc. - */ -public class HoodieFileGroupReaderState { - public String tablePath; - public String latestCommitTime; - public Schema baseFileAvroSchema; - public Schema logRecordAvroSchema; - public TypedProperties mergeProps = new TypedProperties(); -} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieKeyBasedFileGroupRecordBuffer.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieKeyBasedFileGroupRecordBuffer.java index 0430a42e8639..96d4306afd40 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieKeyBasedFileGroupRecordBuffer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieKeyBasedFileGroupRecordBuffer.java @@ -23,6 +23,7 @@ import org.apache.hudi.common.engine.HoodieReaderContext; import org.apache.hudi.common.model.DeleteRecord; import org.apache.hudi.common.model.HoodieRecordMerger; +import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.KeySpec; import org.apache.hudi.common.table.log.block.HoodieDataBlock; import org.apache.hudi.common.table.log.block.HoodieDeleteBlock; @@ -37,7 +38,6 @@ import java.io.IOException; import java.io.Serializable; import java.util.Arrays; -import java.util.Collections; import java.util.Iterator; import java.util.Map; @@ -48,9 +48,9 @@ * {@link #hasNext} method is called. */ public class HoodieKeyBasedFileGroupRecordBuffer extends HoodieBaseFileGroupRecordBuffer { + public HoodieKeyBasedFileGroupRecordBuffer(HoodieReaderContext readerContext, - Schema readerSchema, - Schema baseFileSchema, + HoodieTableMetaClient hoodieTableMetaClient, Option partitionNameOverrideOpt, Option partitionPathFieldOpt, HoodieRecordMerger recordMerger, @@ -59,7 +59,7 @@ public HoodieKeyBasedFileGroupRecordBuffer(HoodieReaderContext readerContext, String spillableMapBasePath, ExternalSpillableMap.DiskMapType diskMapType, boolean isBitCaskDiskMapCompressionEnabled) { - super(readerContext, readerSchema, baseFileSchema, partitionNameOverrideOpt, partitionPathFieldOpt, + super(readerContext, hoodieTableMetaClient, partitionNameOverrideOpt, partitionPathFieldOpt, recordMerger, payloadProps, maxMemorySizeInBytes, spillableMapBasePath, diskMapType, isBitCaskDiskMapCompressionEnabled); } @@ -134,35 +134,14 @@ protected boolean doHasNext() throws IOException { while (baseFileIterator.hasNext()) { T baseRecord = baseFileIterator.next(); - String recordKey = readerContext.getRecordKey(baseRecord, baseFileSchema); + String recordKey = readerContext.getRecordKey(baseRecord, readerSchema); Pair, Map> logRecordInfo = records.remove(recordKey); - Map metadata = readerContext.generateMetadataForRecord( - baseRecord, baseFileSchema); - - Option resultRecord = logRecordInfo != null - ? merge(Option.of(baseRecord), metadata, logRecordInfo.getLeft(), logRecordInfo.getRight()) - : merge(Option.empty(), Collections.emptyMap(), Option.of(baseRecord), metadata); - if (resultRecord.isPresent()) { - nextRecord = readerContext.seal(resultRecord.get()); + if (hasNextBaseRecord(baseRecord, logRecordInfo)) { return true; } } // Handle records solely from log files. - if (logRecordIterator == null) { - logRecordIterator = records.values().iterator(); - } - - while (logRecordIterator.hasNext()) { - Pair, Map> nextRecordInfo = logRecordIterator.next(); - Option resultRecord; - resultRecord = merge(Option.empty(), Collections.emptyMap(), - nextRecordInfo.getLeft(), nextRecordInfo.getRight()); - if (resultRecord.isPresent()) { - nextRecord = readerContext.seal(resultRecord.get()); - return true; - } - } - return false; + return hasNextLogRecord(); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedFileGroupRecordBuffer.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedFileGroupRecordBuffer.java index 50e969343e15..29f05b015ed1 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedFileGroupRecordBuffer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedFileGroupRecordBuffer.java @@ -23,6 +23,7 @@ import org.apache.hudi.common.engine.HoodieReaderContext; import org.apache.hudi.common.model.DeleteRecord; import org.apache.hudi.common.model.HoodieRecordMerger; +import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.KeySpec; import org.apache.hudi.common.table.log.block.HoodieDataBlock; import org.apache.hudi.common.table.log.block.HoodieDeleteBlock; @@ -37,12 +38,12 @@ import java.io.IOException; import java.io.Serializable; import java.util.Arrays; -import java.util.Collections; import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.function.Function; import static org.apache.hudi.common.model.HoodieRecordMerger.DEFAULT_MERGER_STRATEGY_UUID; /** @@ -52,12 +53,12 @@ * {@link #hasNext} method is called. */ public class HoodiePositionBasedFileGroupRecordBuffer extends HoodieBaseFileGroupRecordBuffer { - private static final String ROW_INDEX_COLUMN_NAME = "row_index"; + public static final String ROW_INDEX_COLUMN_NAME = "row_index"; + public static final String ROW_INDEX_TEMPORARY_COLUMN_NAME = "_tmp_metadata_" + ROW_INDEX_COLUMN_NAME; private long nextRecordPosition = 0L; public HoodiePositionBasedFileGroupRecordBuffer(HoodieReaderContext readerContext, - Schema readerSchema, - Schema baseFileSchema, + HoodieTableMetaClient hoodieTableMetaClient, Option partitionNameOverrideOpt, Option partitionPathFieldOpt, HoodieRecordMerger recordMerger, @@ -66,7 +67,7 @@ public HoodiePositionBasedFileGroupRecordBuffer(HoodieReaderContext readerCon String spillableMapBasePath, ExternalSpillableMap.DiskMapType diskMapType, boolean isBitCaskDiskMapCompressionEnabled) { - super(readerContext, readerSchema, baseFileSchema, partitionNameOverrideOpt, partitionPathFieldOpt, + super(readerContext, hoodieTableMetaClient, partitionNameOverrideOpt, partitionPathFieldOpt, recordMerger, payloadProps, maxMemorySizeInBytes, spillableMapBasePath, diskMapType, isBitCaskDiskMapCompressionEnabled); } @@ -92,27 +93,30 @@ public void processDataBlock(HoodieDataBlock dataBlock, Option keySpecO // partial merging. enablePartialMerging = true; } - + // Extract positions from data block. List recordPositions = extractRecordPositions(dataBlock); + Pair, Schema> schemaTransformerWithEvolvedSchema = getSchemaTransformerWithEvolvedSchema(dataBlock); - // TODO: return an iterator that can generate sequence number with the record. - // Then we can hide this logic into data block. + // TODO: Return an iterator that can generate sequence number with the record. + // Then we can hide this logic into data block. try (ClosableIterator recordIterator = dataBlock.getEngineRecordIterator(readerContext)) { int recordIndex = 0; while (recordIterator.hasNext()) { T nextRecord = recordIterator.next(); // Skip a record if it is not contained in the specified keys. - if (shouldSkip(nextRecord, dataBlock.getKeyFieldName(), isFullKey, keys)) { + if (shouldSkip(nextRecord, dataBlock.getKeyFieldName(), isFullKey, keys, dataBlock.getSchema())) { recordIndex++; continue; } long recordPosition = recordPositions.get(recordIndex++); + + T evolvedNextRecord = schemaTransformerWithEvolvedSchema.getLeft().apply(nextRecord); processNextDataRecord( - nextRecord, - readerContext.generateMetadataForRecord(nextRecord, readerSchema), + evolvedNextRecord, + readerContext.generateMetadataForRecord(evolvedNextRecord, schemaTransformerWithEvolvedSchema.getRight()), recordPosition ); } @@ -178,35 +182,12 @@ protected boolean doHasNext() throws IOException { T baseRecord = baseFileIterator.next(); nextRecordPosition = readerContext.extractRecordPosition(baseRecord, readerSchema, ROW_INDEX_COLUMN_NAME, nextRecordPosition); Pair, Map> logRecordInfo = records.remove(nextRecordPosition++); - - Map metadata = readerContext.generateMetadataForRecord( - baseRecord, baseFileSchema); - - Option resultRecord = logRecordInfo != null - ? merge(Option.of(baseRecord), metadata, logRecordInfo.getLeft(), logRecordInfo.getRight()) - : merge(Option.empty(), Collections.emptyMap(), Option.of(baseRecord), metadata); - if (resultRecord.isPresent()) { - nextRecord = readerContext.seal(resultRecord.get()); + if (hasNextBaseRecord(baseRecord, logRecordInfo)) { return true; } } // Handle records solely from log files. - if (logRecordIterator == null) { - logRecordIterator = records.values().iterator(); - } - - while (logRecordIterator.hasNext()) { - Pair, Map> nextRecordInfo = logRecordIterator.next(); - Option resultRecord; - resultRecord = merge(Option.empty(), Collections.emptyMap(), - nextRecordInfo.getLeft(), nextRecordInfo.getRight()); - if (resultRecord.isPresent()) { - nextRecord = readerContext.seal(resultRecord.get()); - return true; - } - } - - return false; + return hasNextLogRecord(); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedSchemaHandler.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedSchemaHandler.java new file mode 100644 index 000000000000..9d34c1d7e332 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedSchemaHandler.java @@ -0,0 +1,71 @@ +/* + * 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.common.table.read; + +import org.apache.hudi.common.engine.HoodieReaderContext; +import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.internal.schema.InternalSchema; + +import org.apache.avro.Schema; + +import java.util.Collections; +import java.util.List; + +import static org.apache.hudi.avro.AvroSchemaUtils.appendFieldsToSchemaDedupNested; + +/** + * This class is responsible for handling the schema for the file group reader that supports positional merge. + */ +public class HoodiePositionBasedSchemaHandler extends HoodieFileGroupReaderSchemaHandler { + public HoodiePositionBasedSchemaHandler(HoodieReaderContext readerContext, + Schema dataSchema, + Schema requestedSchema, + Option internalSchemaOpt, + HoodieTableConfig hoodieTableConfig) { + super(readerContext, dataSchema, requestedSchema, internalSchemaOpt, hoodieTableConfig); + + } + + @Override + protected Schema prepareSchema() { + Schema preMergeSchema = super.prepareSchema(); + return readerContext.getHasLogFiles() + ? addPositionalMergeCol(preMergeSchema) + : preMergeSchema; + } + + private Schema addPositionalMergeCol(Schema input) { + return appendFieldsToSchemaDedupNested(input, Collections.singletonList(getPositionalMergeField())); + } + + private Schema.Field getPositionalMergeField() { + return new Schema.Field(HoodiePositionBasedFileGroupRecordBuffer.ROW_INDEX_TEMPORARY_COLUMN_NAME, + Schema.create(Schema.Type.LONG), "", -1L); + } + + @Override + public Schema createSchemaFromFields(List fields) { + if (readerContext.getHasLogFiles()) { + fields.add(getPositionalMergeField()); + } + return super.createSchemaFromFields(fields); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieUnmergedFileGroupRecordBuffer.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieUnmergedFileGroupRecordBuffer.java index 76aa28308c44..8338c96f5a0e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieUnmergedFileGroupRecordBuffer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieUnmergedFileGroupRecordBuffer.java @@ -23,6 +23,7 @@ import org.apache.hudi.common.engine.HoodieReaderContext; import org.apache.hudi.common.model.DeleteRecord; import org.apache.hudi.common.model.HoodieRecordMerger; +import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.KeySpec; import org.apache.hudi.common.table.log.block.HoodieDataBlock; import org.apache.hudi.common.table.log.block.HoodieDeleteBlock; @@ -48,8 +49,7 @@ public class HoodieUnmergedFileGroupRecordBuffer extends HoodieBaseFileGroupR public HoodieUnmergedFileGroupRecordBuffer( HoodieReaderContext readerContext, - Schema readerSchema, - Schema baseFileSchema, + HoodieTableMetaClient hoodieTableMetaClient, Option partitionNameOverrideOpt, Option partitionPathFieldOpt, HoodieRecordMerger recordMerger, @@ -58,8 +58,8 @@ public HoodieUnmergedFileGroupRecordBuffer( String spillableMapBasePath, ExternalSpillableMap.DiskMapType diskMapType, boolean isBitCaskDiskMapCompressionEnabled) { - super(readerContext, readerSchema, baseFileSchema, partitionNameOverrideOpt, partitionPathFieldOpt, - recordMerger, payloadProps, maxMemorySizeInBytes, spillableMapBasePath, diskMapType, isBitCaskDiskMapCompressionEnabled); + super(readerContext, hoodieTableMetaClient, partitionNameOverrideOpt, partitionPathFieldOpt, recordMerger, + payloadProps, maxMemorySizeInBytes, spillableMapBasePath, diskMapType, isBitCaskDiskMapCompressionEnabled); } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/action/InternalSchemaMerger.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/action/InternalSchemaMerger.java index 9ed55a7e5739..26c364c18004 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/action/InternalSchemaMerger.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/action/InternalSchemaMerger.java @@ -19,12 +19,15 @@ package org.apache.hudi.internal.schema.action; import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.internal.schema.InternalSchema; import org.apache.hudi.internal.schema.Type; import org.apache.hudi.internal.schema.Types; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; +import java.util.Map; /** * Auxiliary class. @@ -59,6 +62,8 @@ public class InternalSchemaMerger { // we can pass colNewName to reWriteRecordWithNewSchema directly, everything is ok. private boolean useColNameFromFileSchema = true; + private final Map renamedFields = new HashMap<>(); + public InternalSchemaMerger(InternalSchema fileSchema, InternalSchema querySchema, boolean ignoreRequiredAttribute, boolean useColumnTypeFromFileSchema, boolean useColNameFromFileSchema) { this.fileSchema = fileSchema; this.querySchema = querySchema; @@ -81,6 +86,15 @@ public InternalSchema mergeSchema() { return new InternalSchema(record); } + /** + * Create final read schema to read avro/parquet file. + * + * @return read schema to read avro/parquet file. + */ + public Pair> mergeSchemaGetRenamed() { + return Pair.of(mergeSchema(), renamedFields); + } + /** * Create final read schema to read avro/parquet file. * this is auxiliary function used by mergeSchema. @@ -150,6 +164,9 @@ private Types.Field dealWithRename(int fieldId, Type newType, Types.Field oldFie String nameFromQuerySchema = querySchema.findField(fieldId).name(); String finalFieldName = useColNameFromFileSchema ? nameFromFileSchema : nameFromQuerySchema; Type typeFromFileSchema = fieldFromFileSchema.type(); + if (!useColNameFromFileSchema) { + renamedFields.put(nameFromQuerySchema, nameFromFileSchema); + } // Current design mechanism guarantees nestedType change is not allowed, so no need to consider. if (newType.isNestedType()) { return Types.Field.get(oldField.fieldId(), oldField.isOptional(), diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java index d8533e7b4a1f..924917d5f515 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java @@ -18,12 +18,15 @@ package org.apache.hudi.internal.schema.convert; +import org.apache.hudi.avro.AvroSchemaUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.VisibleForTesting; import org.apache.hudi.exception.HoodieNullSchemaTypeException; import org.apache.hudi.internal.schema.HoodieSchemaException; import org.apache.hudi.internal.schema.InternalSchema; import org.apache.hudi.internal.schema.Type; import org.apache.hudi.internal.schema.Types; +import org.apache.hudi.internal.schema.utils.InternalSchemaUtils; import org.apache.avro.JsonProperties; import org.apache.avro.LogicalType; @@ -42,6 +45,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; +import static org.apache.avro.Schema.Type.STRING; import static org.apache.avro.Schema.Type.UNION; /** @@ -73,6 +77,75 @@ public static Schema convert(InternalSchema internalSchema, String name) { return buildAvroSchemaFromInternalSchema(internalSchema, name); } + public static InternalSchema pruneAvroSchemaToInternalSchema(Schema schema, InternalSchema originSchema) { + List pruneNames = collectColNamesFromSchema(schema); + return InternalSchemaUtils.pruneInternalSchema(originSchema, pruneNames); + } + + /** + * Collect all the leaf nodes names. + * + * @param schema a avro schema. + * @return leaf nodes full names. + */ + @VisibleForTesting + static List collectColNamesFromSchema(Schema schema) { + List result = new ArrayList<>(); + Deque visited = new LinkedList<>(); + collectColNamesFromAvroSchema(schema, visited, result); + return result; + } + + private static void collectColNamesFromAvroSchema(Schema schema, Deque visited, List resultSet) { + switch (schema.getType()) { + case RECORD: + List fields = schema.getFields(); + for (Schema.Field f : fields) { + visited.push(f.name()); + collectColNamesFromAvroSchema(f.schema(), visited, resultSet); + visited.pop(); + addFullNameIfLeafNode(f.schema(), f.name(), visited, resultSet); + } + return; + + case UNION: + collectColNamesFromAvroSchema(AvroSchemaUtils.resolveNullableSchema(schema), visited, resultSet); + return; + + case ARRAY: + visited.push("element"); + collectColNamesFromAvroSchema(schema.getElementType(), visited, resultSet); + visited.pop(); + addFullNameIfLeafNode(schema.getElementType(), "element", visited, resultSet); + return; + + case MAP: + addFullNameIfLeafNode(STRING, "key", visited, resultSet); + visited.push("value"); + collectColNamesFromAvroSchema(schema.getValueType(), visited, resultSet); + visited.pop(); + addFullNameIfLeafNode(schema.getValueType(), "value", visited, resultSet); + return; + + default: + } + } + + private static void addFullNameIfLeafNode(Schema schema, String name, Deque visited, List resultSet) { + addFullNameIfLeafNode(AvroSchemaUtils.resolveNullableSchema(schema).getType(), name, visited, resultSet); + } + + private static void addFullNameIfLeafNode(Schema.Type type, String name, Deque visited, List resultSet) { + switch (type) { + case RECORD: + case ARRAY: + case MAP: + return; + default: + resultSet.add(InternalSchemaUtils.createFullName(name, visited)); + } + } + /** * Converting from avro -> internal schema -> avro * causes null to always be first in unions. @@ -504,7 +577,7 @@ private static Schema visitInternalPrimitiveToBuildAvroPrimitiveType(Type.Primit return LogicalTypes.decimal(decimal.precision(), decimal.scale()) .addToSchema(fixedSchema); } - + default: throw new UnsupportedOperationException( "Unsupported type ID: " + primitive.typeId()); diff --git a/hudi-common/src/test/java/org/apache/hudi/avro/AvroSchemaTestUtils.java b/hudi-common/src/test/java/org/apache/hudi/avro/AvroSchemaTestUtils.java new file mode 100644 index 000000000000..2580e1e8833c --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/avro/AvroSchemaTestUtils.java @@ -0,0 +1,67 @@ +/* + * 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.avro; + +import org.apache.avro.JsonProperties; +import org.apache.avro.Schema; + +import java.util.Arrays; + +public class AvroSchemaTestUtils { + public static Schema.Field createNestedField(String name, Schema.Type type) { + return createNestedField(name, Schema.create(type)); + } + + public static Schema.Field createNestedField(String name, Schema schema) { + return new Schema.Field(name, createRecord(name, new Schema.Field("nested", schema, null, null)), null, null); + } + + public static Schema.Field createArrayField(String name, Schema.Type type) { + return createArrayField(name, Schema.create(type)); + } + + public static Schema.Field createArrayField(String name, Schema schema) { + return new Schema.Field(name, Schema.createArray(schema), null, null); + } + + public static Schema.Field createMapField(String name, Schema.Type type) { + return createMapField(name, Schema.create(type)); + } + + public static Schema.Field createMapField(String name, Schema schema) { + return new Schema.Field(name, Schema.createMap(schema), null, null); + } + + public static Schema.Field createPrimitiveField(String name, Schema.Type type) { + return new Schema.Field(name, Schema.create(type), null, null); + } + + public static Schema.Field createNullablePrimitiveField(String name, Schema.Type type) { + return new Schema.Field(name, AvroSchemaUtils.createNullableSchema(type), null, JsonProperties.NULL_VALUE); + } + + public static Schema createRecord(String name, Schema.Field... fields) { + return Schema.createRecord(name, null, null, false, Arrays.asList(fields)); + } + + public static Schema createNullableRecord(String name, Schema.Field... fields) { + return AvroSchemaUtils.createNullableSchema(Schema.createRecord(name, null, null, false, Arrays.asList(fields))); + } +} diff --git a/hudi-common/src/test/java/org/apache/hudi/avro/TestAvroSchemaUtils.java b/hudi-common/src/test/java/org/apache/hudi/avro/TestAvroSchemaUtils.java index 1417f2f67dcc..37cea0d93a91 100644 --- a/hudi-common/src/test/java/org/apache/hudi/avro/TestAvroSchemaUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/avro/TestAvroSchemaUtils.java @@ -18,6 +18,7 @@ package org.apache.hudi.avro; +import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.SchemaBackwardsCompatibilityException; import org.apache.hudi.exception.SchemaCompatibilityException; @@ -28,6 +29,7 @@ import java.util.Collections; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -256,10 +258,9 @@ public void testBrokenSchema() { () -> AvroSchemaUtils.checkSchemaCompatible(FULL_SCHEMA, BROKEN_SCHEMA, true, false, Collections.emptySet())); } - /* [HUDI-7045] should uncomment this test @Test public void testAppendFieldsToSchemaDedupNested() { - Schema full_schema = new Schema.Parser().parse("{\n" + Schema fullSchema = new Schema.Parser().parse("{\n" + " \"type\": \"record\",\n" + " \"namespace\": \"example.schema\",\n" + " \"name\": \"source\",\n" @@ -292,7 +293,7 @@ public void testAppendFieldsToSchemaDedupNested() { + " ]\n" + "}\n"); - Schema missing_field_schema = new Schema.Parser().parse("{\n" + Schema missingFieldSchema = new Schema.Parser().parse("{\n" + " \"type\": \"record\",\n" + " \"namespace\": \"example.schema\",\n" + " \"name\": \"source\",\n" @@ -321,9 +322,8 @@ public void testAppendFieldsToSchemaDedupNested() { + " ]\n" + "}\n"); - Option missingField = AvroSchemaUtils.findNestedField(full_schema, "nested_record.long"); - assertTrue(missingField.isPresent()); - assertEquals(full_schema, AvroSchemaUtils.appendFieldsToSchemaDedupNested(missing_field_schema, Collections.singletonList(missingField.get()))); + Option missingField = AvroSchemaUtils.findNestedField(fullSchema, "nested_record.long"); + assertTrue(missingField.isPresent()); + assertEquals(fullSchema, AvroSchemaUtils.appendFieldsToSchemaDedupNested(missingFieldSchema, Collections.singletonList(missingField.get()))); } - */ } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderBase.java b/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderBase.java index ec3457801f76..a8a95887c180 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderBase.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderBase.java @@ -35,6 +35,7 @@ import org.apache.hudi.common.table.view.SyncableFileSystemView; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestUtils; +import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.ExternalSpillableMap; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.hudi.metadata.HoodieTableMetadata; @@ -69,7 +70,7 @@ public abstract class TestHoodieFileGroupReaderBase { public abstract String getBasePath(); - public abstract HoodieReaderContext getHoodieReaderContext(String tablePath, Schema avroSchema); + public abstract HoodieReaderContext getHoodieReaderContext(String tablePath, Schema avroSchema, StorageConfiguration storageConf); public abstract void commitToTable(List recordList, String operation, Map writeConfigs); @@ -169,13 +170,15 @@ private void validateOutputFromFileGroupReader(StorageConfiguration storageCo } assertEquals(containsBaseFile, fileSlice.getBaseFile().isPresent()); HoodieFileGroupReader fileGroupReader = new HoodieFileGroupReader<>( - getHoodieReaderContext(tablePath, avroSchema), + getHoodieReaderContext(tablePath, avroSchema, storageConf), metaClient.getStorage(), tablePath, metaClient.getActiveTimeline().lastInstant().get().getTimestamp(), fileSlice, avroSchema, avroSchema, + Option.empty(), + metaClient, props, metaClient.getTableConfig(), 0, diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieFileGroupReaderTestUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieFileGroupReaderTestUtils.java index 49acf352628e..944722e45f56 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieFileGroupReaderTestUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieFileGroupReaderTestUtils.java @@ -44,7 +44,8 @@ public static HoodieFileGroupReader createFileGroupReader( TypedProperties properties, HoodieStorage storage, HoodieTableConfig tableConfig, - HoodieReaderContext readerContext + HoodieReaderContext readerContext, + HoodieTableMetaClient metaClient ) { assert (fileSliceOpt.isPresent()); return new HoodieFileGroupReaderBuilder() @@ -55,7 +56,7 @@ public static HoodieFileGroupReader createFileGroupReader( .withLength(length) .withProperties(properties) .withTableConfig(tableConfig) - .build(basePath, latestCommitTime, schema, shouldUseRecordPosition); + .build(basePath, latestCommitTime, schema, shouldUseRecordPosition, metaClient); } public static class HoodieFileGroupReaderBuilder { @@ -109,7 +110,8 @@ public HoodieFileGroupReader build( String basePath, String latestCommitTime, Schema schema, - boolean shouldUseRecordPosition + boolean shouldUseRecordPosition, + HoodieTableMetaClient metaClient ) { return new HoodieFileGroupReader<>( readerContext, @@ -119,6 +121,8 @@ public HoodieFileGroupReader build( fileSlice, schema, schema, + Option.empty(), + metaClient, props, tableConfig, start, diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieTestReaderContext.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieTestReaderContext.java index 083738827735..99dcd873fe7e 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieTestReaderContext.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieTestReaderContext.java @@ -169,14 +169,18 @@ public IndexedRecord seal(IndexedRecord record) { } @Override - public ClosableIterator mergeBootstrapReaders( - ClosableIterator skeletonFileIterator, - ClosableIterator dataFileIterator) { + public ClosableIterator mergeBootstrapReaders(ClosableIterator skeletonFileIterator, + Schema skeletonRequiredSchema, + ClosableIterator dataFileIterator, + Schema dataRequiredSchema) { return null; } @Override - public UnaryOperator projectRecord(Schema from, Schema to) { + public UnaryOperator projectRecord(Schema from, Schema to, Map renamedColumns) { + if (!renamedColumns.isEmpty()) { + throw new UnsupportedOperationException("Schema evolution is not supported for the test reader context"); + } Map fromFields = IntStream.range(0, from.getFields().size()) .boxed() .collect(Collectors.toMap( diff --git a/hudi-common/src/test/java/org/apache/hudi/internal/schema/action/TestMergeSchema.java b/hudi-common/src/test/java/org/apache/hudi/internal/schema/action/TestMergeSchema.java index 5240179fb8c5..6b6bdf2d69ae 100644 --- a/hudi-common/src/test/java/org/apache/hudi/internal/schema/action/TestMergeSchema.java +++ b/hudi-common/src/test/java/org/apache/hudi/internal/schema/action/TestMergeSchema.java @@ -18,6 +18,7 @@ package org.apache.hudi.internal.schema.action; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.internal.schema.InternalSchema; import org.apache.hudi.internal.schema.Types; import org.apache.hudi.internal.schema.utils.SchemaChangeUtils; @@ -25,6 +26,7 @@ import org.junit.jupiter.api.Test; import java.util.Arrays; +import java.util.Map; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -58,8 +60,8 @@ public void testPrimitiveMerge() { TableChanges.ColumnUpdateChange updateChange = TableChanges.ColumnUpdateChange.get(newDeleteSchema); updateChange.updateColumnType("col2", Types.LongType.get()) .updateColumnComment("col2", "alter col2 comments") - .renameColumn("col2", "colx").addPositionChange("col2", - "col4", "after"); + .renameColumn("col2", "colx") // rename col2 to colx to assert later on renamed field + .addPositionChange("col2", "col4", "after"); InternalSchema updateSchema = SchemaChangeUtils.applyTableChanges2Schema(newDeleteSchema, updateChange); // add col1 again @@ -67,14 +69,17 @@ public void testPrimitiveMerge() { addChange1.addColumns("col1", Types.BooleanType.get(), "add new col1"); InternalSchema finalSchema = SchemaChangeUtils.applyTableChanges2Schema(updateSchema, addChange1); // merge schema by using columnType from query schema - InternalSchema mergeSchema = new InternalSchemaMerger(oldSchema, finalSchema, true, false).mergeSchema(); - + Pair> mergeSchemaWithRenamedField = new InternalSchemaMerger(oldSchema, finalSchema, true, false, false).mergeSchemaGetRenamed(); + InternalSchema mergeSchema = mergeSchemaWithRenamedField.getLeft(); + assertEquals("col2", mergeSchemaWithRenamedField.getRight().get("colx")); InternalSchema checkedSchema = new InternalSchema(Types.RecordType.get(Arrays.asList( Types.Field.get(4, true, "c1", Types.BooleanType.get(), "add c1 after col1"), Types.Field.get(5, true, "c2", Types.IntType.get(), "add c2 before col3"), Types.Field.get(3, true, "col4", Types.FloatType.get()), Types.Field.get(1, true, "col2", Types.LongType.get(), "alter col2 comments"), Types.Field.get(6, true, "col1suffix", Types.BooleanType.get(), "add new col1")))); + // merged schema without renamed fields + mergeSchema = new InternalSchemaMerger(oldSchema, finalSchema, true, false).mergeSchema(); assertEquals(mergeSchema, checkedSchema); // merge schema by using columnType from file schema diff --git a/hudi-common/src/test/java/org/apache/hudi/internal/schema/convert/TestAvroInternalSchemaConverter.java b/hudi-common/src/test/java/org/apache/hudi/internal/schema/convert/TestAvroInternalSchemaConverter.java new file mode 100644 index 000000000000..8bf7eabce029 --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/internal/schema/convert/TestAvroInternalSchemaConverter.java @@ -0,0 +1,112 @@ +/* + * 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.internal.schema.convert; + +import org.apache.hudi.avro.AvroSchemaTestUtils; + +import org.apache.avro.Schema; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.List; + +import static org.apache.hudi.avro.AvroSchemaTestUtils.createArrayField; +import static org.apache.hudi.avro.AvroSchemaTestUtils.createMapField; +import static org.apache.hudi.avro.AvroSchemaTestUtils.createNullablePrimitiveField; +import static org.apache.hudi.avro.AvroSchemaTestUtils.createNullableRecord; +import static org.apache.hudi.avro.AvroSchemaTestUtils.createPrimitiveField; +import static org.apache.hudi.avro.AvroSchemaTestUtils.createRecord; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class TestAvroInternalSchemaConverter { + + public static Schema getSimpleSchema() { + return createRecord("simpleSchema", + createPrimitiveField("field1", Schema.Type.INT), + createPrimitiveField("field2", Schema.Type.STRING)); + } + + public static List getSimpleSchemaExpectedColumnNames() { + return Arrays.asList("field1", "field2"); + } + + public static Schema getSimpleSchemaWithNullable() { + return createRecord("simpleSchemaWithNullable", + createNullablePrimitiveField("field1", Schema.Type.INT), + createPrimitiveField("field2", Schema.Type.STRING)); + } + + public static Schema getComplexSchemaSingleLevel() { + return createRecord("complexSchemaSingleLevel", + AvroSchemaTestUtils.createNestedField("field1", Schema.Type.INT), + createArrayField("field2", Schema.Type.STRING), + createMapField("field3", Schema.Type.DOUBLE)); + } + + public static List getComplexSchemaSingleLevelExpectedColumnNames() { + return Arrays.asList("field1.nested", "field2.element", "field3.key", "field3.value"); + } + + public static Schema getDeeplyNestedFieldSchema() { + return createRecord("deeplyNestedFieldSchema", + createPrimitiveField("field1", Schema.Type.INT), + new Schema.Field("field2", + createRecord("field2nest", + createArrayField("field2nestarray", + createNullableRecord("field2nestarraynest", + createNullablePrimitiveField("field21", Schema.Type.INT), + createNullablePrimitiveField("field22", Schema.Type.INT)))), null, null), + createNullablePrimitiveField("field3", Schema.Type.INT)); + } + + public static List getDeeplyNestedFieldSchemaExpectedColumnNames() { + return Arrays.asList("field1", "field2.field2nestarray.element.field21", + "field2.field2nestarray.element.field22", "field3"); + } + + @Test + public void testCollectColumnNames() { + Schema simpleSchema = getSimpleSchema(); + List fieldNames = AvroInternalSchemaConverter.collectColNamesFromSchema(simpleSchema); + List expectedOutput = getSimpleSchemaExpectedColumnNames(); + assertEquals(expectedOutput.size(), fieldNames.size()); + assertTrue(fieldNames.containsAll(expectedOutput)); + + + Schema simpleSchemaWithNullable = getSimpleSchemaWithNullable(); + fieldNames = AvroInternalSchemaConverter.collectColNamesFromSchema(simpleSchemaWithNullable); + expectedOutput = getSimpleSchemaExpectedColumnNames(); + assertEquals(expectedOutput.size(), fieldNames.size()); + assertTrue(fieldNames.containsAll(expectedOutput)); + + Schema complexSchemaSingleLevel = getComplexSchemaSingleLevel(); + fieldNames = AvroInternalSchemaConverter.collectColNamesFromSchema(complexSchemaSingleLevel); + expectedOutput = getComplexSchemaSingleLevelExpectedColumnNames(); + assertEquals(expectedOutput.size(), fieldNames.size()); + assertTrue(fieldNames.containsAll(expectedOutput)); + + Schema deeplyNestedFieldSchema = getDeeplyNestedFieldSchema(); + fieldNames = AvroInternalSchemaConverter.collectColNamesFromSchema(deeplyNestedFieldSchema); + expectedOutput = getDeeplyNestedFieldSchemaExpectedColumnNames(); + assertEquals(expectedOutput.size(), fieldNames.size()); + assertTrue(fieldNames.containsAll(expectedOutput)); + } +} diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieFileGroupReaderTestHarness.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieFileGroupReaderTestHarness.java index 5d50f885204f..a2ef6ae39068 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieFileGroupReaderTestHarness.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieFileGroupReaderTestHarness.java @@ -120,7 +120,8 @@ protected ClosableIterator getFileGroupIterator(int numFiles) properties, new HoodieHadoopStorage(basePath, storageConf), tableConfig, - readerContext + readerContext, + metaClient ); fileGroupReader.initRecordIterators(); diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala index 97a6016f8736..246f20edda0a 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala @@ -269,7 +269,6 @@ object DefaultSource { val useNewParquetFileFormat = parameters.getOrElse(HoodieReaderConfig.FILE_GROUP_READER_ENABLED.key(), HoodieReaderConfig.FILE_GROUP_READER_ENABLED.defaultValue().toString).toBoolean && !metaClient.isMetadataTable && (globPaths == null || globPaths.isEmpty) && - !parameters.getOrElse(SCHEMA_EVOLUTION_ENABLED.key(), SCHEMA_EVOLUTION_ENABLED.defaultValue().toString).toBoolean && parameters.getOrElse(REALTIME_MERGE.key(), REALTIME_MERGE.defaultValue()).equalsIgnoreCase(REALTIME_PAYLOAD_COMBINE_OPT_VAL) if (metaClient.getCommitsTimeline.filterCompletedInstants.countInstants() == 0) { new EmptyRelation(sqlContext, resolveSchema(metaClient, parameters, Some(schema))) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieHadoopFsRelationFactory.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieHadoopFsRelationFactory.scala index e326da3c4a8a..be1f234f76c7 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieHadoopFsRelationFactory.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieHadoopFsRelationFactory.scala @@ -26,11 +26,11 @@ import org.apache.hudi.common.config.HoodieMetadataConfig.{DEFAULT_METADATA_ENAB import org.apache.hudi.common.model.HoodieRecord import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, TableSchemaResolver} import org.apache.hudi.common.table.timeline.HoodieTimeline -import org.apache.hudi.common.util.{ConfigUtils, StringUtils} +import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, TableSchemaResolver} import org.apache.hudi.common.util.ValidationUtils.checkState -import org.apache.hudi.config.HoodieBootstrapConfig.DATA_QUERIES_ONLY +import org.apache.hudi.common.util.{ConfigUtils, StringUtils} import org.apache.hudi.config.HoodieWriteConfig -import org.apache.hudi.internal.schema.{HoodieSchemaException, InternalSchema} +import org.apache.hudi.internal.schema.InternalSchema import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter import org.apache.hudi.metadata.HoodieTableMetadataUtil import org.apache.hudi.storage.StoragePath @@ -109,6 +109,12 @@ abstract class HoodieBaseHadoopFsRelationFactory(val sqlContext: SQLContext, (avroSchema, internalSchemaOpt) } + protected lazy val validCommits: String = if (internalSchemaOpt.nonEmpty) { + timeline.getInstants.iterator.asScala.map(_.getFileName).mkString(",") + } else { + "" + } + protected lazy val tableStructSchema: StructType = { val converted = AvroConversionUtils.convertAvroSchemaToStructType(tableAvroSchema) val metaFieldMetadata = sparkAdapter.createCatalystMetadataForMetaField @@ -161,15 +167,14 @@ abstract class HoodieBaseHadoopFsRelationFactory(val sqlContext: SQLContext, val shouldExtractPartitionValueFromPath = optParams.getOrElse(DataSourceReadOptions.EXTRACT_PARTITION_VALUES_FROM_PARTITION_PATH.key, DataSourceReadOptions.EXTRACT_PARTITION_VALUES_FROM_PARTITION_PATH.defaultValue.toString).toBoolean - val shouldUseBootstrapFastRead = optParams.getOrElse(DATA_QUERIES_ONLY.key(), "false").toBoolean - - shouldOmitPartitionColumns || shouldExtractPartitionValueFromPath || shouldUseBootstrapFastRead + shouldOmitPartitionColumns || shouldExtractPartitionValueFromPath } protected lazy val mandatoryFieldsForMerging: Seq[String] = Seq(recordKeyField) ++ preCombineFieldOpt.map(Seq(_)).getOrElse(Seq()) - protected lazy val shouldUseRecordPosition: Boolean = checkIfAConfigurationEnabled(HoodieReaderConfig.MERGE_USE_RECORD_POSITIONS) + //feature added in spark 3.5 + protected lazy val shouldUseRecordPosition: Boolean = checkIfAConfigurationEnabled(HoodieReaderConfig.MERGE_USE_RECORD_POSITIONS) && HoodieSparkUtils.gteqSpark3_5 protected def queryTimestamp: Option[String] = specifiedQueryTimestamp.orElse(toScalaOption(timeline.lastInstant()).map(_.getTimestamp)) @@ -239,8 +244,8 @@ class HoodieMergeOnReadSnapshotHadoopFsRelationFactory(override val sqlContext: } else { new HoodieFileGroupReaderBasedParquetFileFormat( tableState, HoodieTableSchema(tableStructSchema, tableAvroSchema.toString, internalSchemaOpt), - metaClient.getTableConfig.getTableName, mergeType, mandatoryFields, - true, isBootstrap, false, shouldUseRecordPosition, Seq.empty) + metaClient.getTableConfig.getTableName, mergeType, mandatoryFields, true, isBootstrap, + false, fileIndex.isInstanceOf[HoodieCDCFileIndex], validCommits, shouldUseRecordPosition, Seq.empty) } } @@ -284,7 +289,8 @@ class HoodieMergeOnReadIncrementalHadoopFsRelationFactory(override val sqlContex new HoodieFileGroupReaderBasedParquetFileFormat( tableState, HoodieTableSchema(tableStructSchema, tableAvroSchema.toString, internalSchemaOpt), metaClient.getTableConfig.getTableName, mergeType, mandatoryFields, - true, isBootstrap, true, shouldUseRecordPosition, fileIndex.getRequiredFilters) + true, isBootstrap, true, fileIndex.isInstanceOf[HoodieCDCFileIndex], + validCommits, shouldUseRecordPosition, fileIndex.getRequiredFilters) } } } @@ -315,7 +321,8 @@ class HoodieCopyOnWriteSnapshotHadoopFsRelationFactory(override val sqlContext: new HoodieFileGroupReaderBasedParquetFileFormat( tableState, HoodieTableSchema(tableStructSchema, tableAvroSchema.toString, internalSchemaOpt), metaClient.getTableConfig.getTableName, mergeType, mandatoryFields, - false, isBootstrap, false, shouldUseRecordPosition, Seq.empty) + false, isBootstrap, false, fileIndex.isInstanceOf[HoodieCDCFileIndex], validCommits, + shouldUseRecordPosition, Seq.empty) } } } @@ -343,7 +350,8 @@ class HoodieCopyOnWriteIncrementalHadoopFsRelationFactory(override val sqlContex new HoodieFileGroupReaderBasedParquetFileFormat( tableState, HoodieTableSchema(tableStructSchema, tableAvroSchema.toString, internalSchemaOpt), metaClient.getTableConfig.getTableName, mergeType, mandatoryFields, - false, isBootstrap, true, shouldUseRecordPosition, fileIndex.getRequiredFilters) + false, isBootstrap, true, fileIndex.isInstanceOf[HoodieCDCFileIndex], + validCommits, shouldUseRecordPosition, fileIndex.getRequiredFilters) } } } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/CDCFileGroupIterator.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/CDCFileGroupIterator.scala index 6d11b9152b53..60758e840c9d 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/CDCFileGroupIterator.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/CDCFileGroupIterator.scala @@ -19,11 +19,14 @@ package org.apache.hudi.cdc -import org.apache.hudi.HoodieBaseRelation.BaseFileReader +import org.apache.avro.Schema +import org.apache.avro.generic.{GenericData, GenericRecord, IndexedRecord} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path import org.apache.hudi.HoodieConversionUtils.toScalaOption import org.apache.hudi.HoodieDataSourceHelper.AvroDeserializerSupport import org.apache.hudi.avro.HoodieAvroUtils -import org.apache.hudi.common.config.{HoodieMetadataConfig, TypedProperties} +import org.apache.hudi.common.config.{HoodieCommonConfig, HoodieMemoryConfig, HoodieMetadataConfig, TypedProperties} import org.apache.hudi.common.model.{FileSlice, HoodieAvroRecordMerger, HoodieLogFile, HoodieRecord, HoodieRecordMerger, HoodieRecordPayload} import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.common.table.cdc.HoodieCDCInferenceCase._ @@ -31,28 +34,25 @@ import org.apache.hudi.common.table.cdc.HoodieCDCOperation._ import org.apache.hudi.common.table.cdc.HoodieCDCSupplementalLoggingMode._ import org.apache.hudi.common.table.cdc.{HoodieCDCFileSplit, HoodieCDCUtils} import org.apache.hudi.common.table.log.HoodieCDCLogRecordIterator +import org.apache.hudi.common.table.read.HoodieFileGroupReader +import org.apache.hudi.common.util.FileIOUtils import org.apache.hudi.common.util.ValidationUtils.checkState +import org.apache.hudi.common.util.collection.ExternalSpillableMap.DiskMapType import org.apache.hudi.config.HoodiePayloadConfig import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory import org.apache.hudi.storage.{StorageConfiguration, StoragePath} -import org.apache.hudi.{AvroConversionUtils, AvroProjection, HoodieMergeOnReadFileSplit, HoodieTableSchema, HoodieTableState, LogFileIterator, RecordMergingFileIterator, SparkAdapterSupport} - -import org.apache.avro.Schema -import org.apache.avro.generic.{GenericData, GenericRecord, IndexedRecord} -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path +import org.apache.hudi.{AvroConversionUtils, AvroProjection, HoodieMergeOnReadFileSplit, HoodieTableSchema, HoodieTableState, LogFileIterator, SparkAdapterSupport, SparkFileFormatInternalRowReaderContext} import org.apache.spark.sql.HoodieCatalystExpressionUtils.generateUnsafeProjection import org.apache.spark.sql.avro.HoodieAvroDeserializer import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Projection -import org.apache.spark.sql.execution.datasources.PartitionedFile +import org.apache.spark.sql.execution.datasources.parquet.SparkParquetReader import org.apache.spark.sql.types.StructType import org.apache.spark.unsafe.types.UTF8String import java.io.Closeable -import java.util.Properties import java.util.stream.Collectors - +import java.util.{Locale, Properties} import scala.annotation.tailrec import scala.collection.JavaConverters._ import scala.collection.mutable @@ -60,11 +60,12 @@ import scala.collection.mutable class CDCFileGroupIterator(split: HoodieCDCFileGroupSplit, metaClient: HoodieTableMetaClient, conf: StorageConfiguration[_], - parquetReader: PartitionedFile => Iterator[InternalRow], + sparkParquetReader: SparkParquetReader, originTableSchema: HoodieTableSchema, cdcSchema: StructType, requiredCdcSchema: StructType, - props: TypedProperties) + props: TypedProperties, + options: Map[String, String]) extends Iterator[InternalRow] with SparkAdapterSupport with AvroDeserializerSupport with Closeable { @@ -376,7 +377,8 @@ class CDCFileGroupIterator(split: HoodieCDCFileGroupSplit, val pf = sparkPartitionedFileUtils.createPartitionedFile( InternalRow.empty, absCDCPath, 0, fileStatus.getLength) - recordIter = parquetReader(pf) + recordIter = sparkParquetReader.read(pf, structTypeSchema, new StructType(), Seq.empty, + storage.getConf.asInstanceOf[StorageConfiguration[Configuration]]) case BASE_FILE_DELETE => assert(currentCDCFileSplit.getBeforeFileSlice.isPresent) recordIter = loadFileSlice(currentCDCFileSplit.getBeforeFileSlice.get) @@ -479,32 +481,35 @@ class CDCFileGroupIterator(split: HoodieCDCFileGroupSplit, } private def loadFileSlice(fileSlice: FileSlice): Iterator[InternalRow] = { - val baseFileInfo = storage.getPathInfo(fileSlice.getBaseFile.get().getStoragePath) - val basePartitionedFile = sparkPartitionedFileUtils.createPartitionedFile( - InternalRow.empty, - baseFileInfo.getPath, + val readerContext = new SparkFileFormatInternalRowReaderContext(sparkParquetReader, recordKeyField, Seq.empty) + val reader = new HoodieFileGroupReader[InternalRow]( + readerContext, + storage, + tableState.tablePath, + tableState.latestCommitTimestamp.get, + fileSlice, + avroSchema, + avroSchema, + org.apache.hudi.common.util.Option.empty(), + metaClient, + metaClient.getTableConfig.getProps, + metaClient.getTableConfig, 0, - baseFileInfo.getLength - ) - val logFiles = fileSlice.getLogFiles - .sorted(HoodieLogFile.getLogFileComparator) - .collect(Collectors.toList[HoodieLogFile]) - .asScala.toList - .filterNot(_.getFileName.endsWith(HoodieCDCUtils.CDC_LOGFILE_SUFFIX)) - - if (logFiles.isEmpty) { - // no log files, just load the base parquet file - parquetReader(basePartitionedFile) - } else { - // use [[RecordMergingFileIterator]] to load both the base file and log files - val morSplit = HoodieMergeOnReadFileSplit(Some(basePartitionedFile), logFiles) - new RecordMergingFileIterator( - morSplit, - BaseFileReader(parquetReader, originTableSchema.structTypeSchema), - originTableSchema, - originTableSchema, - tableState, - conf.unwrapAs(classOf[Configuration])) + storage.getPathInfo(fileSlice.getBaseFile.get().getStoragePath).getLength, + false, + options.getOrElse(HoodieMemoryConfig.MAX_MEMORY_FOR_MERGE.key(), HoodieMemoryConfig.MAX_MEMORY_FOR_MERGE.defaultValue() + "").toLong, + options.getOrElse(HoodieMemoryConfig.SPILLABLE_MAP_BASE_PATH.key(), FileIOUtils.getDefaultSpillableMapBasePath), + DiskMapType.valueOf(options.getOrElse(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.key(), HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.defaultValue().name()).toUpperCase(Locale.ROOT)), + options.getOrElse(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.key(), HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.defaultValue().toString).toBoolean) + reader.initRecordIterators() + val iter = reader.getClosableIterator + + new Iterator[InternalRow] with Closeable { + override def hasNext: Boolean = iter.hasNext + + override def next(): InternalRow = iter.next() + + override def close(): Unit = iter.close() } } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala index 9f46cf3b3b6e..3dde68fb35de 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala @@ -17,41 +17,35 @@ package org.apache.spark.sql.execution.datasources.parquet +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path import org.apache.hudi.MergeOnReadSnapshotRelation.createPartitionedFile import org.apache.hudi.avro.AvroSchemaUtils import org.apache.hudi.cdc.{CDCFileGroupIterator, CDCRelation, HoodieCDCFileGroupSplit} +import org.apache.hudi.client.utils.SparkInternalSchemaConverter import org.apache.hudi.common.config.{HoodieCommonConfig, HoodieMemoryConfig, TypedProperties} -import org.apache.hudi.common.engine.HoodieReaderContext import org.apache.hudi.common.fs.FSUtils -import org.apache.hudi.common.model.{FileSlice, HoodieLogFile, HoodieRecord} import org.apache.hudi.common.table.read.HoodieFileGroupReader import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} import org.apache.hudi.common.util.FileIOUtils import org.apache.hudi.common.util.collection.ExternalSpillableMap.DiskMapType +import org.apache.hudi.internal.schema.InternalSchema +import org.apache.hudi.internal.schema.utils.SerDeHelper import org.apache.hudi.storage.StorageConfiguration import org.apache.hudi.storage.hadoop.{HadoopStorageConfiguration, HoodieHadoopStorage} -import org.apache.hudi.{AvroConversionUtils, HoodieFileIndex, HoodiePartitionCDCFileGroupMapping, HoodiePartitionFileSliceMapping, HoodieSparkUtils, HoodieTableSchema, HoodieTableState, SparkAdapterSupport, SparkFileFormatInternalRowReaderContext} - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path +import org.apache.hudi.{AvroConversionUtils, HoodieFileIndex, HoodiePartitionCDCFileGroupMapping, HoodiePartitionFileSliceMapping, HoodieTableSchema, HoodieTableState, SparkAdapterSupport, SparkFileFormatInternalRowReaderContext} import org.apache.spark.sql.HoodieCatalystExpressionUtils.generateUnsafeProjection import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.JoinedRow import org.apache.spark.sql.execution.datasources.PartitionedFile -import org.apache.spark.sql.execution.datasources.parquet.HoodieFileGroupReaderBasedParquetFileFormat.{OPTION_RETURNING_BATCH, ROW_INDEX_TEMPORARY_COLUMN_NAME, getAppliedFilters, getAppliedRequiredSchema, getRecordKeyRelatedFilters, makeCloseableFileGroupMappingRecordIterator} -import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.isMetaField +import org.apache.spark.sql.internal.SQLConf.PARQUET_VECTORIZED_READER_ENABLED import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.types.{LongType, Metadata, MetadataBuilder, StringType, StructField, StructType} -import org.apache.spark.util.SerializableConfiguration +import org.apache.spark.sql.types.StructType import java.io.Closeable import java.util.Locale -import scala.annotation.tailrec -import scala.collection.JavaConverters._ -import scala.collection.mutable - trait HoodieFormatTrait { // Used so that the planner only projects once and does not stack overflow @@ -71,12 +65,16 @@ class HoodieFileGroupReaderBasedParquetFileFormat(tableState: HoodieTableState, isMOR: Boolean, isBootstrap: Boolean, isIncremental: Boolean, + isCDC: Boolean, + validCommits: String, shouldUseRecordPosition: Boolean, requiredFilters: Seq[Filter] - ) extends ParquetFileFormat with SparkAdapterSupport with HoodieFormatTrait { + ) extends ParquetFileFormat with SparkAdapterSupport with HoodieFormatTrait { def getRequiredFilters: Seq[Filter] = requiredFilters + private val sanitizedTableName = AvroSchemaUtils.getAvroRecordQualifiedName(tableName) + /** * Support batch needs to remain consistent, even if one side of a bootstrap merge can support * while the other side can't @@ -84,15 +82,21 @@ class HoodieFileGroupReaderBasedParquetFileFormat(tableState: HoodieTableState, private var supportBatchCalled = false private var supportBatchResult = false - private val sanitizedTableName = AvroSchemaUtils.getAvroRecordQualifiedName(tableName) override def supportBatch(sparkSession: SparkSession, schema: StructType): Boolean = { if (!supportBatchCalled || supportBatchResult) { supportBatchCalled = true supportBatchResult = !isMOR && !isIncremental && !isBootstrap && super.supportBatch(sparkSession, schema) } + sparkSession.conf.set(PARQUET_VECTORIZED_READER_ENABLED.key, supportBatchResult) supportBatchResult } + private lazy val internalSchemaOpt: org.apache.hudi.common.util.Option[InternalSchema] = if (tableSchema.internalSchema.isEmpty) { + org.apache.hudi.common.util.Option.empty() + } else { + org.apache.hudi.common.util.Option.of(tableSchema.internalSchema.get) + } + override def isSplitable(sparkSession: SparkSession, options: Map[String, String], path: Path): Boolean = false @@ -108,21 +112,17 @@ class HoodieFileGroupReaderBasedParquetFileFormat(tableState: HoodieTableState, val partitionColumns = partitionSchema.fieldNames val dataSchema = StructType(tableSchema.structTypeSchema.fields.filterNot(f => partitionColumns.contains(f.name))) val outputSchema = StructType(requiredSchema.fields ++ partitionSchema.fields) - spark.conf.set("spark.sql.parquet.enableVectorizedReader", supportBatchResult) - val requiredSchemaWithMandatory = generateRequiredSchemaWithMandatory(requiredSchema, dataSchema, partitionSchema) - val isCount = requiredSchemaWithMandatory.isEmpty - val requiredSchemaSplits = requiredSchemaWithMandatory.fields.partition(f => HoodieRecord.HOODIE_META_COLUMNS_WITH_OPERATION.contains(f.name)) - val requiredMeta = StructType(requiredSchemaSplits._1) - val requiredWithoutMeta = StructType(requiredSchemaSplits._2) - val augmentedHadoopConf = new HadoopStorageConfiguration(hadoopConf).getInline.unwrap - val (baseFileReader, preMergeBaseFileReader, readerMaps, cdcFileReader) = buildFileReaders( - spark, dataSchema, partitionSchema, requiredSchema, filters, options, augmentedHadoopConf, - requiredSchemaWithMandatory, requiredWithoutMeta, requiredMeta) + val isCount = requiredSchema.isEmpty && !isMOR && !isIncremental + val augmentedStorageConf = new HadoopStorageConfiguration(hadoopConf).getInline + setSchemaEvolutionConfigs(augmentedStorageConf) + val baseFileReader = super.buildReaderWithPartitionValues(spark, dataSchema, partitionSchema, requiredSchema, + filters ++ requiredFilters, options, augmentedStorageConf.unwrapCopy()) val requestedAvroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(requiredSchema, sanitizedTableName) val dataAvroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(dataSchema, sanitizedTableName) - - val broadcastedHadoopConf = spark.sparkContext.broadcast(new SerializableConfiguration(augmentedHadoopConf)) + val parquetFileReader = spark.sparkContext.broadcast(sparkAdapter.createParquetFileReader(supportBatchResult, + spark.sessionState.conf, options, augmentedStorageConf.unwrap())) + val broadcastedStorageConf = spark.sparkContext.broadcast(augmentedStorageConf) val broadcastedDataSchema = spark.sparkContext.broadcast(dataAvroSchema) val broadcastedRequestedSchema = spark.sparkContext.broadcast(requestedAvroSchema) val fileIndexProps: TypedProperties = HoodieFileIndex.getConfigProperties(spark, options) @@ -131,21 +131,16 @@ class HoodieFileGroupReaderBasedParquetFileFormat(tableState: HoodieTableState, file.partitionValues match { // Snapshot or incremental queries. case fileSliceMapping: HoodiePartitionFileSliceMapping => - val filePath = sparkAdapter.getSparkPartitionedFileUtils.getPathFromPartitionedFile(file) - val filegroupName = if (FSUtils.isLogFile(filePath)) { - FSUtils.getFileId(filePath.getName).substring(1) - } else { - FSUtils.getFileId(filePath.getName) - } + val filegroupName = FSUtils.getFileIdFromFilePath(sparkAdapter + .getSparkPartitionedFileUtils.getPathFromPartitionedFile(file)) fileSliceMapping.getSlice(filegroupName) match { case Some(fileSlice) if !isCount => if (requiredSchema.isEmpty && !fileSlice.getLogFiles.findAny().isPresent) { val hoodieBaseFile = fileSlice.getBaseFile.get() baseFileReader(createPartitionedFile(fileSliceMapping.getPartitionValues, hoodieBaseFile.getStoragePath, 0, hoodieBaseFile.getFileLen)) } else { - val readerContext: HoodieReaderContext[InternalRow] = new SparkFileFormatInternalRowReaderContext( - readerMaps) - val storageConf = new HadoopStorageConfiguration(broadcastedHadoopConf.value.value) + val readerContext = new SparkFileFormatInternalRowReaderContext(parquetFileReader.value, tableState.recordKeyField, filters) + val storageConf = broadcastedStorageConf.value val metaClient: HoodieTableMetaClient = HoodieTableMetaClient .builder().setConf(storageConf).setBasePath(tableState.tablePath).build val reader = new HoodieFileGroupReader[InternalRow]( @@ -156,6 +151,8 @@ class HoodieFileGroupReaderBasedParquetFileFormat(tableState: HoodieTableState, fileSlice, broadcastedDataSchema.value, broadcastedRequestedSchema.value, + internalSchemaOpt, + metaClient, metaClient.getTableConfig.getProps, metaClient.getTableConfig, file.start, @@ -175,26 +172,36 @@ class HoodieFileGroupReaderBasedParquetFileFormat(tableState: HoodieTableState, fileSliceMapping.getPartitionValues) } - // TODO: Use FileGroupReader here: HUDI-6942. - case _ => baseFileReader(file) + case _ => parquetFileReader.value.read(file, requiredSchema, partitionSchema, filters, + broadcastedStorageConf.value) } // CDC queries. case hoodiePartitionCDCFileGroupSliceMapping: HoodiePartitionCDCFileGroupMapping => val fileSplits = hoodiePartitionCDCFileGroupSliceMapping.getFileSplits().toArray val fileGroupSplit: HoodieCDCFileGroupSplit = HoodieCDCFileGroupSplit(fileSplits) buildCDCRecordIterator( - fileGroupSplit, cdcFileReader, - new HadoopStorageConfiguration(broadcastedHadoopConf.value.value), fileIndexProps, requiredSchema) - // TODO: Use FileGroupReader here: HUDI-6942. - case _ => baseFileReader(file) + fileGroupSplit, parquetFileReader.value, broadcastedStorageConf.value, + fileIndexProps, options, requiredSchema) + + case _ => parquetFileReader.value.read(file, requiredSchema, partitionSchema, filters, + broadcastedStorageConf.value) } } } + private def setSchemaEvolutionConfigs(conf: StorageConfiguration[_]): Unit = { + if (internalSchemaOpt.isPresent) { + conf.set(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA, SerDeHelper.toJson(internalSchemaOpt.get())) + conf.set(SparkInternalSchemaConverter.HOODIE_TABLE_PATH, tableState.tablePath) + conf.set(SparkInternalSchemaConverter.HOODIE_VALID_COMMITS_LIST, validCommits) + } + } + protected def buildCDCRecordIterator(cdcFileGroupSplit: HoodieCDCFileGroupSplit, - cdcFileReader: PartitionedFile => Iterator[InternalRow], + sparkParquetReader: SparkParquetReader, storageConf: StorageConfiguration[_], props: TypedProperties, + options: Map[String, String], requiredSchema: StructType): Iterator[InternalRow] = { props.setProperty(HoodieTableConfig.HOODIE_TABLE_NAME_KEY, tableName) val cdcSchema = CDCRelation.FULL_CDC_SPARK_SCHEMA @@ -204,11 +211,12 @@ class HoodieFileGroupReaderBasedParquetFileFormat(tableState: HoodieTableState, cdcFileGroupSplit, metaClient, storageConf, - cdcFileReader, + sparkParquetReader, tableSchema, cdcSchema, requiredSchema, - props) + props, + options) } private def appendPartitionAndProject(iter: HoodieFileGroupReader.HoodieFileGroupReaderIterator[InternalRow], @@ -232,211 +240,6 @@ class HoodieFileGroupReaderBasedParquetFileFormat(tableState: HoodieTableState, makeCloseableFileGroupMappingRecordIterator(iter, d => unsafeProjection(d)) } - private def generateRequiredSchemaWithMandatory(requiredSchema: StructType, - dataSchema: StructType, - partitionSchema: StructType): StructType = { - val metaFields = Seq( - StructField(HoodieRecord.COMMIT_TIME_METADATA_FIELD, StringType), - StructField(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, StringType), - StructField(HoodieRecord.RECORD_KEY_METADATA_FIELD, StringType), - StructField(HoodieRecord.PARTITION_PATH_METADATA_FIELD, StringType), - StructField(HoodieRecord.FILENAME_METADATA_FIELD, StringType)) - - // Helper method to get the StructField for nested fields - @tailrec - def findNestedField(schema: StructType, fieldParts: Array[String]): Option[StructField] = { - fieldParts.toList match { - case head :: Nil => schema.fields.find(_.name == head) // If it's the last part, find and return the field - case head :: tail => // If there are more parts, find the field and its nested fields - schema.fields.find(_.name == head) match { - case Some(StructField(_, nested: StructType, _, _)) => findNestedField(nested, tail.toArray) - case _ => None // The path is not valid - } - case _ => None // Empty path, should not happen if the input is correct - } - } - - def findMetaField(name: String): Option[StructField] = { - metaFields.find(f => f.name == name) - } - - val added: mutable.Buffer[StructField] = mutable.Buffer[StructField]() - for (field <- mandatoryFields) { - if (requiredSchema.getFieldIndex(field).isEmpty) { - // Support for nested fields - val fieldParts = field.split("\\.") - val fieldToAdd = findNestedField(dataSchema, fieldParts) - .orElse(findNestedField(partitionSchema, fieldParts)) - .orElse(findMetaField(field)) - .getOrElse(throw new IllegalArgumentException(s"Field $field does not exist in the table schema")) - added.append(fieldToAdd) - } - } - val addedFields = StructType(added.toArray) - StructType(requiredSchema.toArray ++ addedFields.fields) - } - - protected def buildFileReaders(sparkSession: SparkSession, dataSchema: StructType, partitionSchema: StructType, - requiredSchema: StructType, filters: Seq[Filter], options: Map[String, String], - hadoopConf: Configuration, requiredSchemaWithMandatory: StructType, - requiredWithoutMeta: StructType, requiredMeta: StructType): - (PartitionedFile => Iterator[InternalRow], - PartitionedFile => Iterator[InternalRow], - mutable.Map[Long, PartitionedFile => Iterator[InternalRow]], - PartitionedFile => Iterator[InternalRow]) = { - - val m = scala.collection.mutable.Map[Long, PartitionedFile => Iterator[InternalRow]]() - - val recordKeyRelatedFilters = getRecordKeyRelatedFilters(filters, tableState.recordKeyField) - val baseFileReader = super.buildReaderWithPartitionValues(sparkSession, dataSchema, partitionSchema, requiredSchema, - filters ++ requiredFilters, options, new Configuration(hadoopConf)) - m.put(generateKey(dataSchema, requiredSchema), baseFileReader) - - // File reader for reading a Hoodie base file that needs to be merged with log files - // Add support for reading files using inline file system. - val appliedRequiredSchema: StructType = getAppliedRequiredSchema( - requiredSchemaWithMandatory, shouldUseRecordPosition, ROW_INDEX_TEMPORARY_COLUMN_NAME) - val appliedFilters = getAppliedFilters( - requiredFilters, recordKeyRelatedFilters, shouldUseRecordPosition) - val preMergeBaseFileReader = super.buildReaderWithPartitionValues( - sparkSession, - dataSchema, - StructType(Nil), - appliedRequiredSchema, - appliedFilters, - options, - new Configuration(hadoopConf)) - m.put(generateKey(dataSchema, appliedRequiredSchema), preMergeBaseFileReader) - - val cdcFileReader = super.buildReaderWithPartitionValues( - sparkSession, - tableSchema.structTypeSchema, - StructType(Nil), - tableSchema.structTypeSchema, - Nil, - options + (OPTION_RETURNING_BATCH -> super.supportBatch(sparkSession, tableSchema.structTypeSchema).toString), - new Configuration(hadoopConf)) - - //Rules for appending partitions and filtering in the bootstrap readers: - // 1. if it is mor, we don't want to filter data or append partitions - // 2. if we need to merge the bootstrap base and skeleton files then we cannot filter - // 3. if we need to merge the bootstrap base and skeleton files then we should never append partitions to the - // skeleton reader - val needMetaCols = requiredMeta.nonEmpty - val needDataCols = requiredWithoutMeta.nonEmpty - - //file reader for bootstrap skeleton files - if (needMetaCols && isBootstrap) { - val key = generateKey(HoodieSparkUtils.getMetaSchema, requiredMeta) - if (needDataCols || isMOR) { - // no filter and no append - m.put(key, super.buildReaderWithPartitionValues(sparkSession, HoodieSparkUtils.getMetaSchema, StructType(Seq.empty), - requiredMeta, Seq.empty, options, new Configuration(hadoopConf))) - } else { - // filter - m.put(key, super.buildReaderWithPartitionValues(sparkSession, HoodieSparkUtils.getMetaSchema, StructType(Seq.empty), - requiredMeta, filters ++ requiredFilters, options, new Configuration(hadoopConf))) - } - - val requestedMeta = StructType(requiredSchema.fields.filter(sf => isMetaField(sf.name))) - m.put(generateKey(HoodieSparkUtils.getMetaSchema, requestedMeta), - super.buildReaderWithPartitionValues(sparkSession, HoodieSparkUtils.getMetaSchema, StructType(Seq.empty), requestedMeta, - Seq.empty, options, new Configuration(hadoopConf))) - } - - //file reader for bootstrap base files - if (needDataCols && isBootstrap) { - val dataSchemaWithoutMeta = StructType(dataSchema.fields.filterNot(sf => isMetaField(sf.name))) - val key = generateKey(dataSchemaWithoutMeta, requiredWithoutMeta) - if (isMOR || needMetaCols) { - m.put(key, super.buildReaderWithPartitionValues(sparkSession, dataSchemaWithoutMeta, StructType(Seq.empty), requiredWithoutMeta, - Seq.empty, options, new Configuration(hadoopConf))) - // no filter and no append - - } else { - // filter - m.put(key, super.buildReaderWithPartitionValues(sparkSession, dataSchemaWithoutMeta, StructType(Seq.empty), requiredWithoutMeta, - filters ++ requiredFilters, options, new Configuration(hadoopConf))) - } - - val requestedWithoutMeta = StructType(requiredSchema.fields.filterNot(sf => isMetaField(sf.name))) - m.put(generateKey(dataSchemaWithoutMeta, requestedWithoutMeta), - super.buildReaderWithPartitionValues(sparkSession, dataSchemaWithoutMeta, StructType(Seq.empty), requestedWithoutMeta, - Seq.empty, options, new Configuration(hadoopConf))) - } - - (baseFileReader, preMergeBaseFileReader, m, cdcFileReader) - } - - protected def generateKey(dataSchema: StructType, requestedSchema: StructType): Long = { - AvroConversionUtils.convertStructTypeToAvroSchema(dataSchema, sanitizedTableName).hashCode() + AvroConversionUtils.convertStructTypeToAvroSchema(requestedSchema, sanitizedTableName).hashCode() - } -} - -object HoodieFileGroupReaderBasedParquetFileFormat { - // From "ParquetFileFormat.scala": The names of the field for record position. - private val ROW_INDEX = "row_index" - private val ROW_INDEX_TEMPORARY_COLUMN_NAME = s"_tmp_metadata_$ROW_INDEX" - - // From "namedExpressions.scala": Used to construct to record position field metadata. - private val FILE_SOURCE_GENERATED_METADATA_COL_ATTR_KEY = "__file_source_generated_metadata_col" - private val FILE_SOURCE_METADATA_COL_ATTR_KEY = "__file_source_metadata_col" - private val METADATA_COL_ATTR_KEY = "__metadata_col" - - /** - * A required option (since Spark 3.3.2) to pass to buildReaderWithPartitionValues to return columnar batch output or not. - * For ParquetFileFormat and OrcFileFormat, passing this option is required. - * This should only be passed as true if it can actually be supported, which can be checked - * by calling supportBatch. - */ - private val OPTION_RETURNING_BATCH = "returning_batch" - - def getRecordKeyRelatedFilters(filters: Seq[Filter], recordKeyColumn: String): Seq[Filter] = { - filters.filter(f => f.references.exists(c => c.equalsIgnoreCase(recordKeyColumn))) - } - - def getLogFilesFromSlice(fileSlice: FileSlice): List[HoodieLogFile] = { - fileSlice.getLogFiles.sorted(HoodieLogFile.getLogFileComparator).iterator().asScala.toList - } - - def getFieldMetadata(name: String, internalName: String): Metadata = { - new MetadataBuilder() - .putString(METADATA_COL_ATTR_KEY, name) - .putBoolean(FILE_SOURCE_METADATA_COL_ATTR_KEY, value = true) - .putString(FILE_SOURCE_GENERATED_METADATA_COL_ATTR_KEY, internalName) - .build() - } - - def getAppliedRequiredSchema(requiredSchema: StructType, - shouldUseRecordPosition: Boolean, - recordPositionColumn: String): StructType = { - if (shouldAddRecordPositionColumn(shouldUseRecordPosition)) { - val metadata = getFieldMetadata(recordPositionColumn, ROW_INDEX_TEMPORARY_COLUMN_NAME) - val rowIndexField = StructField(recordPositionColumn, LongType, nullable = false, metadata) - StructType(requiredSchema.fields :+ rowIndexField) - } else { - requiredSchema - } - } - - def getAppliedFilters(requiredFilters: Seq[Filter], - recordKeyRelatedFilters: Seq[Filter], - shouldUseRecordPosition: Boolean): Seq[Filter] = { - if (shouldAddRecordKeyFilters(shouldUseRecordPosition)) { - requiredFilters ++ recordKeyRelatedFilters - } else { - requiredFilters - } - } - - def shouldAddRecordPositionColumn(shouldUseRecordPosition: Boolean): Boolean = { - HoodieSparkUtils.gteqSpark3_5 && shouldUseRecordPosition - } - - def shouldAddRecordKeyFilters(shouldUseRecordPosition: Boolean): Boolean = { - (!shouldUseRecordPosition) || HoodieSparkUtils.gteqSpark3_5 - } - def makeCloseableFileGroupMappingRecordIterator(closeableFileGroupRecordIterator: HoodieFileGroupReader.HoodieFileGroupReaderIterator[InternalRow], mappingFunction: Function[InternalRow, InternalRow]): Iterator[InternalRow] = { new Iterator[InternalRow] with Closeable { diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkParquetReaderBase.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkParquetReaderBase.scala index 2b47da764565..ddfa4ba68370 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkParquetReaderBase.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkParquetReaderBase.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.datasources.parquet import org.apache.hadoop.conf.Configuration +import org.apache.hudi.storage.StorageConfiguration import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.internal.SQLConf @@ -46,15 +47,15 @@ abstract class SparkParquetReaderBase(enableVectorizedReader: Boolean, * @param requiredSchema desired output schema of the data * @param partitionSchema schema of the partition columns. Partition values will be appended to the end of every row * @param filters filters for data skipping. Not guaranteed to be used; the spark plan will also apply the filters. - * @param sharedConf the hadoop conf + * @param storageConf the hadoop conf * @return iterator of rows read from the file output type says [[InternalRow]] but could be [[ColumnarBatch]] */ final def read(file: PartitionedFile, requiredSchema: StructType, partitionSchema: StructType, filters: Seq[Filter], - sharedConf: Configuration): Iterator[InternalRow] = { - val conf = new Configuration(sharedConf) + storageConf: StorageConfiguration[Configuration]): Iterator[InternalRow] = { + val conf = storageConf.unwrapCopy() conf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, requiredSchema.json) conf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, requiredSchema.json) ParquetWriteSupport.setSchema(requiredSchema, conf) diff --git a/hudi-spark-datasource/hudi-spark-common/src/test/java/org/apache/hudi/TestHoodieSchemaUtils.java b/hudi-spark-datasource/hudi-spark-common/src/test/java/org/apache/hudi/TestHoodieSchemaUtils.java index a69f50777ce0..ec496a98cee9 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/test/java/org/apache/hudi/TestHoodieSchemaUtils.java +++ b/hudi-spark-datasource/hudi-spark-common/src/test/java/org/apache/hudi/TestHoodieSchemaUtils.java @@ -27,16 +27,20 @@ import org.apache.hudi.exception.MissingSchemaFieldException; import org.apache.hudi.exception.SchemaBackwardsCompatibilityException; -import org.apache.avro.JsonProperties; import org.apache.avro.Schema; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; -import java.util.Arrays; import java.util.HashMap; import java.util.Map; +import static org.apache.hudi.avro.AvroSchemaTestUtils.createArrayField; +import static org.apache.hudi.avro.AvroSchemaTestUtils.createMapField; +import static org.apache.hudi.avro.AvroSchemaTestUtils.createNestedField; +import static org.apache.hudi.avro.AvroSchemaTestUtils.createNullablePrimitiveField; +import static org.apache.hudi.avro.AvroSchemaTestUtils.createPrimitiveField; +import static org.apache.hudi.avro.AvroSchemaTestUtils.createRecord; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -279,9 +283,9 @@ void testFieldReordering() { createPrimitiveField("nestedField1", Schema.Type.INT), createPrimitiveField("nestedField2", Schema.Type.INT), createPrimitiveField("nestedField3", Schema.Type.INT), - createNullableField("nestedField4", Schema.Type.INT))), + createNullablePrimitiveField("nestedField4", Schema.Type.INT))), createPrimitiveField("field4", Schema.Type.INT), - createNullableField("field5", Schema.Type.INT)); + createNullablePrimitiveField("field5", Schema.Type.INT)); assertEquals(expected, deduceWriterSchema(end, start, true)); } @@ -297,40 +301,4 @@ private static Schema deduceWriterSchema(Schema incomingSchema, Schema latestTab Option.empty(), TYPED_PROPERTIES); } - private static Schema.Field createNestedField(String name, Schema.Type type) { - return createNestedField(name, Schema.create(type)); - } - - private static Schema.Field createNestedField(String name, Schema schema) { - return new Schema.Field(name, createRecord(name, new Schema.Field("nested", schema, null, null)), null, null); - } - - private static Schema.Field createArrayField(String name, Schema.Type type) { - return createArrayField(name, Schema.create(type)); - } - - private static Schema.Field createArrayField(String name, Schema schema) { - return new Schema.Field(name, Schema.createArray(schema), null, null); - } - - private static Schema.Field createMapField(String name, Schema.Type type) { - return createMapField(name, Schema.create(type)); - } - - private static Schema.Field createMapField(String name, Schema schema) { - return new Schema.Field(name, Schema.createMap(schema), null, null); - } - - private static Schema.Field createPrimitiveField(String name, Schema.Type type) { - return new Schema.Field(name, Schema.create(type), null, null); - } - - private static Schema.Field createNullableField(String name, Schema.Type type) { - return new Schema.Field(name, Schema.createUnion(Arrays.asList(Schema.create(Schema.Type.NULL), Schema.create(type))), null, JsonProperties.NULL_VALUE); - } - - private static Schema createRecord(String name, Schema.Field... fields) { - return Schema.createRecord(name, null, null, false, Arrays.asList(fields)); - } - } diff --git a/hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/execution/datasources/parquet/TestHoodieFileGroupReaderBasedParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/execution/datasources/parquet/TestHoodieFileGroupReaderBasedParquetFileFormat.scala index 5a5456cb1734..00afbcd7b170 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/execution/datasources/parquet/TestHoodieFileGroupReaderBasedParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/execution/datasources/parquet/TestHoodieFileGroupReaderBasedParquetFileFormat.scala @@ -19,12 +19,12 @@ package org.apache.spark.execution.datasources.parquet -import org.apache.hudi.HoodieSparkUtils +import org.apache.hudi.SparkFileFormatInternalRowReaderContext +import org.apache.hudi.common.table.read.HoodiePositionBasedFileGroupRecordBuffer.ROW_INDEX_TEMPORARY_COLUMN_NAME import org.apache.hudi.testutils.SparkClientFunctionalTestHarness -import org.apache.spark.sql.execution.datasources.parquet.HoodieFileGroupReaderBasedParquetFileFormat import org.apache.spark.sql.sources.{EqualTo, GreaterThan, IsNotNull} import org.apache.spark.sql.types.{LongType, StringType, StructField, StructType} -import org.junit.jupiter.api.Assertions.assertEquals +import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} import org.junit.jupiter.api.Test class TestHoodieFileGroupReaderBasedParquetFileFormat extends SparkClientFunctionalTestHarness { @@ -34,7 +34,7 @@ class TestHoodieFileGroupReaderBasedParquetFileFormat extends SparkClientFunctio IsNotNull("non_key_column"), EqualTo("non_key_column", 1) ) - val filtersWithoutKeyColumn = HoodieFileGroupReaderBasedParquetFileFormat.getRecordKeyRelatedFilters( + val filtersWithoutKeyColumn = SparkFileFormatInternalRowReaderContext.getRecordKeyRelatedFilters( filters, "key_column"); assertEquals(0, filtersWithoutKeyColumn.size) @@ -42,7 +42,7 @@ class TestHoodieFileGroupReaderBasedParquetFileFormat extends SparkClientFunctio EqualTo("key_column", 1), GreaterThan("non_key_column", 2) ) - val filtersWithKeyColumn = HoodieFileGroupReaderBasedParquetFileFormat.getRecordKeyRelatedFilters( + val filtersWithKeyColumn = SparkFileFormatInternalRowReaderContext.getRecordKeyRelatedFilters( filtersWithKeys, "key_column") assertEquals(1, filtersWithKeyColumn.size) assertEquals("key_column", filtersWithKeyColumn.head.references.head) @@ -55,42 +55,9 @@ class TestHoodieFileGroupReaderBasedParquetFileFormat extends SparkClientFunctio StructField("column_b", StringType, nullable = false)) val requiredSchema = StructType(fields) - val appliedSchema: StructType = HoodieFileGroupReaderBasedParquetFileFormat.getAppliedRequiredSchema( - requiredSchema, shouldUseRecordPosition = true, "row_index") - if (HoodieSparkUtils.gteqSpark3_5) { - assertEquals(3, appliedSchema.fields.length) - } else { - assertEquals(2, appliedSchema.fields.length) - } - - val schemaWithoutRowIndexColumn = HoodieFileGroupReaderBasedParquetFileFormat.getAppliedRequiredSchema( - requiredSchema, shouldUseRecordPosition = false, "row_index") - assertEquals(2, schemaWithoutRowIndexColumn.fields.length) - } - - @Test - def testGetAppliedFilters(): Unit = { - val filters = Seq( - IsNotNull("non_key_column"), - EqualTo("non_key_column", 1) - ) - val keyRelatedFilters = Seq( - EqualTo("key_column", 2) - ) - - val appliedFilters = HoodieFileGroupReaderBasedParquetFileFormat.getAppliedFilters( - filters, keyRelatedFilters, shouldUseRecordPosition = true - ) - if (!HoodieSparkUtils.gteqSpark3_5) { - assertEquals(2, appliedFilters.size) - } else { - assertEquals(3, appliedFilters.size) - } - - val appliedFiltersWithoutUsingRecordPosition = HoodieFileGroupReaderBasedParquetFileFormat.getAppliedFilters( - filters, keyRelatedFilters, shouldUseRecordPosition = false - ) - assertEquals(3, appliedFiltersWithoutUsingRecordPosition.size) + val appliedSchema: StructType = SparkFileFormatInternalRowReaderContext.getAppliedRequiredSchema( + requiredSchema) + assertEquals(3, appliedSchema.fields.length) + assertTrue(appliedSchema.fields.map(f => f.name).contains(ROW_INDEX_TEMPORARY_COLUMN_NAME)) } } - diff --git a/hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/TestSparkParquetReaderFormat.scala b/hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/TestSparkParquetReaderFormat.scala index bf513847cfcb..be409e50534e 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/TestSparkParquetReaderFormat.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/TestSparkParquetReaderFormat.scala @@ -21,12 +21,12 @@ package org.apache.spark.sql.execution.datasources.parquet import org.apache.hadoop.conf.Configuration import org.apache.hudi.SparkAdapterSupport +import org.apache.hudi.hadoop.fs.HadoopFSUtils import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType -import org.apache.spark.util.SerializableConfiguration /** * Class used to test [[SparkParquetReader]] @@ -45,12 +45,12 @@ class TestSparkParquetReaderFormat extends ParquetFileFormat with SparkAdapterSu val reader = sparkAdapter.createParquetFileReader(supportBatch(sparkSession, StructType(partitionSchema.fields ++ requiredSchema.fields)), sparkSession.sqlContext.conf, options, hadoopConf) - val broadcastedHadoopConf = - sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) + val broadcastedStorageConf = + sparkSession.sparkContext.broadcast(HadoopFSUtils.getStorageConf(hadoopConf)) (file: PartitionedFile) => { //code inside the lambda will run on the executor - reader.read(file, requiredSchema, partitionSchema, filters, broadcastedHadoopConf.value.value) + reader.read(file, requiredSchema, partitionSchema, filters, broadcastedStorageConf.value) } } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestHoodiePositionBasedFileGroupRecordBuffer.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestHoodiePositionBasedFileGroupRecordBuffer.java index bfa9054f8d51..c5c9ba775c99 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestHoodiePositionBasedFileGroupRecordBuffer.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestHoodiePositionBasedFileGroupRecordBuffer.java @@ -21,6 +21,7 @@ import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.engine.HoodieReaderContext; import org.apache.hudi.common.model.DeleteRecord; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordMerger; @@ -30,6 +31,7 @@ import org.apache.hudi.common.table.log.block.HoodieDeleteBlock; import org.apache.hudi.common.table.log.block.HoodieLogBlock; import org.apache.hudi.common.table.read.HoodiePositionBasedFileGroupRecordBuffer; +import org.apache.hudi.common.table.read.HoodiePositionBasedSchemaHandler; import org.apache.hudi.common.table.read.TestHoodieFileGroupReaderOnSpark; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.SchemaTestUtil; @@ -98,13 +100,19 @@ public void prepareBuffer(boolean useCustomMerger) throws Exception { Option partitionNameOpt = StringUtils.isNullOrEmpty(partitionPaths[0]) ? Option.empty() : Option.of(partitionPaths[0]); + HoodieReaderContext ctx = getHoodieReaderContext(getBasePath(), avroSchema, getStorageConf()); + ctx.setHasBootstrapBaseFile(false); + ctx.setHasLogFiles(true); + ctx.setNeedsBootstrapMerge(false); + ctx.setRecordMerger(useCustomMerger ? new CustomMerger() : new HoodieSparkRecordMerger()); + ctx.setSchemaHandler(new HoodiePositionBasedSchemaHandler<>(ctx, avroSchema, avroSchema, + Option.empty(), metaClient.getTableConfig())); buffer = new HoodiePositionBasedFileGroupRecordBuffer<>( - getHoodieReaderContext(getBasePath(), avroSchema), - avroSchema, - avroSchema, + ctx, + metaClient, partitionNameOpt, partitionFields, - useCustomMerger ? new CustomMerger() : new HoodieSparkRecordMerger(), + ctx.getRecordMerger(), new TypedProperties(), 1024 * 1024 * 1000, metaClient.getTempFolderPath(), @@ -198,4 +206,3 @@ public HoodieRecord.HoodieRecordType getRecordType() { } } } - diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala index 619647555194..e81aca2069fa 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala @@ -146,11 +146,7 @@ class TestAvroSchemaResolutionSupport extends HoodieClientTestBase with ScalaAss upsertData(upsertDf, tempRecordPath, isCow) // read out the table - val readDf = spark.read.format("hudi") - // NOTE: type promotion is not supported for the custom file format and the filegroup reader - // HUDI-7045 and PR#10007 in progress to fix the issue - .option(HoodieReaderConfig.FILE_GROUP_READER_ENABLED.key(), "false") - .load(tempRecordPath) + val readDf = spark.read.format("hudi").load(tempRecordPath) readDf.printSchema() readDf.show(false) readDf.foreach(_ => {}) @@ -389,11 +385,7 @@ class TestAvroSchemaResolutionSupport extends HoodieClientTestBase with ScalaAss withSQLConf("spark.sql.parquet.enableNestedColumnVectorizedReader" -> "false") { // read out the table - val readDf = spark.read.format("hudi") - // NOTE: long to int type change is not supported for the custom file format and the filegroup reader - // HUDI-7045 and PR#10007 in progress to fix the issue - .option(HoodieReaderConfig.FILE_GROUP_READER_ENABLED.key(), "false") - .load(tempRecordPath) + val readDf = spark.read.format("hudi").load(tempRecordPath) readDf.printSchema() readDf.show(false) readDf.foreach(_ => {}) @@ -487,11 +479,7 @@ class TestAvroSchemaResolutionSupport extends HoodieClientTestBase with ScalaAss withSQLConf("spark.sql.parquet.enableNestedColumnVectorizedReader" -> "false") { // read out the table - val readDf = spark.read.format("hudi") - // NOTE: type promotion is not supported for the custom file format and the filegroup reader - // HUDI-7045 and PR#10007 in progress to fix the issue - .option(HoodieReaderConfig.FILE_GROUP_READER_ENABLED.key(), "false") - .load(tempRecordPath) + val readDf = spark.read.format("hudi").load(tempRecordPath) readDf.printSchema() readDf.show(false) readDf.foreach(_ => {}) @@ -555,11 +543,7 @@ class TestAvroSchemaResolutionSupport extends HoodieClientTestBase with ScalaAss withSQLConf("spark.sql.parquet.enableNestedColumnVectorizedReader" -> "false") { // read out the table - val readDf = spark.read.format("hudi") - // NOTE: type promotion is not supported for the custom file format and the filegroup reader - // HUDI-7045 and PR#10007 in progress to fix the issue - .option(HoodieReaderConfig.FILE_GROUP_READER_ENABLED.key(), "false") - .load(tempRecordPath) + val readDf = spark.read.format("hudi").load(tempRecordPath) readDf.printSchema() readDf.show(false) readDf.foreach(_ => {}) @@ -826,11 +810,7 @@ class TestAvroSchemaResolutionSupport extends HoodieClientTestBase with ScalaAss upsertData(df7, tempRecordPath) // read out the table - val readDf = spark.read.format("hudi") - // NOTE: type promotion is not supported for the custom file format and the filegroup reader - // HUDI-7045 and PR#10007 in progress to fix the issue - .option(HoodieReaderConfig.FILE_GROUP_READER_ENABLED.key(), "false") - .load(tempRecordPath) + val readDf = spark.read.format("hudi").load(tempRecordPath) readDf.printSchema() readDf.show(false) readDf.foreach(_ => {}) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala index 6d988ed00b7f..4bee2bfa6e03 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala @@ -24,14 +24,12 @@ import org.apache.hadoop.conf.Configuration import org.apache.hudi.common.config.HoodieReaderConfig.FILE_GROUP_READER_ENABLED import org.apache.hudi.common.engine.HoodieReaderContext import org.apache.hudi.common.model.{HoodieRecord, WriteOperationType} +import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.common.testutils.HoodieTestUtils import org.apache.hudi.storage.StorageConfiguration -import org.apache.hudi.{AvroConversionUtils, SparkFileFormatInternalRowReaderContext} +import org.apache.hudi.{HoodieSparkRecordMerger, SparkAdapterSupport, SparkFileFormatInternalRowReaderContext} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.execution.datasources.PartitionedFile -import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.functions.col -import org.apache.spark.sql.types.StructType import org.apache.spark.sql.{Dataset, HoodieInternalRowUtils, HoodieUnsafeUtils, Row, SaveMode, SparkSession} import org.apache.spark.{HoodieSparkKryoRegistrar, SparkConf} import org.junit.jupiter.api.Assertions.assertEquals @@ -44,7 +42,7 @@ import scala.collection.JavaConverters._ * Tests {@link HoodieFileGroupReader} with {@link SparkFileFormatInternalRowReaderContext} * on Spark */ -class TestHoodieFileGroupReaderOnSpark extends TestHoodieFileGroupReaderBase[InternalRow] { +class TestHoodieFileGroupReaderOnSpark extends TestHoodieFileGroupReaderBase[InternalRow] with SparkAdapterSupport { var spark: SparkSession = _ @BeforeEach @@ -81,16 +79,11 @@ class TestHoodieFileGroupReaderOnSpark extends TestHoodieFileGroupReaderBase[Int tempDir.toAbsolutePath.toUri.toString } - override def getHoodieReaderContext(tablePath: String, avroSchema: Schema): HoodieReaderContext[InternalRow] = { - val parquetFileFormat = new ParquetFileFormat - val structTypeSchema = AvroConversionUtils.convertAvroSchemaToStructType(avroSchema) - - val recordReaderIterator = parquetFileFormat.buildReaderWithPartitionValues( - spark, structTypeSchema, StructType(Seq.empty), structTypeSchema, Seq.empty, Map.empty, getStorageConf.unwrapAs(classOf[Configuration])) - - val m = scala.collection.mutable.Map[Long, PartitionedFile => Iterator[InternalRow]]() - m.put(2*avroSchema.hashCode(), recordReaderIterator) - new SparkFileFormatInternalRowReaderContext(m) + override def getHoodieReaderContext(tablePath: String, avroSchema: Schema, storageConf: StorageConfiguration[_]): HoodieReaderContext[InternalRow] = { + val reader = sparkAdapter.createParquetFileReader(vectorized = false, spark.sessionState.conf, Map.empty, storageConf.unwrapAs(classOf[Configuration])) + val metaClient = HoodieTableMetaClient.builder().setConf(storageConf).setBasePath(tablePath).build + val recordKeyField = new HoodieSparkRecordMerger().getMandatoryFieldsForMerging(metaClient.getTableConfig)(0) + new SparkFileFormatInternalRowReaderContext(reader, recordKeyField, Seq.empty) } override def commitToTable(recordList: util.List[String], operation: String, options: util.Map[String, String]): Unit = { diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala index 8a01583ae5fd..5fab505df6de 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala @@ -19,26 +19,21 @@ package org.apache.hudi.common.table.read -import org.apache.hudi.{DataSourceWriteOptions, HoodieSparkUtils} +import org.apache.hadoop.conf.Configuration import org.apache.hudi.SparkAdapterSupport.sparkAdapter -import org.apache.hudi.common.config.{HoodieReaderConfig, HoodieStorageConfig} -import org.apache.hudi.common.model.HoodieTableType +import org.apache.hudi.common.config.HoodieStorageConfig +import org.apache.hudi.common.model.{HoodieRecord, HoodieTableType} import org.apache.hudi.common.testutils.HoodieTestTable import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.hadoop.fs.HadoopFSUtils import org.apache.hudi.testutils.SparkClientFunctionalTestHarness -import org.apache.hudi.util.CloseableInternalRowIterator - -import org.apache.hadoop.conf.Configuration -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.execution.datasources.parquet.{HoodieFileGroupReaderBasedParquetFileFormat, ParquetFileFormat} +import org.apache.hudi.{DataSourceWriteOptions, HoodieSparkUtils, SparkFileFormatInternalRowReaderContext} import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} +import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse} import org.junit.jupiter.api.{BeforeEach, Test} -import org.junit.jupiter.api.Assertions.{assertArrayEquals, assertEquals, assertFalse} class TestSpark35RecordPositionMetadataColumn extends SparkClientFunctionalTestHarness { private val PARQUET_FORMAT = "parquet" - private val ROW_INDEX_COLUMN = "_tmp_metadata_row_index" private val SPARK_MERGER = "org.apache.hudi.HoodieSparkRecordMerger" @BeforeEach @@ -69,81 +64,54 @@ class TestSpark35RecordPositionMetadataColumn extends SparkClientFunctionalTestH @Test def testRecordPositionColumn(): Unit = { - val _spark = spark - // Prepare the schema - val dataSchema = new StructType( - Array( - StructField("userid", IntegerType, nullable = false), - StructField("country", StringType, nullable = false), - StructField("ts", StringType, nullable = false) + //feature is only supported in spark 3.5+ + if (HoodieSparkUtils.gteqSpark3_5) { + val _spark = spark + // Prepare the schema + val dataSchema = new StructType( + Array( + StructField("userid", IntegerType, nullable = false), + StructField("country", StringType, nullable = false), + StructField("ts", StringType, nullable = false) + ) ) - ) - val requiredSchema = HoodieFileGroupReaderBasedParquetFileFormat - .getAppliedRequiredSchema( - dataSchema, - shouldUseRecordPosition = true, - ROW_INDEX_COLUMN) + val hadoopConf = new Configuration(spark().sparkContext.hadoopConfiguration) + val props = Map("spark.sql.parquet.enableVectorizedReader" -> "false") + _spark.conf.set("spark.sql.parquet.enableVectorizedReader", "false") + val reader = sparkAdapter.createParquetFileReader(vectorized = false, _spark.sessionState.conf, props, hadoopConf) + val requiredSchema = SparkFileFormatInternalRowReaderContext.getAppliedRequiredSchema(dataSchema) - // Confirm if the schema is as expected. - if (HoodieSparkUtils.gteqSpark3_5) { + // Confirm if the schema is as expected. assertEquals(4, requiredSchema.fields.length) assertEquals( "StructField(_tmp_metadata_row_index,LongType,false)", requiredSchema.fields(3).toString) - } - // Prepare the file and Parquet file reader. - _spark.conf.set("spark.sql.parquet.enableVectorizedReader", "false") - val metaClient = getHoodieMetaClient( - HadoopFSUtils.getStorageConfWithCopy(_spark.sparkContext.hadoopConfiguration), basePath) - val fileReader = new ParquetFileFormat().buildReaderWithPartitionValues( - _spark, - dataSchema, - StructType(Nil), - requiredSchema, - Nil, - Map.empty, - new Configuration(spark().sparkContext.hadoopConfiguration)) - val allBaseFiles = HoodieTestTable.of(metaClient).listAllBaseFiles - assertFalse(allBaseFiles.isEmpty) + // Prepare the file and Parquet file reader. + val metaClient = getHoodieMetaClient( + HadoopFSUtils.getStorageConfWithCopy(hadoopConf), basePath) + val allBaseFiles = HoodieTestTable.of(metaClient).listAllBaseFiles + assertFalse(allBaseFiles.isEmpty) + val readerContext = new SparkFileFormatInternalRowReaderContext(reader, HoodieRecord.RECORD_KEY_METADATA_FIELD, Seq.empty) + readerContext.setUseRecordPosition(true) + readerContext.setHasLogFiles(true) + readerContext.setNeedsBootstrapMerge(false) + readerContext.setHasBootstrapBaseFile(false) + //dataschema param is set to null because it is not used + val fileRecordIterator = readerContext.getFileRecordIterator(allBaseFiles.get(0).getPath, 0, allBaseFiles.get(0).getLength, null, + sparkAdapter.getAvroSchemaConverters.toAvroType(dataSchema, nullable = true, "record"), metaClient.getStorage) - // Make sure we can read all the positions out from base file. - // Here we don't add filters since enabling filter push-down - // for parquet file is tricky. - if (HoodieSparkUtils.gteqSpark3_5) { - val fileInfo = sparkAdapter.getSparkPartitionedFileUtils - .createPartitionedFile( - InternalRow.empty, - allBaseFiles.get(0).getPath, - 0, - allBaseFiles.get(0).getLength) - val iterator = new CloseableInternalRowIterator(fileReader.apply(fileInfo)) + // Make sure we can read all the positions out from base file. + // Here we don't add filters since enabling filter push-down + // for parquet file is tricky. var rowIndices: Set[Long] = Set() - while (iterator.hasNext) { - val row = iterator.next() + while (fileRecordIterator.hasNext) { + val row = fileRecordIterator.next() rowIndices += row.getLong(3) } - iterator.close() + fileRecordIterator.close() val expectedRowIndices: Set[Long] = Set(0L, 1L, 2L, 3L) assertEquals(expectedRowIndices, rowIndices) } } - - @Test - def testUseFileGroupReaderDirectly(): Unit = { - val _spark = spark - import _spark.implicits._ - - // Read the records out with record positions. - val allRecords = _spark.read.format("hudi") - .option(HoodieReaderConfig.MERGE_USE_RECORD_POSITIONS.key, "true") - .load(basePath) - - // Ensure the number of outcomes are correct for all Spark versions - // including Spark3.5. - val usRecords = allRecords - .select("userid") - .filter("country = 'US'").map(_.getInt(0)).collect() - assertArrayEquals(Array[Int](1, 3), usRecords) - } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/util/TestSparkInternalSchemaConverter.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/util/TestSparkInternalSchemaConverter.scala new file mode 100644 index 000000000000..d2ba8c3582c9 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/util/TestSparkInternalSchemaConverter.scala @@ -0,0 +1,93 @@ +/* + * 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.util + +import org.apache.avro.Schema +import org.apache.hudi.SparkAdapterSupport +import org.apache.hudi.client.utils.SparkInternalSchemaConverter.collectColNamesFromSparkStruct +import org.apache.hudi.internal.schema.convert.TestAvroInternalSchemaConverter._ +import org.apache.hudi.testutils.HoodieSparkClientTestHarness +import org.apache.spark.sql.types._ +import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} +import org.junit.jupiter.api.Test + +class TestSparkInternalSchemaConverter extends HoodieSparkClientTestHarness with SparkAdapterSupport { + + private def getStructType(schema: Schema): DataType = { + sparkAdapter.getAvroSchemaConverters.toSqlType(schema)._1 + } + + @Test + def testCollectColumnNames(): Unit = { + val simpleSchema = StructType(Seq( + StructField("field1", IntegerType, nullable = false, Metadata.empty), + StructField("field2", StringType, nullable = false, Metadata.empty))) + + assertEquals(getStructType(getSimpleSchema).json, simpleSchema.json) + var fieldNames = collectColNamesFromSparkStruct(simpleSchema) + var expectedOutput = getSimpleSchemaExpectedColumnNames() + assertEquals(expectedOutput.size(), fieldNames.size()) + assertTrue(fieldNames.containsAll(expectedOutput)) + + val simpleSchemaWithNullable = StructType(Seq( + StructField("field1", IntegerType, nullable = true, Metadata.empty), + StructField("field2", StringType, nullable = false, Metadata.empty))) + + assertEquals(getStructType(getSimpleSchemaWithNullable).json, simpleSchemaWithNullable.json) + fieldNames = collectColNamesFromSparkStruct(simpleSchemaWithNullable) + expectedOutput = getSimpleSchemaExpectedColumnNames() + assertEquals(expectedOutput.size(), fieldNames.size()) + assertTrue(fieldNames.containsAll(expectedOutput)) + + val complexSchemaSingleLevel = StructType(Seq( + StructField("field1", StructType(Seq( + StructField("nested", IntegerType, nullable = false, Metadata.empty) + )), nullable = false, Metadata.empty), + StructField("field2", ArrayType(StringType, containsNull = false), nullable = false, Metadata.empty), + StructField("field3", MapType(StringType, DoubleType, valueContainsNull = false), nullable = false, Metadata.empty) + )) + + assertEquals(getStructType(getComplexSchemaSingleLevel).json, complexSchemaSingleLevel.json) + fieldNames = collectColNamesFromSparkStruct(complexSchemaSingleLevel) + expectedOutput = getComplexSchemaSingleLevelExpectedColumnNames() + assertEquals(expectedOutput.size(), fieldNames.size()) + assertTrue(fieldNames.containsAll(expectedOutput)) + + val deeplyNestedField = StructType(Seq( + StructField("field1", IntegerType, nullable = false, Metadata.empty), + StructField("field2", StructType(Seq( + StructField("field2nestarray", + ArrayType( + StructType(Seq( + StructField("field21", IntegerType, nullable = true, Metadata.empty), + StructField("field22", IntegerType, nullable = true, Metadata.empty) + )), containsNull = true), + nullable = false) + )), nullable = false), + StructField("field3", IntegerType, nullable = true, Metadata.empty) + )) + + assertEquals(getStructType(getDeeplyNestedFieldSchema).json, deeplyNestedField.json) + fieldNames = collectColNamesFromSparkStruct(deeplyNestedField) + expectedOutput = getDeeplyNestedFieldSchemaExpectedColumnNames() + assertEquals(expectedOutput.size(), fieldNames.size()) + assertTrue(fieldNames.containsAll(expectedOutput)) + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestSpark3DDL.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestSpark3DDL.scala index e5b4beb97d1d..511916f7bb7e 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestSpark3DDL.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestSpark3DDL.scala @@ -139,6 +139,7 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase { spark.sessionState.catalog.dropTable(TableIdentifier(tableName), true, true) spark.sessionState.catalog.refreshTable(TableIdentifier(tableName)) spark.sessionState.conf.unsetConf(DataSourceWriteOptions.SPARK_SQL_INSERT_INTO_OPERATION.key) + spark.sessionState.conf.unsetConf("spark.sql.storeAssignmentPolicy") } } }) @@ -176,6 +177,7 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase { checkAnswer(spark.sql(s"select col0 from $tableName where id = 1").collect())( Seq("11") ) + spark.sessionState.conf.unsetConf("spark.sql.storeAssignmentPolicy") } } } @@ -337,6 +339,7 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase { spark.sql(s"select id, col1_new, col2 from $tableName where id = 1 or id = 6 or id = 2 or id = 11 order by id").show(false) } } + spark.sessionState.conf.unsetConf("spark.sql.storeAssignmentPolicy") spark.sessionState.conf.unsetConf(DataSourceWriteOptions.SPARK_SQL_INSERT_INTO_OPERATION.key) } } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamerSchemaEvolutionBase.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamerSchemaEvolutionBase.java index a10d98bcd7e7..945f64ece6ee 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamerSchemaEvolutionBase.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamerSchemaEvolutionBase.java @@ -156,7 +156,6 @@ protected HoodieDeltaStreamer.Config getDeltaStreamerConfig(String[] transformer protected HoodieDeltaStreamer.Config getDeltaStreamerConfig(String[] transformerClasses, boolean nullForDeletedCols, TypedProperties extraProps) throws IOException { - extraProps.setProperty(HoodieReaderConfig.FILE_GROUP_READER_ENABLED.key(), "false"); extraProps.setProperty("hoodie.datasource.write.table.type", tableType); extraProps.setProperty("hoodie.datasource.write.row.writer.enable", rowWriterEnable.toString()); extraProps.setProperty(DataSourceWriteOptions.SET_NULL_FOR_MISSING_COLUMNS().key(), Boolean.toString(nullForDeletedCols));