umehrot2 commented on a change in pull request #1848:
URL: https://github.com/apache/hudi/pull/1848#discussion_r459729836



##########
File path: hudi-spark/src/main/scala/org/apache/hudi/HudiMergeOnReadRDD.scala
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.hudi.common.fs.FSUtils
+import org.apache.hudi.common.table.log.{HoodieMergedLogRecordScanner, 
LogReaderUtils}
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.hadoop.config.{HadoopSerializableConfiguration, 
HoodieRealtimeConfig}
+import 
org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.HOODIE_RECORD_KEY_COL_POS
+
+import org.apache.avro.Schema
+import org.apache.hadoop.conf.Configuration
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.{Partition, SparkContext, TaskContext}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.avro.{AvroDeserializer, SchemaConverters}
+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
+
+import scala.collection.JavaConverters._
+import scala.util.Try
+
+case class HudiMergeOnReadPartition(index: Int, split: 
HudiMergeOnReadFileSplit) extends Partition
+
+class HudiMergeOnReadRDD(sc: SparkContext,
+                         broadcastedConf: 
Broadcast[HadoopSerializableConfiguration],
+                         dataReadFunction: PartitionedFile => Iterator[Any],
+                         dataSchema: StructType,
+                         hudiRealtimeFileSplits: 
List[HudiMergeOnReadFileSplit])
+  extends RDD[InternalRow](sc, Nil) {
+
+  // Broadcast the hadoop Configuration to executors.
+  def this(sc: SparkContext,
+           config: Configuration,
+           dataReadFunction: PartitionedFile => Iterator[Any],
+           dataSchema: StructType,
+           hudiRealtimeFileSplits: List[HudiMergeOnReadFileSplit]) = {
+    this(
+      sc,
+      sc.broadcast(new HadoopSerializableConfiguration(config))
+      .asInstanceOf[Broadcast[HadoopSerializableConfiguration]],
+      dataReadFunction,
+      dataSchema,
+      hudiRealtimeFileSplits)
+  }
+
+  override def compute(split: Partition, context: TaskContext): 
Iterator[InternalRow] = {
+    val mergeParquetPartition = split.asInstanceOf[HudiMergeOnReadPartition]
+    mergeParquetPartition.split match {
+      case dataFileOnlySplit if dataFileOnlySplit.logPaths.isEmpty =>
+        read(mergeParquetPartition.split.dataFile, dataReadFunction)
+      case unMergeSplit if unMergeSplit.skipMerge =>
+        unMergeFileIterator(mergeParquetPartition.split, dataReadFunction)
+      case mergeSplit if !mergeSplit.skipMerge =>
+        mergeFileIterator(mergeParquetPartition.split, dataReadFunction)
+      case _ => throw new HoodieException("Unable to select an Iterator to 
read the Hudi MOR File Split")
+    }
+  }
+
+  override protected def getPartitions: Array[Partition] = {
+    hudiRealtimeFileSplits.zipWithIndex.map(file => 
HudiMergeOnReadPartition(file._2, file._1)).toArray
+  }
+
+  private def getConfig(): Configuration = {
+    broadcastedConf.value.config
+  }
+
+  private def read(partitionedFile: PartitionedFile,
+                   readFileFunction: PartitionedFile => Iterator[Any]): 
Iterator[InternalRow] = {
+    val fileIterator = readFileFunction(partitionedFile)
+    val rows = fileIterator.flatMap(_ match {
+      case r: InternalRow => Seq(r)
+      case b: ColumnarBatch => b.rowIterator().asScala
+    })
+    rows
+  }
+
+  private def unMergeFileIterator(split: HudiMergeOnReadFileSplit,
+                                  readFileFunction: PartitionedFile => 
Iterator[Any]): Iterator[InternalRow] =
+    new Iterator[InternalRow] {
+      private val dataFileIterator = read(split.dataFile, readFileFunction)
+      private val logSchema = getLogAvroSchema(split)
+      private val sparkTypes = 
SchemaConverters.toSqlType(logSchema).dataType.asInstanceOf[StructType]
+      private val converter = new AvroDeserializer(logSchema, sparkTypes)
+      private val hudiLogRecords = scanLog(split, logSchema).getRecords
+      private var hudiLogRecordsIterator = 
hudiLogRecords.keySet().iterator().asScala
+
+      override def hasNext: Boolean = {
+        dataFileIterator.hasNext || hudiLogRecordsIterator.hasNext
+      }
+
+      override def next(): InternalRow = {
+        if (dataFileIterator.hasNext) {
+          dataFileIterator.next()
+        } else {
+          val curAvrokey = hudiLogRecordsIterator.next()
+          val curAvroRecord = 
hudiLogRecords.get(curAvrokey).getData.getInsertValue(logSchema).get()
+          converter.deserialize(curAvroRecord).asInstanceOf[InternalRow]
+        }
+      }
+    }
+
+  private def mergeFileIterator(split: HudiMergeOnReadFileSplit,
+                                readFileFunction: PartitionedFile => 
Iterator[Any]): Iterator[InternalRow] =
+    new Iterator[InternalRow] {
+      private val dataFileIterator = read(split.dataFile, readFileFunction)
+      private val logSchema = getLogAvroSchema(split)
+      private val sparkTypes = 
SchemaConverters.toSqlType(logSchema).dataType.asInstanceOf[StructType]
+      private val converter = new AvroDeserializer(logSchema, sparkTypes)
+      private val hudiLogRecords = scanLog(split, logSchema).getRecords
+      private var parquetFinished = false
+      private var logRecordToRead = hudiLogRecords.keySet()
+      private var hudiLogRecordsIterator: Iterator[String] = _
+
+      override def hasNext: Boolean = {
+        if (dataFileIterator.hasNext) {
+          true
+        } else {
+          if (!parquetFinished) {
+            parquetFinished = true
+            hudiLogRecordsIterator = logRecordToRead.iterator().asScala
+          }
+          hudiLogRecordsIterator.hasNext

Review comment:
       @vinothchandar @garyli1019 Spark's FileFormat uses can split the parquet 
files up if there are multiple row groups in the file. In Spark's 
implementation one `PartitionedFile` is not a complete file, that is why you 
can see the start position and length in that.
   
   But in both this implementation and in bootstrap case we are packaging 
complete file as a split, to avoid the complexity of dealing with partial 
splits.




----------------------------------------------------------------
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:
us...@infra.apache.org


Reply via email to