You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2022/02/26 01:14:11 UTC

[GitHub] [hudi] nsivabalan commented on a change in pull request #4818: [HUDI-3396] Make sure `BaseFileOnlyViewRelation` only reads projected columns

nsivabalan commented on a change in pull request #4818:
URL: https://github.com/apache/hudi/pull/4818#discussion_r815229255



##########
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:
       can we take in AvroSchema as an argument. may be create another overloaded method which calls into this w/ some default for avro schema.

##########
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:
       I see why we are doing this. do you think we can fix 
   HoodieDataSourceHelper.splitFiles() only to not take in the last arg and directly set InternalRow.empty when creating Partitionedfile. I guess this is the only usage/caller. so should be safe to do. 

##########
File path: hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDataSourceHelper.scala
##########
@@ -33,43 +33,6 @@ import scala.collection.JavaConverters._
 
 object HoodieDataSourceHelper extends PredicateHelper {
 
-  /**
-   * Partition the given condition into two sequence of conjunctive predicates:
-   * - predicates that can be evaluated using metadata only.
-   * - other predicates.
-   */
-  def splitPartitionAndDataPredicates(
-      spark: SparkSession,
-      condition: Expression,
-      partitionColumns: Seq[String]): (Seq[Expression], Seq[Expression]) = {
-    splitConjunctivePredicates(condition).partition(
-      isPredicateMetadataOnly(spark, _, partitionColumns))
-  }
-
-  /**
-   * Check if condition can be evaluated using only metadata. In Delta, this means the condition
-   * only references partition columns and involves no subquery.
-   */
-  def isPredicateMetadataOnly(

Review comment:
       guess it was copied over from here. 

##########
File path: hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieUnsafeRDD.scala
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.{Partition, SparkContext, TaskContext}
+
+/**
+ * !!! PLEASE READ CAREFULLY !!!
+ *
+ * Base class for all of the custom low-overhead RDD implementations for Hudi.
+ *
+ * To keep memory allocation footprint as low as possible, each inheritor of this RDD base class
+ *
+ * <pre>
+ *   1. Does NOT deserialize from [[InternalRow]] to [[Row]] (therefore only providing access to
+ *   Catalyst internal representations (often mutable) of the read row)
+ *
+ *   2. DOES NOT COPY UNDERLYING ROW OUT OF THE BOX, meaning that
+ *
+ *      a) access to this RDD is NOT thread-safe
+ *
+ *      b) iterating over it reference to a _mutable_ underlying instance (of [[InternalRow]]) is
+ *      returned, entailing that after [[Iterator#next()]] is invoked on the provided iterator,
+ *      previous reference becomes **invalid**. Therefore, you will have to copy underlying mutable
+ *      instance of [[InternalRow]] if you plan to access it after [[Iterator#next()]] is invoked (filling
+ *      it with the next row's payload)
+ *
+ *      c) due to item b) above, no operation other than the iteration will produce meaningful
+ *      results on it and will likely fail [1]
+ * </pre>
+ *
+ * [1] For example, [[RDD#collect]] method on this implementation would not work correctly, as it's
+ * simply using Scala's default [[Iterator#toArray]] method which will simply concat all the references onto
+ * the same underlying mutable object into [[Array]]. Instead each individual [[InternalRow]] _has to be copied_,
+ * before concatenating into the final output. Please refer to [[HoodieRDDUtils#collect]] for more details.

Review comment:
       minor: HoodieUnsafeRDDUtils
   

##########
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:
       do you mean to say, need to evaluate if we need this entire class itself ?

##########
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:
       is this something being added in this patch? 

##########
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:
       may be we can rename fileIndex to something like "morFileSplits" since it represents a list of HoodieMergeOnReadFileSplit

##########
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:
       nice. I didn't know we can do something like this.

##########
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:
       I see we don't ever call into this 
   HoodieDataSourceHelper.extractRequiredSchema(
   anywhere in this patch. So, this wasn't effective at all prior to this patch? 
   bcoz, in this patch, I don't see any special handling and just setting requiredcolumns to buildScan() works out of the box. 
   can you confirm my understanding please.

##########
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:
       how come it worked for metadata table then ? did we not try read optimized query for metadata table then ? 




-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org