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



##########
File path: hudi-spark/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDD.scala
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.config.SerializableConfiguration
+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.HoodieRealtimeConfig
+import 
org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.HOODIE_RECORD_KEY_COL_POS
+
+import org.apache.avro.Schema
+import org.apache.avro.generic.{GenericRecord, IndexedRecord}
+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, AvroSerializer, 
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 HoodieMergeOnReadPartition(index: Int, split: 
HoodieMergeOnReadFileSplit) extends Partition
+
+class HoodieMergeOnReadRDD(sc: SparkContext,
+                           broadcastedConf: 
Broadcast[SerializableConfiguration],
+                           baseFileReadFunction: PartitionedFile => 
Iterator[Any],
+                           dataSchema: StructType,
+                           hoodieRealtimeFileSplits: 
List[HoodieMergeOnReadFileSplit])
+  extends RDD[InternalRow](sc, Nil) {
+
+  // Broadcast the hadoop Configuration to executors.
+  def this(sc: SparkContext,
+           config: Configuration,
+           dataReadFunction: PartitionedFile => Iterator[Any],
+           dataSchema: StructType,
+           hoodieRealtimeFileSplits: List[HoodieMergeOnReadFileSplit]) = {
+    this(
+      sc,
+      sc.broadcast(new SerializableConfiguration(config))
+      .asInstanceOf[Broadcast[SerializableConfiguration]],
+      dataReadFunction,
+      dataSchema,
+      hoodieRealtimeFileSplits)
+  }
+
+  override def compute(split: Partition, context: TaskContext): 
Iterator[InternalRow] = {
+    val mergeParquetPartition = split.asInstanceOf[HoodieMergeOnReadPartition]
+    val baseFileIterator = read(mergeParquetPartition.split.dataFile, 
baseFileReadFunction)
+    mergeParquetPartition.split match {
+      case dataFileOnlySplit if dataFileOnlySplit.logPaths.isEmpty =>
+        baseFileIterator
+      case unMergeSplit if unMergeSplit.payload
+        .equals(DataSourceReadOptions.DEFAULT_MERGE_ON_READ_PAYLOAD_VAL) =>
+        unMergeFileIterator(unMergeSplit, baseFileIterator)
+      case mergeSplit if !mergeSplit.payload.isEmpty =>
+        mergeFileIterator(mergeSplit, baseFileIterator)
+      case _ => throw new HoodieException(s"Unable to select an Iterator to 
read the Hoodie MOR File Split for " +
+        s"file path: ${mergeParquetPartition.split.dataFile.filePath}" +
+        s"log paths: ${mergeParquetPartition.split.logPaths.toString}" +
+        s"hoodie table path: ${mergeParquetPartition.split.tablePath}" +
+        s"spark partition Index: ${mergeParquetPartition.index}")
+    }
+  }
+
+  override protected def getPartitions: Array[Partition] = {
+    hoodieRealtimeFileSplits.zipWithIndex.map(file => 
HoodieMergeOnReadPartition(file._2, file._1)).toArray
+  }
+
+  private def getConfig(): Configuration = {
+    broadcastedConf.value.get()
+  }
+
+  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: HoodieMergeOnReadFileSplit,
+                                  baseFileIterator: Iterator[InternalRow]): 
Iterator[InternalRow] =
+    new Iterator[InternalRow] {
+      private val logSchema = getLogAvroSchema(split)
+      private val sparkTypes = 
SchemaConverters.toSqlType(logSchema).dataType.asInstanceOf[StructType]
+      private val converter = new AvroDeserializer(logSchema, sparkTypes)
+      private val logRecords = scanLog(split, logSchema).getRecords
+      private val logRecordsIterator = logRecords.keySet().iterator().asScala
+
+      override def hasNext: Boolean = {
+        baseFileIterator.hasNext || logRecordsIterator.hasNext
+      }
+
+      override def next(): InternalRow = {
+        if (baseFileIterator.hasNext) {
+          baseFileIterator.next()
+        } else {
+          val curAvrokey = logRecordsIterator.next()
+          val curAvroRecord = 
logRecords.get(curAvrokey).getData.getInsertValue(logSchema).get()
+          converter.deserialize(curAvroRecord).asInstanceOf[InternalRow]
+        }
+      }
+    }
+
+  private def mergeFileIterator(split: HoodieMergeOnReadFileSplit,
+                                baseFileIterator: Iterator[InternalRow]): 
Iterator[InternalRow] =
+    new Iterator[InternalRow] {
+      private val avroSchema = getLogAvroSchema(split)
+      private val sparkSchema = 
SchemaConverters.toSqlType(avroSchema).dataType.asInstanceOf[StructType]
+      private val avroToRowConverter = new AvroDeserializer(avroSchema, 
sparkSchema)
+      private val rowToAvroConverter = new AvroSerializer(sparkSchema, 
avroSchema, false)
+      private val logRecords = scanLog(split, avroSchema).getRecords
+      private val logRecordToRead = logRecords.keySet()
+
+      private var baseFileFinished = false
+      private var logRecordsIterator: Iterator[String] = _
+      private var recordToLoad: InternalRow = _
+
+      @scala.annotation.tailrec
+      override def hasNext: Boolean = {
+        if (baseFileIterator.hasNext) {

Review comment:
       Actually on further thought the first point may not be possible, since 
if we iterator over log records it will be difficult to find the corresponding 
base parquet record using record key (for merging).




----------------------------------------------------------------
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]


Reply via email to