garyli1019 commented on a change in pull request #1702: URL: https://github.com/apache/hudi/pull/1702#discussion_r439878477
########## File path: hudi-spark/src/main/scala/org/apache/hudi/HudiBootstrapRDD.scala ########## @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi + +import org.apache.spark.{Partition, TaskContext} +import org.apache.spark.rdd.RDD +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.types.StructType +import org.apache.spark.sql.vectorized.ColumnarBatch + +class HudiBootstrapRDD(@transient spark: SparkSession, + dataReadFunction: PartitionedFile => Iterator[Any], + skeletonReadFunction: PartitionedFile => Iterator[Any], + regularReadFunction: PartitionedFile => Iterator[Any], + dataSchema: StructType, + skeletonSchema: StructType, + requiredColumns: Array[String], + tableState: HudiBootstrapTableState) + extends RDD[InternalRow](spark.sparkContext, Nil) { + + override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = { + val bootstrapPartition = split.asInstanceOf[HudiBootstrapPartition] + + if (log.isDebugEnabled) { + if (bootstrapPartition.split.skeletonFile.isDefined) { + logDebug("Got Split => Index: " + bootstrapPartition.index + ", Data File: " + + bootstrapPartition.split.dataFile.filePath + ", Skeleton File: " + + bootstrapPartition.split.skeletonFile.get.filePath) + } else { + logDebug("Got Split => Index: " + bootstrapPartition.index + ", Data File: " + + bootstrapPartition.split.dataFile.filePath) + } + } + + var partitionedFileIterator: Iterator[InternalRow] = null + + if (bootstrapPartition.split.skeletonFile.isDefined) { + // It is a bootstrap split. Check both skeleton and data files. + if (dataSchema.isEmpty) { + // No data column to fetch, hence fetch only from skeleton file + partitionedFileIterator = read(bootstrapPartition.split.skeletonFile.get, skeletonReadFunction) + } else if (skeletonSchema.isEmpty) { + // No metadata column to fetch, hence fetch only from data file + partitionedFileIterator = read(bootstrapPartition.split.dataFile, dataReadFunction) + } else { + // Fetch from both data and skeleton file, and merge + val dataFileIterator = read(bootstrapPartition.split.dataFile, dataReadFunction) + val skeletonFileIterator = read(bootstrapPartition.split.skeletonFile.get, skeletonReadFunction) + partitionedFileIterator = merge(skeletonFileIterator, dataFileIterator) + } + } else { + partitionedFileIterator = read(bootstrapPartition.split.dataFile, regularReadFunction) + } + partitionedFileIterator + } + + def merge(skeletonFileIterator: Iterator[InternalRow], dataFileIterator: Iterator[InternalRow]) + : Iterator[InternalRow] = { + new Iterator[InternalRow] { + override def hasNext: Boolean = dataFileIterator.hasNext && skeletonFileIterator.hasNext + override def next(): InternalRow = { + mergeInternalRow(skeletonFileIterator.next(), dataFileIterator.next()) + } + } + } + + def mergeInternalRow(skeletonRow: InternalRow, dataRow: InternalRow): InternalRow = { + val skeletonArr = skeletonRow.copy().toSeq(skeletonSchema) + val dataArr = dataRow.copy().toSeq(dataSchema) + // We need to return it in the order requested + val mergedArr = requiredColumns.map(col => { + if (skeletonSchema.fieldNames.contains(col)) { + val idx = skeletonSchema.fieldIndex(col) + skeletonArr(idx) + } else { + val idx = dataSchema.fieldIndex(col) + dataArr(idx) + } + }) + + logDebug("Merged data and skeleton values => " + mergedArr.mkString(",")) + val mergedRow = InternalRow.fromSeq(mergedArr) + mergedRow + } + + def read(partitionedFile: PartitionedFile, readFileFunction: PartitionedFile => Iterator[Any]) + : Iterator[InternalRow] = { + val fileIterator = readFileFunction(partitionedFile) + + import scala.collection.JavaConverters._ + + val rows = fileIterator.flatMap(_ match { + case r: InternalRow => Seq(r) + case b: ColumnarBatch => b.rowIterator().asScala Review comment: If we use vectorized reader this way, does it still have a huge performance boost? From my understanding, the regular reader iterator will read the whole row as `UnsafeRow` then do the column pruning before load it into memory. The vectorized reader will do the column pruning and loading data in one step. So theoretically vectorized reader would still be faster even we read it as `InternalRow` The description I found from Spark code `This class can either return InternalRows or ColumnarBatches. With whole stage codegen enabled, this class returns ColumnarBatches which offers significant performance gains.` ########## File path: hudi-spark/src/main/scala/org/apache/hudi/IncrementalRelation.scala ########## @@ -71,13 +78,16 @@ class IncrementalRelation(val sqlContext: SQLContext, optParams.getOrElse(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, lastInstant.getTimestamp)) .getInstants.iterator().toList - // use schema from latest metadata, if not present, read schema from the data file - private val latestSchema = { - val schemaUtil = new TableSchemaResolver(metaClient) - val tableSchema = HoodieAvroUtils.createHoodieWriteSchema(schemaUtil.getTableAvroSchemaWithoutMetadataFields); - AvroConversionUtils.convertAvroSchemaToStructType(tableSchema) + // use schema from a file produced in the latest instant + val latestSchema: StructType = { + log.info("Inferring schema..") + val schemaResolver = new TableSchemaResolver(metaClient) + val tableSchema = schemaResolver.getTableAvroSchemaWithoutMetadataFields + val dataSchema = AvroConversionUtils.convertAvroSchemaToStructType(tableSchema) + StructType(skeletonSchema.fields ++ dataSchema.fields) Review comment: Are we changing the metadata columns? ########## File path: hudi-spark/src/main/scala/org/apache/hudi/IncrementalRelation.scala ########## @@ -92,36 +102,69 @@ class IncrementalRelation(val sqlContext: SQLContext, override def schema: StructType = latestSchema override def buildScan(): RDD[Row] = { - val fileIdToFullPath = mutable.HashMap[String, String]() + val regularFileIdToFullPath = mutable.HashMap[String, String]() + var metaBootstrapFileIdToFullPath = mutable.HashMap[String, String]() + for (commit <- commitsToReturn) { val metadata: HoodieCommitMetadata = HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commit) .get, classOf[HoodieCommitMetadata]) - fileIdToFullPath ++= metadata.getFileIdAndFullPaths(basePath).toMap + + if (HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS == commit.getTimestamp) { + metaBootstrapFileIdToFullPath ++= metadata.getFileIdAndFullPaths(basePath).toMap + } else { + regularFileIdToFullPath ++= metadata.getFileIdAndFullPaths(basePath).toMap + } + } + + if (metaBootstrapFileIdToFullPath.nonEmpty) { + // filer out meta bootstrap files that have had more commits since metadata bootstrap + metaBootstrapFileIdToFullPath = metaBootstrapFileIdToFullPath + .filterNot(fileIdFullPath => regularFileIdToFullPath.contains(fileIdFullPath._1)) } + val pathGlobPattern = optParams.getOrElse( DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY, DataSourceReadOptions.DEFAULT_INCR_PATH_GLOB_OPT_VAL) - val filteredFullPath = if(!pathGlobPattern.equals(DataSourceReadOptions.DEFAULT_INCR_PATH_GLOB_OPT_VAL)) { - val globMatcher = new GlobPattern("*" + pathGlobPattern) - fileIdToFullPath.filter(p => globMatcher.matches(p._2)) - } else { - fileIdToFullPath + val (filteredRegularFullPaths, filteredMetaBootstrapFullPaths) = { + if(!pathGlobPattern.equals(DataSourceReadOptions.DEFAULT_INCR_PATH_GLOB_OPT_VAL)) { + val globMatcher = new GlobPattern("*" + pathGlobPattern) + (regularFileIdToFullPath.filter(p => globMatcher.matches(p._2)).values, + metaBootstrapFileIdToFullPath.filter(p => globMatcher.matches(p._2)).values) + } else { + (regularFileIdToFullPath.values, metaBootstrapFileIdToFullPath.values) + } } // unset the path filter, otherwise if end_instant_time is not the latest instant, path filter set for RO view // will filter out all the files incorrectly. sqlContext.sparkContext.hadoopConfiguration.unset("mapreduce.input.pathFilter.class") val sOpts = optParams.filter(p => !p._1.equalsIgnoreCase("path")) - if (filteredFullPath.isEmpty) { + if (filteredRegularFullPaths.isEmpty && filteredMetaBootstrapFullPaths.isEmpty) { sqlContext.sparkContext.emptyRDD[Row] } else { log.info("Additional Filters to be applied to incremental source are :" + filters) - filters.foldLeft(sqlContext.read.options(sOpts) - .schema(latestSchema) - .parquet(filteredFullPath.values.toList: _*) - .filter(String.format("%s >= '%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitsToReturn.head.getTimestamp)) - .filter(String.format("%s <= '%s'", - HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitsToReturn.last.getTimestamp)))((e, f) => e.filter(f)) - .toDF().rdd + + var df: DataFrame = sqlContext.createDataFrame(sqlContext.sparkContext.emptyRDD[Row], latestSchema) Review comment: Should we avoid `var` here? seem like avoidable. ########## File path: hudi-spark/src/main/scala/org/apache/hudi/HudiBootstrapRDD.scala ########## @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi + +import org.apache.spark.{Partition, TaskContext} +import org.apache.spark.rdd.RDD +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.types.StructType +import org.apache.spark.sql.vectorized.ColumnarBatch + +class HudiBootstrapRDD(@transient spark: SparkSession, + dataReadFunction: PartitionedFile => Iterator[Any], + skeletonReadFunction: PartitionedFile => Iterator[Any], + regularReadFunction: PartitionedFile => Iterator[Any], + dataSchema: StructType, + skeletonSchema: StructType, + requiredColumns: Array[String], + tableState: HudiBootstrapTableState) + extends RDD[InternalRow](spark.sparkContext, Nil) { + + override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = { + val bootstrapPartition = split.asInstanceOf[HudiBootstrapPartition] + + if (log.isDebugEnabled) { + if (bootstrapPartition.split.skeletonFile.isDefined) { + logDebug("Got Split => Index: " + bootstrapPartition.index + ", Data File: " + + bootstrapPartition.split.dataFile.filePath + ", Skeleton File: " + + bootstrapPartition.split.skeletonFile.get.filePath) + } else { + logDebug("Got Split => Index: " + bootstrapPartition.index + ", Data File: " + + bootstrapPartition.split.dataFile.filePath) + } + } + + var partitionedFileIterator: Iterator[InternalRow] = null + + if (bootstrapPartition.split.skeletonFile.isDefined) { + // It is a bootstrap split. Check both skeleton and data files. + if (dataSchema.isEmpty) { + // No data column to fetch, hence fetch only from skeleton file + partitionedFileIterator = read(bootstrapPartition.split.skeletonFile.get, skeletonReadFunction) + } else if (skeletonSchema.isEmpty) { + // No metadata column to fetch, hence fetch only from data file + partitionedFileIterator = read(bootstrapPartition.split.dataFile, dataReadFunction) Review comment: Is this the regular COW table data files? ########## File path: hudi-spark/src/main/scala/org/apache/hudi/HudiBootstrapRDD.scala ########## @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi + +import org.apache.spark.{Partition, TaskContext} +import org.apache.spark.rdd.RDD +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.types.StructType +import org.apache.spark.sql.vectorized.ColumnarBatch + +class HudiBootstrapRDD(@transient spark: SparkSession, + dataReadFunction: PartitionedFile => Iterator[Any], + skeletonReadFunction: PartitionedFile => Iterator[Any], + regularReadFunction: PartitionedFile => Iterator[Any], + dataSchema: StructType, + skeletonSchema: StructType, + requiredColumns: Array[String], + tableState: HudiBootstrapTableState) + extends RDD[InternalRow](spark.sparkContext, Nil) { + + override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = { + val bootstrapPartition = split.asInstanceOf[HudiBootstrapPartition] + + if (log.isDebugEnabled) { + if (bootstrapPartition.split.skeletonFile.isDefined) { + logDebug("Got Split => Index: " + bootstrapPartition.index + ", Data File: " + + bootstrapPartition.split.dataFile.filePath + ", Skeleton File: " + + bootstrapPartition.split.skeletonFile.get.filePath) + } else { + logDebug("Got Split => Index: " + bootstrapPartition.index + ", Data File: " + + bootstrapPartition.split.dataFile.filePath) + } + } + + var partitionedFileIterator: Iterator[InternalRow] = null + + if (bootstrapPartition.split.skeletonFile.isDefined) { + // It is a bootstrap split. Check both skeleton and data files. + if (dataSchema.isEmpty) { + // No data column to fetch, hence fetch only from skeleton file + partitionedFileIterator = read(bootstrapPartition.split.skeletonFile.get, skeletonReadFunction) Review comment: IIUC, the skeleton file will only exist if there is a data file. Did I miss anything that there is a case with a stand-alone skeleton file? ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected]
