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



##########
File path: hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala
##########
@@ -58,26 +61,20 @@ class DefaultSource extends RelationProvider
       throw new HoodieException("'path' must be specified.")
     }
 
+    val fs = FSUtils.getFs(path.get, 
sqlContext.sparkContext.hadoopConfiguration)
+    val globPaths = HudiSparkUtils.checkAndGlobPathIfNecessary(Seq(path.get), 
fs)
+    val tablePath = DataSourceUtils.getTablePath(fs, globPaths.toArray)
+    val metaClient = new HoodieTableMetaClient(fs.getConf, tablePath)

Review comment:
       I think I can move this into the SNAPSHOT_QUERY session.

##########
File path: hudi-spark/src/main/scala/org/apache/hudi/SnapshotRelation.scala
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.avro.HoodieAvroUtils
+import org.apache.hudi.common.model.HoodieBaseFile
+import org.apache.hudi.common.table.{HoodieTableMetaClient, 
TableSchemaResolver}
+import org.apache.hudi.common.table.view.HoodieTableFileSystemView
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils
+import 
org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils.getMaxCompactionMemoryInBytes
+
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapred.JobConf
+import org.apache.spark.internal.Logging
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
+import org.apache.spark.sql.{Row, SQLContext}
+import org.apache.spark.sql.sources.{BaseRelation, TableScan}
+import org.apache.spark.sql.types.StructType
+
+import scala.collection.JavaConverters._
+
+case class HudiMergeOnReadFileSplit(dataFile: PartitionedFile,
+                                    logPaths: Option[List[String]],
+                                    latestCommit: String,
+                                    tablePath: String,
+                                    maxCompactionMemoryInBytes: Long,
+                                    skipMerge: Boolean)
+
+class SnapshotRelation (val sqlContext: SQLContext,
+                        val optParams: Map[String, String],
+                        val userSchema: StructType,
+                        val globPaths: Seq[Path],
+                        val metaClient: HoodieTableMetaClient)
+  extends BaseRelation with TableScan with Logging{
+
+  private val conf = sqlContext.sparkContext.hadoopConfiguration
+
+  // 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)
+  }
+
+  private val skipMerge = optParams.getOrElse(
+    DataSourceReadOptions.REALTIME_SKIP_MERGE_KEY,
+    DataSourceReadOptions.DEFAULT_REALTIME_SKIP_MERGE_VAL).toBoolean
+  private val maxCompactionMemoryInBytes = getMaxCompactionMemoryInBytes(new 
JobConf(conf))
+  private val fileIndex = buildFileIndex()
+
+  override def schema: StructType = latestSchema
+
+  override def needConversion: Boolean = false
+
+  override def buildScan(): RDD[Row] = {
+    val parquetReaderFunction = new 
ParquetFileFormat().buildReaderWithPartitionValues(
+      sparkSession = sqlContext.sparkSession,
+      dataSchema = latestSchema,
+      partitionSchema = StructType(Nil),
+      requiredSchema = latestSchema,

Review comment:
       When I pushdown nothing and pass the full schema as user requested 
schema, with simply changing from `TableScan()` to `PrunedFilteredScan`, the 
behavior of the parquet reader was changed and not reading the correct schema. 
I need to dig deeper here.
   Let's focus on making the basic functionality work in this PR. I will figure 
this out with a follow-up PR.

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

Review comment:
       I followed the HadoopRDD implementation here 
https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala#L120

##########
File path: hudi-spark/src/main/scala/org/apache/hudi/SnapshotRelation.scala
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.avro.HoodieAvroUtils
+import org.apache.hudi.common.model.HoodieBaseFile
+import org.apache.hudi.common.table.{HoodieTableMetaClient, 
TableSchemaResolver}
+import org.apache.hudi.common.table.view.HoodieTableFileSystemView
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils
+import 
org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils.getMaxCompactionMemoryInBytes
+
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapred.JobConf
+import org.apache.spark.internal.Logging
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
+import org.apache.spark.sql.{Row, SQLContext}
+import org.apache.spark.sql.sources.{BaseRelation, TableScan}
+import org.apache.spark.sql.types.StructType
+
+import scala.collection.JavaConverters._
+
+case class HoodieMergeOnReadFileSplit(dataFile: PartitionedFile,
+                                      logPaths: Option[List[String]],
+                                      latestCommit: String,
+                                      tablePath: String,
+                                      maxCompactionMemoryInBytes: Long,
+                                      payload: String,
+                                      orderingVal: String)
+
+class SnapshotRelation (val sqlContext: SQLContext,
+                        val optParams: Map[String, String],
+                        val userSchema: StructType,
+                        val globPaths: Seq[Path],
+                        val metaClient: HoodieTableMetaClient)
+  extends BaseRelation with TableScan with Logging {
+
+  private val conf = sqlContext.sparkContext.hadoopConfiguration
+
+  // 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)

Review comment:
       good to know. thanks

##########
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:
       - The log scanner scans all the log files in one batch and handled the 
merging internally. The output is a hashmap we can use directly. This 
`logRecordsIterator` is just looping through the hashmap and doesn't load the 
row one by one like the `daseFileIterator`.
   
   - This is a little bit tricky. If the hasNext return true, but next() 
doesn't return a value, Spark will throw an exception. In our logic, we don't 
know `hasNext` will be true of false until we find the qualified record to 
read. For example, 100 records in base file and 100 delete records in the log 
file. We will read 0 row and hasNext should return false in the first call, but 
we have iterated through the whole base file already. There is a test case for 
this example.

##########
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)

Review comment:
       yes we have this option for Hive 
https://github.com/apache/hudi/blob/master/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeUnmergedRecordReader.java
   The performance will be better without merging. We can avoid the type 
conversion at least Row -> Avro -> Merge -> Avro -> Row 

##########
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,

Review comment:
       This problem is tricky. Both sides make sense to me. It seems like 
impossible to completely switch from hoodie to hudi everywhere. Should we 
define a standard for the naming convension? Like class name -> `hoodie`, 
package name -> `hudi` @vinothchandar 

##########
File path: hudi-spark/src/main/scala/org/apache/hudi/SnapshotRelation.scala
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.avro.HoodieAvroUtils
+import org.apache.hudi.common.model.HoodieBaseFile
+import org.apache.hudi.common.table.{HoodieTableMetaClient, 
TableSchemaResolver}
+import org.apache.hudi.common.table.view.HoodieTableFileSystemView
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils
+import 
org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils.getMaxCompactionMemoryInBytes
+
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapred.JobConf
+import org.apache.spark.internal.Logging
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
+import org.apache.spark.sql.{Row, SQLContext}
+import org.apache.spark.sql.sources.{BaseRelation, TableScan}
+import org.apache.spark.sql.types.StructType
+
+import scala.collection.JavaConverters._
+
+case class HudiMergeOnReadFileSplit(dataFile: PartitionedFile,
+                                    logPaths: Option[List[String]],
+                                    latestCommit: String,
+                                    tablePath: String,
+                                    maxCompactionMemoryInBytes: Long,
+                                    skipMerge: Boolean)
+
+class SnapshotRelation (val sqlContext: SQLContext,
+                        val optParams: Map[String, String],
+                        val userSchema: StructType,
+                        val globPaths: Seq[Path],
+                        val metaClient: HoodieTableMetaClient)
+  extends BaseRelation with TableScan with Logging{
+
+  private val conf = sqlContext.sparkContext.hadoopConfiguration
+
+  // 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)
+  }
+
+  private val skipMerge = optParams.getOrElse(
+    DataSourceReadOptions.REALTIME_SKIP_MERGE_KEY,
+    DataSourceReadOptions.DEFAULT_REALTIME_SKIP_MERGE_VAL).toBoolean
+  private val maxCompactionMemoryInBytes = getMaxCompactionMemoryInBytes(new 
JobConf(conf))
+  private val fileIndex = buildFileIndex()
+
+  override def schema: StructType = latestSchema
+
+  override def needConversion: Boolean = false
+
+  override def buildScan(): RDD[Row] = {
+    val parquetReaderFunction = new 
ParquetFileFormat().buildReaderWithPartitionValues(
+      sparkSession = sqlContext.sparkSession,
+      dataSchema = latestSchema,
+      partitionSchema = StructType(Nil),
+      requiredSchema = latestSchema,
+      filters = Seq.empty,

Review comment:
       Missing the filter for log file probably ok because Spark will apply the 
filter again after the pushdown filter. Description 
https://github.com/apache/spark/blob/branch-2.4/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala#L268

##########
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)

Review comment:
       agree here. Our `AvroConversionHelper` is handling `Row`, which is not 
an extension of `InternalRow`. If we don't need `Row` specifically, I think we 
can adapt to the Spark Internal serializer.




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