alexeykudinkin commented on a change in pull request #4818:
URL: https://github.com/apache/hudi/pull/4818#discussion_r815255066
##########
File path:
hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java
##########
@@ -348,6 +355,21 @@ protected HoodieWriteConfig getConfig(Boolean autoCommit,
Boolean rollbackUsingM
.withRollbackUsingMarkers(rollbackUsingMarkers);
}
+ protected Dataset<Row> toDataset(List<HoodieRecord> records) {
Review comment:
Good call
##########
File path:
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileScanRDD.scala
##########
@@ -18,56 +18,37 @@
package org.apache.hudi
-import org.apache.spark.{Partition, TaskContext}
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow}
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.execution.QueryExecutionException
-import org.apache.spark.sql.{Row, SparkSession}
import org.apache.spark.sql.execution.datasources.{FilePartition,
PartitionedFile, SchemaColumnConvertNotSupportedException}
-import org.apache.spark.sql.types.StructType
+import org.apache.spark.{Partition, TaskContext}
/**
- * Similar to [[org.apache.spark.sql.execution.datasources.FileScanRDD]].
- *
- * This class will extract the fields needed according to [[requiredColumns]]
and
- * return iterator of [[org.apache.spark.sql.Row]] directly.
+ * TODO eval if we actually need it
Review comment:
This would be cleaned up in a stacked on PR
##########
File path:
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileScanRDD.scala
##########
@@ -93,17 +74,8 @@ class HoodieFileScanRDD(
// Register an on-task-completion callback to close the input stream.
context.addTaskCompletionListener[Unit](_ => iterator.close())
- // extract required columns from row
- val iterAfterExtract = HoodieDataSourceHelper.extractRequiredSchema(
Review comment:
This utility itself would be cleaned up in a stacked PR
##########
File path:
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestParquetColumnProjection.scala
##########
@@ -0,0 +1,293 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.functional
+
+import org.apache.avro.Schema
+import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions,
DefaultSource, HoodieBaseRelation, HoodieSparkUtils, HoodieUnsafeRDD}
+import org.apache.hudi.common.config.HoodieMetadataConfig
+import org.apache.hudi.common.model.HoodieRecord
+import org.apache.hudi.common.testutils.{HadoopMapRedUtils,
HoodieTestDataGenerator}
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.keygen.NonpartitionedKeyGenerator
+import org.apache.hudi.testutils.SparkClientFunctionalTestHarness
+import org.apache.parquet.hadoop.util.counters.BenchmarkCounter
+import org.apache.spark.HoodieUnsafeRDDUtils
+import org.apache.spark.sql.{Dataset, Row, SaveMode}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.junit.jupiter.api.Assertions.assertEquals
+import org.junit.jupiter.api.{Tag, Test}
+
+import scala.collection.JavaConverters._
+
+@Tag("functional")
+class TestParquetColumnProjection extends SparkClientFunctionalTestHarness {
+
+ val defaultWriteOpts = Map(
+ "hoodie.insert.shuffle.parallelism" -> "4",
+ "hoodie.upsert.shuffle.parallelism" -> "4",
+ "hoodie.bulkinsert.shuffle.parallelism" -> "2",
+ "hoodie.delete.shuffle.parallelism" -> "1",
+ DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
+ DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
+ HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
+ HoodieMetadataConfig.ENABLE.key -> "true",
+ // NOTE: It's critical that we use non-partitioned table, since the way we
track amount of bytes read
+ // is not robust, and works most reliably only when we read just a
single file. As such, making table
+ // non-partitioned makes it much more likely just a single file will
be written
+ DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key ->
classOf[NonpartitionedKeyGenerator].getName
+ )
+
+ @Test
+ def testBaseFileOnlyViewRelation(): Unit = {
+ val tablePath = s"$basePath/cow"
+ val targetRecordsCount = 100
+ val (_, schema) = bootstrapTable(tablePath,
DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL, targetRecordsCount,
+ defaultWriteOpts, populateMetaFields = true)
+ val tableState = TableState(tablePath, schema, targetRecordsCount, 0.0)
+
+ // Stats for the reads fetching only _projected_ columns (note how amount
of bytes read
+ // increases along w/ the # of columns)
+ val projectedColumnsReadStats: Array[(String, Long)] = Array(
+ ("rider", 2452),
+ ("rider,driver", 2552),
+ ("rider,driver,tip_history", 3517)
+ )
+
+ // Test COW / Snapshot
+ runTest(tableState, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL, "",
projectedColumnsReadStats)
+ }
+
+ @Test
+ def testMergeOnReadSnapshotRelationWithDeltaLogs(): Unit = {
+ val tablePath = s"$basePath/mor-with-logs"
+ val targetRecordsCount = 100
+ val targetUpdatedRecordsRatio = 0.5
+
+ val (_, schema) = bootstrapMORTable(tablePath, targetRecordsCount,
targetUpdatedRecordsRatio, defaultWriteOpts, populateMetaFields = true)
+ val tableState = TableState(tablePath, schema, targetRecordsCount,
targetUpdatedRecordsRatio)
+
+ // Stats for the reads fetching only _projected_ columns (note how amount
of bytes read
+ // increases along w/ the # of columns)
+ val projectedColumnsReadStats: Array[(String, Long)] = Array(
+ ("rider", 2452),
+ ("rider,driver", 2552),
+ ("rider,driver,tip_history", 3517)
+ )
+
+ // Stats for the reads fetching _all_ columns (note, how amount of bytes
read
+ // is invariant of the # of columns)
+ val fullColumnsReadStats: Array[(String, Long)] = Array(
+ ("rider", 14665),
+ ("rider,driver", 14665),
+ ("rider,driver,tip_history", 14665)
+ )
+
+ // Test MOR / Snapshot / Skip-merge
+ runTest(tableState, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL,
DataSourceReadOptions.REALTIME_SKIP_MERGE_OPT_VAL, projectedColumnsReadStats)
+
+ // Test MOR / Snapshot / Payload-combine
+ runTest(tableState, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL,
DataSourceReadOptions.REALTIME_PAYLOAD_COMBINE_OPT_VAL, fullColumnsReadStats)
+
+ // Test MOR / Read Optimized
+ runTest(tableState,
DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL, "null",
projectedColumnsReadStats)
+ }
+
+ @Test
+ def testMergeOnReadSnapshotRelationWithNoDeltaLogs(): Unit = {
+ val tablePath = s"$basePath/mor-no-logs"
+ val targetRecordsCount = 100
+ val targetUpdatedRecordsRatio = 0.0
+
+ val (_, schema) = bootstrapMORTable(tablePath, targetRecordsCount,
targetUpdatedRecordsRatio, defaultWriteOpts, populateMetaFields = true)
+ val tableState = TableState(tablePath, schema, targetRecordsCount,
targetUpdatedRecordsRatio)
+
+ //
+ // Test #1: MOR table w/ Delta Logs
+ //
+
+ // Stats for the reads fetching only _projected_ columns (note how amount
of bytes read
+ // increases along w/ the # of columns)
+ val projectedColumnsReadStats: Array[(String, Long)] = Array(
+ ("rider", 2452),
+ ("rider,driver", 2552),
+ ("rider,driver,tip_history", 3517)
+ )
+
+ // Test MOR / Snapshot / Skip-merge
+ runTest(tableState, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL,
DataSourceReadOptions.REALTIME_SKIP_MERGE_OPT_VAL, projectedColumnsReadStats)
+
+ // Test MOR / Snapshot / Payload-combine
+ runTest(tableState, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL,
DataSourceReadOptions.REALTIME_PAYLOAD_COMBINE_OPT_VAL,
projectedColumnsReadStats)
+
+ // Test MOR / Read Optimized
+ runTest(tableState,
DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL, "null",
projectedColumnsReadStats)
+ }
+
+ @Test
+ def testMergeOnReadIncrementalRelationWithDeltaLogs(): Unit = {
+ val tablePath = s"$basePath/mor-with-logs"
+ val targetRecordsCount = 100
+ val targetUpdatedRecordsRatio = 0.5
+
+ val (_, schema) = bootstrapMORTable(tablePath, targetRecordsCount,
targetUpdatedRecordsRatio, defaultWriteOpts, populateMetaFields = true)
+ val tableState = TableState(tablePath, schema, targetRecordsCount,
targetUpdatedRecordsRatio)
+
+ // Stats for the reads fetching only _projected_ columns (note how amount
of bytes read
+ // increases along w/ the # of columns)
+ val projectedColumnsReadStats: Array[(String, Long)] = Array(
+ ("rider", 2560),
+ ("rider,driver", 2660),
+ ("rider,driver,tip_history", 3625)
+ )
+
+ // Stats for the reads fetching _all_ columns (note, how amount of bytes
read
+ // is invariant of the # of columns)
+ val fullColumnsReadStats: Array[(String, Long)] = Array(
+ ("rider", 14667),
+ ("rider,driver", 14667),
+ ("rider,driver,tip_history", 14667)
+ )
+
+ val incrementalOpts: Map[String, String] = Map(
+ DataSourceReadOptions.BEGIN_INSTANTTIME.key -> "001"
+ )
+
+ // Test MOR / Incremental / Skip-merge
+ runTest(tableState, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL,
DataSourceReadOptions.REALTIME_SKIP_MERGE_OPT_VAL,
+ projectedColumnsReadStats, incrementalOpts)
+
+ // Test MOR / Incremental / Payload-combine
+ runTest(tableState, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL,
DataSourceReadOptions.REALTIME_PAYLOAD_COMBINE_OPT_VAL,
+ fullColumnsReadStats, incrementalOpts)
+ }
+
+
+ // Test routine
+ private def runTest(tableState: TableState,
+ queryType: String,
+ mergeType: String,
+ expectedStats: Array[(String, Long)],
+ additionalOpts: Map[String, String] = Map.empty): Unit =
{
+ val tablePath = tableState.path
+ val readOpts = defaultWriteOpts ++ Map(
+ "path" -> tablePath,
+ DataSourceReadOptions.QUERY_TYPE.key -> queryType,
+ DataSourceReadOptions.REALTIME_MERGE.key -> mergeType
+ ) ++ additionalOpts
+
+ val ds = new DefaultSource()
+ val relation: HoodieBaseRelation = ds.createRelation(spark.sqlContext,
readOpts).asInstanceOf[HoodieBaseRelation]
+
+ for ((columnListStr, expectedBytesRead) <- expectedStats) {
+ val targetColumns = columnListStr.split(",")
+
+ println(s"Running test for $tablePath / $queryType / $mergeType /
$columnListStr")
+
+ val (rows, bytesRead) = measureBytesRead { () =>
+ val rdd = relation.buildScan(targetColumns,
Array.empty).asInstanceOf[HoodieUnsafeRDD]
+ HoodieUnsafeRDDUtils.collect(rdd)
+ }
+
+ val targetRecordCount = tableState.targetRecordCount;
+ val targetUpdatedRecordsRatio = tableState.targetUpdatedRecordsRatio
+
+ val expectedRecordCount =
+ if
(DataSourceReadOptions.REALTIME_SKIP_MERGE_OPT_VAL.equals(mergeType))
targetRecordCount * (1 + targetUpdatedRecordsRatio)
+ else targetRecordCount
+
+ assertEquals(expectedRecordCount, rows.length)
+ assertEquals(expectedBytesRead, bytesRead)
+
+ val readColumns = targetColumns ++ relation.mandatoryColumns
+ val (_, projectedStructType) =
HoodieSparkUtils.getRequiredSchema(tableState.schema, readColumns)
+
+ val row: InternalRow = rows.take(1).head
+
+ // This check is mostly about making sure InternalRow deserializes
properly into projected schema
+ val deserializedColumns = row.toSeq(projectedStructType)
+ assertEquals(readColumns.length, deserializedColumns.size)
+ }
+ }
+
+ private def bootstrapTable(path: String,
+ tableType: String,
+ recordCount: Int,
+ opts: Map[String, String],
+ populateMetaFields: Boolean,
+ dataGenOpt: Option[HoodieTestDataGenerator] =
None): (List[HoodieRecord[_]], Schema) = {
+ val dataGen = dataGenOpt.getOrElse(new HoodieTestDataGenerator(0x12345))
+
+ // Bulk Insert Operation
+ val schema =
+ if (populateMetaFields)
HoodieTestDataGenerator.AVRO_SCHEMA_WITH_METADATA_FIELDS
+ else HoodieTestDataGenerator.AVRO_SCHEMA
+
+ val records = dataGen.generateInserts("001", recordCount)
+ val inputDF: Dataset[Row] = toDataset(records)
+
+ inputDF.write.format("org.apache.hudi")
+ .options(opts)
+ .option(DataSourceWriteOptions.TABLE_TYPE.key, tableType)
+ .option(DataSourceWriteOptions.OPERATION.key,
DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
+ .mode(SaveMode.Overwrite)
+ .save(path)
+
+ (records.asScala.toList, schema)
+ }
+
+ private def bootstrapMORTable(path: String,
+ recordCount: Int,
+ updatedRecordsRatio: Double,
+ opts: Map[String, String],
+ populateMetaFields: Boolean,
+ dataGenOpt: Option[HoodieTestDataGenerator] =
None): (List[HoodieRecord[_]], Schema) = {
+ val dataGen = dataGenOpt.getOrElse(new HoodieTestDataGenerator(0x12345))
+
+ // Step 1: Bootstrap table w/ N records (t/h bulk-insert)
+ val (insertedRecords, schema) = bootstrapTable(path,
DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL, recordCount, opts,
populateMetaFields, Some(dataGen))
+
+ if (updatedRecordsRatio == 0) {
+ (insertedRecords, schema)
+ } else {
+ val updatesCount = (insertedRecords.length * updatedRecordsRatio).toInt
+ val recordsToUpdate = insertedRecords.take(updatesCount)
+ val updatedRecords = dataGen.generateUpdates("002",
recordsToUpdate.asJava)
+
+ // Step 2: Update M records out of those (t/h update)
+ val inputDF = toDataset(updatedRecords)
+
+ inputDF.write.format("org.apache.hudi")
+ .options(opts)
+ .option(DataSourceWriteOptions.OPERATION.key,
DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
+ .mode(SaveMode.Append)
+ .save(path)
+
+ (updatedRecords.asScala.toList ++ insertedRecords.drop(updatesCount),
schema)
+ }
+ }
+
+ def measureBytesRead[T](f: () => T): (T, Int) = {
Review comment:
Yeah, it's not without its limit though: it only works for **single
file**. I've found some metrics that we can peek into w/in Spark tasks
themselves (though still relying on Hadoop) but didn't want to over-invest in
it right away
##########
File path:
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala
##########
@@ -148,20 +140,20 @@ class MergeOnReadIncrementalRelation(sqlContext:
SQLContext,
hadoopConf = new Configuration(conf)
)
- val hoodieTableState = HoodieMergeOnReadTableState(fileIndex,
HoodieRecord.RECORD_KEY_METADATA_FIELD, preCombineFieldOpt)
+ val hoodieTableState =
HoodieTableState(HoodieRecord.RECORD_KEY_METADATA_FIELD, preCombineFieldOpt)
// TODO implement incremental span record filtering w/in RDD to make
sure returned iterator is appropriately
// filtered, since file-reader might not be capable to perform
filtering
- val rdd = new HoodieMergeOnReadRDD(
+ new HoodieMergeOnReadRDD(
sqlContext.sparkContext,
jobConf,
fullSchemaParquetReader,
requiredSchemaParquetReader,
hoodieTableState,
tableSchema,
- requiredSchema
+ requiredSchema,
+ fileIndex
Review comment:
Already addressed in a stacked PR
##########
File path:
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/BaseFileOnlyViewRelation.scala
##########
@@ -89,18 +108,46 @@ class BaseFileOnlyViewRelation(
inMemoryFileIndex.listFiles(partitionFilters, dataFilters)
}
- val partitionFiles = partitionDirectories.flatMap { partition =>
+ val partitions = partitionDirectories.flatMap { partition =>
partition.files.flatMap { file =>
+ // TODO move to adapter
+ // TODO fix, currently assuming parquet as underlying format
Review comment:
It doesn't work for MT
##########
File path:
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/BaseFileOnlyViewRelation.scala
##########
@@ -89,18 +108,46 @@ class BaseFileOnlyViewRelation(
inMemoryFileIndex.listFiles(partitionFilters, dataFilters)
}
- val partitionFiles = partitionDirectories.flatMap { partition =>
+ val partitions = partitionDirectories.flatMap { partition =>
partition.files.flatMap { file =>
+ // TODO move to adapter
+ // TODO fix, currently assuming parquet as underlying format
HoodieDataSourceHelper.splitFiles(
sparkSession = sparkSession,
file = file,
- partitionValues = partition.values
+ // TODO clarify why this is required
+ partitionValues = InternalRow.empty
Review comment:
We'd want to preserve API as close to the source as possible
##########
File path:
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileScanRDD.scala
##########
@@ -93,17 +74,8 @@ class HoodieFileScanRDD(
// Register an on-task-completion callback to close the input stream.
context.addTaskCompletionListener[Unit](_ => iterator.close())
- // extract required columns from row
- val iterAfterExtract = HoodieDataSourceHelper.extractRequiredSchema(
Review comment:
It's working, but not in the way we need: currently we read base-file's
in full-schema, then do projection. Instead we should read _only_ projected
schema right away (since Parquet reader supports that)
##########
File path:
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/BaseFileOnlyViewRelation.scala
##########
@@ -89,18 +108,46 @@ class BaseFileOnlyViewRelation(
inMemoryFileIndex.listFiles(partitionFilters, dataFilters)
}
- val partitionFiles = partitionDirectories.flatMap { partition =>
+ val partitions = partitionDirectories.flatMap { partition =>
partition.files.flatMap { file =>
+ // TODO move to adapter
+ // TODO fix, currently assuming parquet as underlying format
HoodieDataSourceHelper.splitFiles(
sparkSession = sparkSession,
file = file,
- partitionValues = partition.values
+ // TODO clarify why this is required
+ partitionValues = InternalRow.empty
)
}
}
- partitionFiles.map{ f =>
- PartitionedFile(InternalRow.empty, f.filePath, f.start, f.length)
+ val maxSplitBytes = sparkSession.sessionState.conf.filesMaxPartitionBytes
+
+ sparkAdapter.getFilePartitions(sparkSession, partitions, maxSplitBytes)
+ }
+
+ private def convertToExpressions(filters: Array[Filter]): Array[Expression]
= {
+ val catalystExpressions =
HoodieSparkUtils.convertToCatalystExpressions(filters, tableStructSchema)
+
+ val failedExprs = catalystExpressions.zipWithIndex.filter { case (opt, _)
=> opt.isEmpty }
+ if (failedExprs.nonEmpty) {
+ val failedFilters = failedExprs.map(p => filters(p._2))
+ logWarning(s"Failed to convert Filters into Catalyst expressions
(${failedFilters.map(_.toString)})")
}
+
+ catalystExpressions.filter(_.isDefined).map(_.get).toArray
}
+
+ /**
+ * Checks whether given expression only references only references partition
columns
+ * (and involves no sub-query)
+ */
+ private def isPartitionPredicate(condition: Expression): Boolean = {
Review comment:
Streamlining previous impl
--
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.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]