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 2020/06/09 19:33:26 UTC

[GitHub] [hudi] garyli1019 opened a new pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

garyli1019 opened a new pull request #1722:
URL: https://github.com/apache/hudi/pull/1722


   ## What is the purpose of the pull request
   
   This PR implement Spark Datasource for MOR table
   
   ## Brief change log
   
     - Implemented realtimeRelation
     - Implemented HoodieRealtimeInputFormat on top of Spark SQL internal ParquetFileFormat
     - Implemented HoodieParquetRecordReaderIterator and RecordReader
   
   ## Verify this pull request
   
   This change added tests and can be verified as follows:
   
     - *Added TestRealtimeDataSource to verify this feature.*
     
   ## Committer checklist
   
    - [ ] Has a corresponding JIRA in PR title & commit
    
    - [ ] Commit message is descriptive of the change
    
    - [ ] CI is green
   
    - [ ] Necessary doc changes done or have another open PR
          
    - [ ] For large changes, please consider breaking it into sub-tasks under an umbrella JIRA.


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



[GitHub] [hudi] garyli1019 commented on pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
garyli1019 commented on pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#issuecomment-649235630


   @vinothchandar Thanks for reviewing! I created tickets for the follow-up work. All the file listing and globing can be improved after @umehrot2 's PR merged. 


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



[GitHub] [hudi] garyli1019 commented on a change in pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
garyli1019 commented on a change in pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#discussion_r438266305



##########
File path: hudi-spark/src/main/scala/org/apache/hudi/RealtimeRelation.scala
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.table.{HoodieTableMetaClient, TableSchemaResolver}
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.hadoop.{HoodieParquetInputFormat, HoodieROTablePathFilter}
+import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.table.HoodieTable
+import org.apache.hadoop.mapred.JobConf
+import org.apache.log4j.LogManager
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{Row, SQLContext}
+import org.apache.spark.sql.sources.{BaseRelation, TableScan}
+import org.apache.spark.sql.types.StructType
+
+import java.util
+import scala.collection.JavaConverters._
+
+/**
+ * This is the Spark DataSourceV1 relation to read Hudi MOR table.
+ * @param sqlContext
+ * @param basePath
+ * @param optParams
+ * @param userSchema
+ */
+class RealtimeRelation(val sqlContext: SQLContext,
+                       val basePath: String,
+                       val optParams: Map[String, String],
+                       val userSchema: StructType) extends BaseRelation with TableScan {
+
+  private val log = LogManager.getLogger(classOf[RealtimeRelation])
+  private val conf = sqlContext.sparkContext.hadoopConfiguration
+
+  // Set config for listStatus() in HoodieParquetInputFormat
+  conf.setClass(
+    "mapreduce.input.pathFilter.class",
+    classOf[HoodieROTablePathFilter],
+    classOf[org.apache.hadoop.fs.PathFilter])
+  conf.setStrings("mapreduce.input.fileinputformat.inputdir", basePath)
+  conf.setStrings("mapreduce.input.fileinputformat.input.dir.recursive", "true")
+
+  private val HoodieInputFormat = new HoodieParquetInputFormat
+  HoodieInputFormat.setConf(conf)
+  private val fileStatus = HoodieInputFormat.listStatus(new JobConf(conf))
+  log.debug("All parquet files" + fileStatus.map(s => s.getPath.toString).mkString(","))
+  private val fileGroup = HoodieRealtimeInputFormatUtils.groupLogsByBaseFile(conf, util.Arrays.stream(fileStatus)).asScala
+
+  // Split the file group to: parquet file without a matching log file, parquet file need to merge with log files
+  private val parquetWithoutLogPaths: List[String] = fileGroup.filter(p => p._2.size() == 0).keys.toList
+  private val fileWithLogMap: Map[String, String] = fileGroup.filter(p => p._2.size() > 0).map{ case(k, v) => (k, v.asScala.toList.mkString(","))}.toMap
+  log.debug("ParquetWithoutLogPaths" + parquetWithoutLogPaths.mkString(","))
+  log.debug("ParquetWithLogPaths" + fileWithLogMap.map(m => s"${m._1}:${m._2}").mkString(","))
+
+  // Add log file map to options
+  private val finalOps = optParams ++ fileWithLogMap
+
+  // Load Hudi metadata
+  val metaClient = new HoodieTableMetaClient(conf, basePath, true)
+  private val hoodieTable = HoodieTable.create(metaClient, HoodieWriteConfig.newBuilder().withPath(basePath).build(), conf)
+
+  private val commitTimeline = hoodieTable.getMetaClient.getCommitsAndCompactionTimeline
+  if (commitTimeline.empty()) {
+    throw new HoodieException("No Valid Hudi timeline exists")
+  }
+  private val completedCommitTimeline = hoodieTable.getMetaClient.getCommitsTimeline.filterCompletedInstants()
+  private val lastInstant = completedCommitTimeline.lastInstant().get()
+  conf.setStrings("hoodie.realtime.last.commit", lastInstant.getTimestamp)
+
+  // 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)
+  }
+
+  override def schema: StructType = latestSchema
+
+  override def buildScan(): RDD[Row] = {
+    // Read parquet file doesn't have matching log file to merge as normal parquet
+    val regularParquet = sqlContext
+        .read
+        .options(finalOps)
+        .schema(schema)
+        .format("parquet")
+        .load(parquetWithoutLogPaths:_*)
+        .toDF()
+    // Hudi parquet files needed to merge with log file
+    sqlContext
+      .read
+      .options(finalOps)
+      .schema(schema)
+      .format("org.apache.spark.sql.execution.datasources.parquet.HoodieRealtimeInputFormat")
+      .load(fileWithLogMap.keys.toList:_*)
+      .toDF()
+      .union(regularParquet)
+      .rdd

Review comment:
       I think it would be the same. The interface requires rdd here. We have done this in incrementalRelation as well.




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



[GitHub] [hudi] vinothchandar commented on pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#issuecomment-641708414


   @umehrot2 take a look as well? 


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



[GitHub] [hudi] garyli1019 commented on a change in pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
garyli1019 commented on a change in pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#discussion_r445313298



##########
File path: hudi-spark/src/main/scala/org/apache/hudi/SnapshotRelation.scala
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.table.{HoodieTableMetaClient, TableSchemaResolver}
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.hadoop.{HoodieParquetInputFormat, HoodieROTablePathFilter}
+import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.table.HoodieTable
+
+import org.apache.hadoop.mapred.JobConf
+import org.apache.log4j.LogManager
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{Row, SQLContext}
+import org.apache.spark.sql.sources.{BaseRelation, TableScan}
+import org.apache.spark.sql.types.StructType
+
+import java.util
+import scala.collection.JavaConverters._
+
+/**
+ * This is the Spark DataSourceV1 relation to read Hudi MOR table.
+ * @param sqlContext
+ * @param basePath
+ * @param optParams
+ * @param userSchema
+ */
+class SnapshotRelation(val sqlContext: SQLContext,
+                       val basePath: String,
+                       val optParams: Map[String, String],
+                       val userSchema: StructType) extends BaseRelation with TableScan {
+
+  private val log = LogManager.getLogger(classOf[SnapshotRelation])
+  private val conf = sqlContext.sparkContext.hadoopConfiguration
+
+  // Set config for listStatus() in HoodieParquetInputFormat
+  // TODO(garyli): Switch to bootstrap file listing methods
+  conf.setClass(
+    "mapreduce.input.pathFilter.class",
+    classOf[HoodieROTablePathFilter],
+    classOf[org.apache.hadoop.fs.PathFilter])
+  conf.setStrings("mapreduce.input.fileinputformat.inputdir", basePath)
+  conf.setStrings("mapreduce.input.fileinputformat.input.dir.recursive", "true")
+
+  private val HoodieInputFormat = new HoodieParquetInputFormat
+  HoodieInputFormat.setConf(conf)
+  private val fileStatus = HoodieInputFormat.listStatus(new JobConf(conf))
+  private val fileGroup = HoodieRealtimeInputFormatUtils.groupLogsByBaseFile(conf, util.Arrays.stream(fileStatus)).asScala
+
+  // Split the file group to: parquet file without a matching log file, parquet file need to merge with log files
+  private val parquetWithoutLogPaths: List[String] = fileGroup.filter(p => p._2.size() == 0).keys.toList
+  private val fileWithLogMap: Map[String, String] = fileGroup.filter(p => p._2.size() > 0).map{ case(k, v) => (k, v.asScala.toList.mkString(","))}.toMap
+
+  if (log.isDebugEnabled) {
+    log.debug("All parquet files" + fileStatus.map(s => s.getPath.toString).mkString(","))
+    log.debug("ParquetWithoutLogPaths" + parquetWithoutLogPaths.mkString(","))
+    log.debug("ParquetWithLogPaths" + fileWithLogMap.map(m => s"${m._1}:${m._2}").mkString(","))
+  }
+
+  // Add log file map to options
+  private val finalOps = optParams ++ fileWithLogMap
+
+  // Load Hudi metadata
+  val metaClient = new HoodieTableMetaClient(conf, basePath, true)
+  private val hoodieTable = HoodieTable.create(metaClient, HoodieWriteConfig.newBuilder().withPath(basePath).build(), conf)
+
+  private val commitTimeline = hoodieTable.getMetaClient.getCommitsAndCompactionTimeline
+  if (commitTimeline.empty()) {

Review comment:
       Once we integrate COW and MOR into one relation, then we can return `HudiEmptyRelation` on the `DefaultSource`. We probably can't do it inside this relation.




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



[GitHub] [hudi] garyli1019 commented on a change in pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
garyli1019 commented on a change in pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#discussion_r445306899



##########
File path: hudi-spark/src/main/scala/org/apache/hudi/SnapshotRelation.scala
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.table.{HoodieTableMetaClient, TableSchemaResolver}
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.hadoop.{HoodieParquetInputFormat, HoodieROTablePathFilter}
+import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.table.HoodieTable
+
+import org.apache.hadoop.mapred.JobConf
+import org.apache.log4j.LogManager
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{Row, SQLContext}
+import org.apache.spark.sql.sources.{BaseRelation, TableScan}
+import org.apache.spark.sql.types.StructType
+
+import java.util
+import scala.collection.JavaConverters._
+
+/**
+ * This is the Spark DataSourceV1 relation to read Hudi MOR table.
+ * @param sqlContext
+ * @param basePath
+ * @param optParams
+ * @param userSchema
+ */
+class SnapshotRelation(val sqlContext: SQLContext,
+                       val basePath: String,
+                       val optParams: Map[String, String],
+                       val userSchema: StructType) extends BaseRelation with TableScan {

Review comment:
       I was wrong here. We need the `PruneFilteredScan` to push down the filter and projection. 




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



[GitHub] [hudi] codecov-commenter edited a comment on pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#issuecomment-643095877


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/1722?src=pr&el=h1) Report
   > Merging [#1722](https://codecov.io/gh/apache/hudi/pull/1722?src=pr&el=desc) into [master](https://codecov.io/gh/apache/hudi/commit/8919be6a5d8038db7265bfd7459d72fbd545f133&el=desc) will **decrease** coverage by `3.03%`.
   > The diff coverage is `1.44%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/1722/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/1722?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #1722      +/-   ##
   ============================================
   - Coverage     62.82%   59.79%   -3.04%     
   - Complexity     3437     3610     +173     
   ============================================
     Files           401      439      +38     
     Lines         17091    19183    +2092     
     Branches       1698     1946     +248     
   ============================================
   + Hits          10737    11470     +733     
   - Misses         5623     6924    +1301     
   - Partials        731      789      +58     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | #hudicli | `68.45% <ø> (?)` | `1430.00 <ø> (?)` | |
   | #hudiclient | `79.24% <ø> (ø)` | `1258.00 <ø> (ø)` | |
   | #hudicommon | `54.29% <ø> (+0.04%)` | `1486.00 <ø> (+1.00)` | |
   | #hudihadoopmr | `39.36% <ø> (ø)` | `163.00 <ø> (ø)` | |
   | #hudihivesync | `72.25% <ø> (ø)` | `121.00 <ø> (ø)` | |
   | #hudispark | `36.39% <1.44%> (-7.83%)` | `76.00 <0.00> (ø)` | |
   | #huditimelineservice | `63.47% <ø> (ø)` | `47.00 <ø> (ø)` | |
   | #hudiutilities | `73.75% <ø> (ø)` | `287.00 <ø> (ø)` | |
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/1722?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [.../main/scala/org/apache/hudi/SnapshotRelation.scala](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2h1ZGkvU25hcHNob3RSZWxhdGlvbi5zY2FsYQ==) | `0.00% <0.00%> (ø)` | `0.00 <0.00> (?)` | |
   | [...urces/parquet/HoodieMergedParquetRowIterator.scala](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL3NwYXJrL3NxbC9leGVjdXRpb24vZGF0YXNvdXJjZXMvcGFycXVldC9Ib29kaWVNZXJnZWRQYXJxdWV0Um93SXRlcmF0b3Iuc2NhbGE=) | `0.00% <0.00%> (ø)` | `0.00 <0.00> (?)` | |
   | [...rces/parquet/HoodieParquetRealtimeFileFormat.scala](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL3NwYXJrL3NxbC9leGVjdXRpb24vZGF0YXNvdXJjZXMvcGFycXVldC9Ib29kaWVQYXJxdWV0UmVhbHRpbWVGaWxlRm9ybWF0LnNjYWxh) | `0.00% <0.00%> (ø)` | `0.00 <0.00> (?)` | |
   | [...src/main/scala/org/apache/hudi/DefaultSource.scala](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2h1ZGkvRGVmYXVsdFNvdXJjZS5zY2FsYQ==) | `65.71% <33.33%> (-4.88%)` | `7.00 <0.00> (ø)` | |
   | [...main/scala/org/apache/hudi/DataSourceOptions.scala](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2h1ZGkvRGF0YVNvdXJjZU9wdGlvbnMuc2NhbGE=) | `93.54% <100.00%> (ø)` | `0.00 <0.00> (ø)` | |
   | [...a/org/apache/hudi/cli/commands/RepairsCommand.java](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree#diff-aHVkaS1jbGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY2xpL2NvbW1hbmRzL1JlcGFpcnNDb21tYW5kLmphdmE=) | `71.23% <0.00%> (ø)` | `10.00% <0.00%> (?%)` | |
   | [...src/main/java/org/apache/hudi/cli/TableHeader.java](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree#diff-aHVkaS1jbGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY2xpL1RhYmxlSGVhZGVyLmphdmE=) | `77.77% <0.00%> (ø)` | `5.00% <0.00%> (?%)` | |
   | [...n/java/org/apache/hudi/cli/HoodieSplashScreen.java](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree#diff-aHVkaS1jbGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY2xpL0hvb2RpZVNwbGFzaFNjcmVlbi5qYXZh) | `42.85% <0.00%> (ø)` | `2.00% <0.00%> (?%)` | |
   | [...di-cli/src/main/java/org/apache/hudi/cli/Main.java](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree#diff-aHVkaS1jbGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY2xpL01haW4uamF2YQ==) | `0.00% <0.00%> (ø)` | `0.00% <0.00%> (?%)` | |
   | [...va/org/apache/hudi/cli/commands/ExportCommand.java](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree#diff-aHVkaS1jbGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY2xpL2NvbW1hbmRzL0V4cG9ydENvbW1hbmQuamF2YQ==) | `1.09% <0.00%> (ø)` | `1.00% <0.00%> (?%)` | |
   | ... and [34 more](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree-more) | |
   


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



[GitHub] [hudi] garyli1019 commented on a change in pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
garyli1019 commented on a change in pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#discussion_r439769643



##########
File path: hudi-spark/src/main/scala/org/apache/hudi/SnapshotRelation.scala
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.table.{HoodieTableMetaClient, TableSchemaResolver}
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.hadoop.{HoodieParquetInputFormat, HoodieROTablePathFilter}
+import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.table.HoodieTable
+
+import org.apache.hadoop.mapred.JobConf
+import org.apache.log4j.LogManager
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{Row, SQLContext}
+import org.apache.spark.sql.sources.{BaseRelation, TableScan}
+import org.apache.spark.sql.types.StructType
+
+import java.util
+import scala.collection.JavaConverters._
+
+/**
+ * This is the Spark DataSourceV1 relation to read Hudi MOR table.
+ * @param sqlContext
+ * @param basePath
+ * @param optParams
+ * @param userSchema
+ */
+class SnapshotRelation(val sqlContext: SQLContext,
+                       val basePath: String,
+                       val optParams: Map[String, String],
+                       val userSchema: StructType) extends BaseRelation with TableScan {
+
+  private val log = LogManager.getLogger(classOf[SnapshotRelation])
+  private val conf = sqlContext.sparkContext.hadoopConfiguration
+
+  // Set config for listStatus() in HoodieParquetInputFormat
+  conf.setClass(
+    "mapreduce.input.pathFilter.class",
+    classOf[HoodieROTablePathFilter],
+    classOf[org.apache.hadoop.fs.PathFilter])
+  conf.setStrings("mapreduce.input.fileinputformat.inputdir", basePath)
+  conf.setStrings("mapreduce.input.fileinputformat.input.dir.recursive", "true")
+
+  private val HoodieInputFormat = new HoodieParquetInputFormat
+  HoodieInputFormat.setConf(conf)
+  private val fileStatus = HoodieInputFormat.listStatus(new JobConf(conf))
+  private val fileGroup = HoodieRealtimeInputFormatUtils.groupLogsByBaseFile(conf, util.Arrays.stream(fileStatus)).asScala
+
+  // Split the file group to: parquet file without a matching log file, parquet file need to merge with log files
+  private val parquetWithoutLogPaths: List[String] = fileGroup.filter(p => p._2.size() == 0).keys.toList
+  private val fileWithLogMap: Map[String, String] = fileGroup.filter(p => p._2.size() > 0).map{ case(k, v) => (k, v.asScala.toList.mkString(","))}.toMap
+
+  if (log.isDebugEnabled) {
+    log.debug("All parquet files" + fileStatus.map(s => s.getPath.toString).mkString(","))
+    log.debug("ParquetWithoutLogPaths" + parquetWithoutLogPaths.mkString(","))
+    log.debug("ParquetWithLogPaths" + fileWithLogMap.map(m => s"${m._1}:${m._2}").mkString(","))
+  }
+
+  // Add log file map to options
+  private val finalOps = optParams ++ fileWithLogMap

Review comment:
       I agree here. Other possible implementations could be:
   - Add it to Hadoop config, but it could cause the same issue as using the option hashmap.
   ~- Concatenate the logPath to the parquetPath, then split path in the executor before doing the file split.~(EDIT: Doesn't work, in the `FileFormat`, the file is already a `PartitionedFile` type)
   - Search log based on parquetPath on each executor. This could put pressure on the name node as well. (EDIT: This probably doesn't work as well. Getting the meta client and file system view is very expensive.)
   
   ~I personally prefer the second way if we can efficiently handle all the file listing on the driver side. I will see if it's possible to implement it.~




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



[GitHub] [hudi] garyli1019 commented on a change in pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
garyli1019 commented on a change in pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#discussion_r438261014



##########
File path: hudi-spark/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieRealtimeInputFormat.scala
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.spark.sql.execution.datasources.parquet
+
+import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit
+import org.apache.hudi.realtime.HoodieRealtimeParquetRecordReader
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapred.{FileSplit, JobConf}
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
+import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType}
+import org.apache.parquet.filter2.compat.FilterCompat
+import org.apache.parquet.filter2.predicate.FilterApi
+import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS
+import org.apache.parquet.hadoop.{ParquetFileReader, ParquetInputFormat}
+import org.apache.spark.TaskContext
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
+import org.apache.spark.sql.catalyst.expressions.{JoinedRow, UnsafeRow}
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+import java.net.URI
+import scala.collection.JavaConverters._
+
+/**
+ * This class is an extension of ParquetFileFormat from Spark SQL.
+ * The file split, record reader, record reader iterator are customized to read Hudi MOR table.
+ */
+class HoodieRealtimeInputFormat extends ParquetFileFormat {
+  //TODO: Better usage of this short name.
+  override def shortName(): String = "hudi.realtime"
+  override def toString(): String = "hudi.realtime"
+
+  override def buildReaderWithPartitionValues(sparkSession: SparkSession,
+                                               dataSchema: StructType,
+                                               partitionSchema: StructType,
+                                               requiredSchema: StructType,
+                                               filters: Seq[Filter],
+                                               options: Map[String, String],
+                                               hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = {
+    hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName)
+    hadoopConf.set(
+      ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA,
+      requiredSchema.json)
+    hadoopConf.set(
+      ParquetWriteSupport.SPARK_ROW_SCHEMA,
+      requiredSchema.json)
+    hadoopConf.set(
+      SQLConf.SESSION_LOCAL_TIMEZONE.key,
+      sparkSession.sessionState.conf.sessionLocalTimeZone)
+    hadoopConf.setBoolean(
+      SQLConf.CASE_SENSITIVE.key,
+      sparkSession.sessionState.conf.caseSensitiveAnalysis)
+
+    ParquetWriteSupport.setSchema(requiredSchema, hadoopConf)
+
+    // Sets flags for `ParquetToSparkSchemaConverter`
+    hadoopConf.setBoolean(
+      SQLConf.PARQUET_BINARY_AS_STRING.key,
+      sparkSession.sessionState.conf.isParquetBinaryAsString)
+    hadoopConf.setBoolean(
+      SQLConf.PARQUET_INT96_AS_TIMESTAMP.key,
+      sparkSession.sessionState.conf.isParquetINT96AsTimestamp)
+
+    val broadcastedHadoopConf =
+      sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf))
+
+    // TODO: if you move this into the closure it reverts to the default values.
+    // If true, enable using the custom RecordReader for parquet. This only works for
+    // a subset of the types (no complex types).
+    val resultSchema = StructType(partitionSchema.fields ++ requiredSchema.fields)
+    val sqlConf = sparkSession.sessionState.conf
+    val enableRecordFilter: Boolean = sqlConf.parquetRecordFilterEnabled
+    val timestampConversion: Boolean = sqlConf.isParquetINT96TimestampConversion
+    val enableParquetFilterPushDown: Boolean = sqlConf.parquetFilterPushDown
+    // Whole stage codegen (PhysicalRDD) is able to deal with batches directly
+    val returningBatch = supportBatch(sparkSession, resultSchema)
+    val pushDownDate = sqlConf.parquetFilterPushDownDate
+    val pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp
+    val pushDownDecimal = sqlConf.parquetFilterPushDownDecimal
+    val pushDownStringStartWith = sqlConf.parquetFilterPushDownStringStartWith
+    val pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold
+    val isCaseSensitive = sqlConf.caseSensitiveAnalysis
+
+    (file: PartitionedFile) => {
+      assert(file.partitionValues.numFields == partitionSchema.size)
+
+      val sharedConf = broadcastedHadoopConf.value.value
+      val fileSplit =
+        new FileSplit(new Path(new URI(file.filePath)), file.start, file.length, new Array[String](0))
+      val filePath = fileSplit.getPath
+
+      val basePath = sharedConf.get("mapreduce.input.fileinputformat.inputdir")
+      val maxCommitTime = sharedConf.get("hoodie.realtime.last.commit")
+      // Read the log file path from the option
+      val logPathStr = options.getOrElse(fileSplit.getPath.toString, "").split(",")
+      log.debug(s"fileSplit.getPath in HoodieRealtimeInputFormat: ${fileSplit.getPath} and ${fileSplit.getPath.getName}")
+      log.debug(s"logPath in HoodieRealtimeInputFormat: ${logPathStr.toString}")
+      val hoodieRealtimeFileSplit = new HoodieRealtimeFileSplit(fileSplit, basePath, logPathStr.toList.asJava, maxCommitTime)
+
+      lazy val footerFileMetaData =
+        ParquetFileReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData
+      // Try to push down filters when filter push-down is enabled.
+      val pushed = if (enableParquetFilterPushDown) {
+        val parquetSchema = footerFileMetaData.getSchema
+        val parquetFilters = new ParquetFilters(pushDownDate, pushDownTimestamp, pushDownDecimal,
+          pushDownStringStartWith, pushDownInFilterThreshold, isCaseSensitive)
+        filters
+          // Collects all converted Parquet filter predicates. Notice that not all predicates can be
+          // converted (`ParquetFilters.createFilter` returns an `Option`). That's why a `flatMap`
+          // is used here.
+          .flatMap(parquetFilters.createFilter(parquetSchema, _))
+          .reduceOption(FilterApi.and)
+      } else {
+        None
+      }
+
+      // PARQUET_INT96_TIMESTAMP_CONVERSION says to apply timezone conversions to int96 timestamps'
+      // *only* if the file was created by something other than "parquet-mr", so check the actual
+      // writer here for this file.  We have to do this per-file, as each file in the table may
+      // have different writers.
+      // Define isCreatedByParquetMr as function to avoid unnecessary parquet footer reads.
+      def isCreatedByParquetMr: Boolean =
+        footerFileMetaData.getCreatedBy().startsWith("parquet-mr")
+
+      val convertTz =
+        if (timestampConversion && !isCreatedByParquetMr) {
+          Some(DateTimeUtils.getTimeZone(sharedConf.get(SQLConf.SESSION_LOCAL_TIMEZONE.key)))
+        } else {
+          None
+        }
+
+      val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0)
+      val hadoopAttemptContext =
+        new TaskAttemptContextImpl(broadcastedHadoopConf.value.value, attemptId)
+
+      // Try to push down filters when filter push-down is enabled.
+      // Notice: This push-down is RowGroups level, not individual records.
+      if (pushed.isDefined) {
+        ParquetInputFormat.setFilterPredicate(hadoopAttemptContext.getConfiguration, pushed.get)
+      }
+      val taskContext = Option(TaskContext.get())
+      //TODO: Support the vectorized reader.
+      logDebug(s"Falling back to parquet-mr")
+      // ParquetRecordReader returns UnsafeRow
+      val reader = if (pushed.isDefined && enableRecordFilter) {
+        val parquetFilter = FilterCompat.get(pushed.get, null)
+        new HoodieRealtimeParquetRecordReader[UnsafeRow](new ParquetReadSupport(convertTz), parquetFilter, hoodieRealtimeFileSplit, new JobConf(sharedConf))
+      } else {
+        new HoodieRealtimeParquetRecordReader[UnsafeRow](new ParquetReadSupport(convertTz), hoodieRealtimeFileSplit, new JobConf(sharedConf))
+      }
+      val iter = new HoodieParquetRecordReaderIterator(reader)
+      // SPARK-23457 Register a task completion lister before `initialization`.
+      taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close()))
+      reader.initialize(hoodieRealtimeFileSplit, hadoopAttemptContext)
+      iter.init()
+
+      val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes
+      val joinedRow = new JoinedRow()
+      val appendPartitionColumns = GenerateUnsafeProjection.generate(fullSchema, fullSchema)
+
+      // This is a horrible erasure hack...  if we type the iterator above, then it actually check

Review comment:
       yes this is from Spark 2.4 https://github.com/apache/spark/blob/branch-2.4/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala#L445
   This horrible erasure hack was updated in the latest master though...




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



[GitHub] [hudi] umehrot2 commented on a change in pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
umehrot2 commented on a change in pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#discussion_r439705157



##########
File path: hudi-spark/src/main/scala/org/apache/hudi/SnapshotRelation.scala
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.table.{HoodieTableMetaClient, TableSchemaResolver}
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.hadoop.{HoodieParquetInputFormat, HoodieROTablePathFilter}
+import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.table.HoodieTable
+
+import org.apache.hadoop.mapred.JobConf
+import org.apache.log4j.LogManager
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{Row, SQLContext}
+import org.apache.spark.sql.sources.{BaseRelation, TableScan}
+import org.apache.spark.sql.types.StructType
+
+import java.util
+import scala.collection.JavaConverters._
+
+/**
+ * This is the Spark DataSourceV1 relation to read Hudi MOR table.
+ * @param sqlContext
+ * @param basePath
+ * @param optParams
+ * @param userSchema
+ */
+class SnapshotRelation(val sqlContext: SQLContext,
+                       val basePath: String,
+                       val optParams: Map[String, String],
+                       val userSchema: StructType) extends BaseRelation with TableScan {

Review comment:
       Does this does not support `predicate pushdown` or `column pruning` ? Seems like not because we need to implement `PruneFilteredScan` instead of `TableScan` to support these, and pass it down for reading to the parquet file format

##########
File path: hudi-spark/src/main/scala/org/apache/hudi/SnapshotRelation.scala
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.table.{HoodieTableMetaClient, TableSchemaResolver}
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.hadoop.{HoodieParquetInputFormat, HoodieROTablePathFilter}
+import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.table.HoodieTable
+
+import org.apache.hadoop.mapred.JobConf
+import org.apache.log4j.LogManager
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{Row, SQLContext}
+import org.apache.spark.sql.sources.{BaseRelation, TableScan}
+import org.apache.spark.sql.types.StructType
+
+import java.util
+import scala.collection.JavaConverters._
+
+/**
+ * This is the Spark DataSourceV1 relation to read Hudi MOR table.
+ * @param sqlContext
+ * @param basePath
+ * @param optParams
+ * @param userSchema
+ */
+class SnapshotRelation(val sqlContext: SQLContext,
+                       val basePath: String,
+                       val optParams: Map[String, String],
+                       val userSchema: StructType) extends BaseRelation with TableScan {
+
+  private val log = LogManager.getLogger(classOf[SnapshotRelation])
+  private val conf = sqlContext.sparkContext.hadoopConfiguration
+
+  // Set config for listStatus() in HoodieParquetInputFormat
+  conf.setClass(
+    "mapreduce.input.pathFilter.class",
+    classOf[HoodieROTablePathFilter],
+    classOf[org.apache.hadoop.fs.PathFilter])
+  conf.setStrings("mapreduce.input.fileinputformat.inputdir", basePath)
+  conf.setStrings("mapreduce.input.fileinputformat.input.dir.recursive", "true")
+
+  private val HoodieInputFormat = new HoodieParquetInputFormat
+  HoodieInputFormat.setConf(conf)
+  private val fileStatus = HoodieInputFormat.listStatus(new JobConf(conf))
+  private val fileGroup = HoodieRealtimeInputFormatUtils.groupLogsByBaseFile(conf, util.Arrays.stream(fileStatus)).asScala
+
+  // Split the file group to: parquet file without a matching log file, parquet file need to merge with log files
+  private val parquetWithoutLogPaths: List[String] = fileGroup.filter(p => p._2.size() == 0).keys.toList
+  private val fileWithLogMap: Map[String, String] = fileGroup.filter(p => p._2.size() > 0).map{ case(k, v) => (k, v.asScala.toList.mkString(","))}.toMap
+
+  if (log.isDebugEnabled) {
+    log.debug("All parquet files" + fileStatus.map(s => s.getPath.toString).mkString(","))
+    log.debug("ParquetWithoutLogPaths" + parquetWithoutLogPaths.mkString(","))
+    log.debug("ParquetWithLogPaths" + fileWithLogMap.map(m => s"${m._1}:${m._2}").mkString(","))
+  }
+
+  // Add log file map to options
+  private val finalOps = optParams ++ fileWithLogMap

Review comment:
       I am really not sure how this will play out with huge amounts of data containing thousands of files. We would be serializing passing over this huge map to each and every task, even though each spark task is supposed to work on only one or few file splits. We need to test how this plays out with extremely large number of files.

##########
File path: hudi-spark/src/main/scala/org/apache/hudi/SnapshotRelation.scala
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.table.{HoodieTableMetaClient, TableSchemaResolver}
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.hadoop.{HoodieParquetInputFormat, HoodieROTablePathFilter}
+import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.table.HoodieTable
+
+import org.apache.hadoop.mapred.JobConf
+import org.apache.log4j.LogManager
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{Row, SQLContext}
+import org.apache.spark.sql.sources.{BaseRelation, TableScan}
+import org.apache.spark.sql.types.StructType
+
+import java.util
+import scala.collection.JavaConverters._
+
+/**
+ * This is the Spark DataSourceV1 relation to read Hudi MOR table.
+ * @param sqlContext
+ * @param basePath
+ * @param optParams
+ * @param userSchema
+ */
+class SnapshotRelation(val sqlContext: SQLContext,
+                       val basePath: String,
+                       val optParams: Map[String, String],
+                       val userSchema: StructType) extends BaseRelation with TableScan {
+
+  private val log = LogManager.getLogger(classOf[SnapshotRelation])
+  private val conf = sqlContext.sparkContext.hadoopConfiguration
+
+  // Set config for listStatus() in HoodieParquetInputFormat
+  conf.setClass(
+    "mapreduce.input.pathFilter.class",
+    classOf[HoodieROTablePathFilter],
+    classOf[org.apache.hadoop.fs.PathFilter])
+  conf.setStrings("mapreduce.input.fileinputformat.inputdir", basePath)
+  conf.setStrings("mapreduce.input.fileinputformat.input.dir.recursive", "true")
+
+  private val HoodieInputFormat = new HoodieParquetInputFormat
+  HoodieInputFormat.setConf(conf)
+  private val fileStatus = HoodieInputFormat.listStatus(new JobConf(conf))

Review comment:
       This will likely have performance implications and be slower than spark's listing mechanism which uses spark context to parallelize listing (across the cluster) along with other optimizations. Also, we have already known `HoodieROTablePathFilter` to be really expensive specially when it comes to S3 and it is still a bottleneck for read performance in `read optimized` queries where it is used with `parquet relation`, where it is applied sequentially at the driver to all the files.
   
   It might be worth for you to checkout my implementation once as well where I am re-using the `InMemoryFileIndex` implementation of spark to perform the listing so that all the listing optimizations of spark are retained. Also instead of passing it this expensive filter, which gets applied sequentially to each and every file, I am instead using `fsView.getLatestBaseFiles` to get the latest view of the files.
   
   https://github.com/apache/hudi/pull/1702/files#diff-f14ac7b3cff88313d650b01a56a2b8f8R155
   
   Also does this handle glob paths like `s3://bucket/huditable/*/*` ?

##########
File path: hudi-spark/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieRealtimeInputFormat.scala
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.spark.sql.execution.datasources.parquet
+
+import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapred.{FileSplit, JobConf}
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
+import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType}
+import org.apache.parquet.filter2.compat.FilterCompat
+import org.apache.parquet.filter2.predicate.FilterApi
+import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS
+import org.apache.parquet.hadoop.{ParquetFileReader, ParquetInputFormat, ParquetRecordReader}
+import org.apache.spark.TaskContext
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
+import org.apache.spark.sql.catalyst.expressions.{JoinedRow, UnsafeRow}
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+import java.net.URI
+import scala.collection.JavaConverters._
+
+/**
+ * This class is an extension of ParquetFileFormat from Spark SQL.
+ * The file split, record reader, record reader iterator are customized to read Hudi MOR table.
+ */
+class HoodieRealtimeInputFormat extends ParquetFileFormat {

Review comment:
       Just something to think about, Spark forms file splits by default when you use a `FileFormat` type datasource. It can potentially split this parquet files (with multiple row groups) in between and send it to task for reading. Does this play out well with merging splits with corresponding log files ? I do think that it should be okay in this case, but wanted to point it out if you guys see any potential issues.




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



[GitHub] [hudi] umehrot2 commented on pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
umehrot2 commented on pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#issuecomment-659010020


   > @vinothchandar I agree we should use @umehrot2 RDD approach.
   > 
   > > So you can also in parallel just proceed?
   > 
   > Yes, I will change this PR in parallel.
   > 
   > > Do you just want the two util classes landed? SparkUtils/TablePathUtils?
   > 
   > Yes, @umehrot2 do you mind making a separate PR for these two classes?
   
   @vinothchandar @garyli1019 thank you guys for reviewing the PR. Will look at the comments and start addressing. As for the `HudiSparkUtils/TablePathUtils` let me create a separate PR so that you can re-use those pieces for your PR and are unblocked. Meanwhile you can copy the code over to unblock coding from your side while I work on the PR. Thanks!


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



[GitHub] [hudi] vinothchandar commented on pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#issuecomment-643569262


   Thanks for the detailed comments, @umehrot2 ! Will process and also look at your other pr. 
   
   So, on some of the perf issues you pointed out; please raise 0.6.0 jiras with blocker priority, so we can get them fixed in the nexr major release 


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



[GitHub] [hudi] garyli1019 commented on pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
garyli1019 commented on pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#issuecomment-643901550


   Few major concerns here:
   - Listing files are too expensive.
   Solution: Switch to bootstrap file listing methods once udit's PR merged. Move to RFC-15 once it was ready.
   - Broadcasting paths in option hashmap could cause performance issues.
   I am not sure if there is a better way to do this until RFC-15 is ready. Search log files from executors could be more expensive since it requires TableView and metaClient e.t.c. Even we have thousands of log files, the hashmap might be a few MBs, so I guess it could be ok? 
   
   Major follow-ups after this PR:
   - Incremental view on MOR table.
   - Vectorized reader
   - More efficient type conversion.
   - Support custom payload.
   Those four can be done in parallel but all depend on this PR. We can make a baseline in this PR and iterate through different topics in parallel.
   Also, we can ask help from the community to test in their production for a very large dataset. This could be quite easy if they are using MOR table.


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



[GitHub] [hudi] garyli1019 commented on pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
garyli1019 commented on pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#issuecomment-642182315


   @vinothchandar Thanks for the feedback. Your approach makes sense to me. If we can do it that way then we can reduce some maintenance overhead and be more flexible for the future Spark upgrade. I will try to see if I can do it that way.


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



[GitHub] [hudi] garyli1019 commented on a change in pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
garyli1019 commented on a change in pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#discussion_r439768931



##########
File path: hudi-spark/src/main/scala/org/apache/hudi/SnapshotRelation.scala
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.table.{HoodieTableMetaClient, TableSchemaResolver}
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.hadoop.{HoodieParquetInputFormat, HoodieROTablePathFilter}
+import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.table.HoodieTable
+
+import org.apache.hadoop.mapred.JobConf
+import org.apache.log4j.LogManager
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{Row, SQLContext}
+import org.apache.spark.sql.sources.{BaseRelation, TableScan}
+import org.apache.spark.sql.types.StructType
+
+import java.util
+import scala.collection.JavaConverters._
+
+/**
+ * This is the Spark DataSourceV1 relation to read Hudi MOR table.
+ * @param sqlContext
+ * @param basePath
+ * @param optParams
+ * @param userSchema
+ */
+class SnapshotRelation(val sqlContext: SQLContext,
+                       val basePath: String,
+                       val optParams: Map[String, String],
+                       val userSchema: StructType) extends BaseRelation with TableScan {
+
+  private val log = LogManager.getLogger(classOf[SnapshotRelation])
+  private val conf = sqlContext.sparkContext.hadoopConfiguration
+
+  // Set config for listStatus() in HoodieParquetInputFormat
+  conf.setClass(
+    "mapreduce.input.pathFilter.class",
+    classOf[HoodieROTablePathFilter],
+    classOf[org.apache.hadoop.fs.PathFilter])
+  conf.setStrings("mapreduce.input.fileinputformat.inputdir", basePath)
+  conf.setStrings("mapreduce.input.fileinputformat.input.dir.recursive", "true")
+
+  private val HoodieInputFormat = new HoodieParquetInputFormat
+  HoodieInputFormat.setConf(conf)
+  private val fileStatus = HoodieInputFormat.listStatus(new JobConf(conf))

Review comment:
       This is great to know! We definitely can reuse your implementation there. At this point, I don't know if there is any other way to list all the file status without using `HoodieROTablePathFilter`. RFC-15 will make this more efficient as well.
   No it doesn't support glob paths right now. I think we could put all the paths handling into your `HudiSparkUtils`

##########
File path: hudi-spark/src/main/scala/org/apache/hudi/SnapshotRelation.scala
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.table.{HoodieTableMetaClient, TableSchemaResolver}
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.hadoop.{HoodieParquetInputFormat, HoodieROTablePathFilter}
+import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.table.HoodieTable
+
+import org.apache.hadoop.mapred.JobConf
+import org.apache.log4j.LogManager
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{Row, SQLContext}
+import org.apache.spark.sql.sources.{BaseRelation, TableScan}
+import org.apache.spark.sql.types.StructType
+
+import java.util
+import scala.collection.JavaConverters._
+
+/**
+ * This is the Spark DataSourceV1 relation to read Hudi MOR table.
+ * @param sqlContext
+ * @param basePath
+ * @param optParams
+ * @param userSchema
+ */
+class SnapshotRelation(val sqlContext: SQLContext,
+                       val basePath: String,
+                       val optParams: Map[String, String],
+                       val userSchema: StructType) extends BaseRelation with TableScan {

Review comment:
       I think it does. I checked the physical plan of a filter query and the filter was pushed down.
   IIUC, `TableScan` can automatically push down the filter and column pruning, like what we did in the incremental relation. `PruneFilteredScan` was usually used for Spark connector for other databases to pushdown Spark filter to db SQL query. 
   I found an example on spark-cassandra-connector
   https://github.com/garyli1019/spark-cassandra-connector/blob/master/connector/src/main/scala/org/apache/spark/sql/cassandra/CassandraSourceRelation.scala#L367

##########
File path: hudi-spark/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieRealtimeInputFormat.scala
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.spark.sql.execution.datasources.parquet
+
+import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapred.{FileSplit, JobConf}
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
+import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType}
+import org.apache.parquet.filter2.compat.FilterCompat
+import org.apache.parquet.filter2.predicate.FilterApi
+import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS
+import org.apache.parquet.hadoop.{ParquetFileReader, ParquetInputFormat, ParquetRecordReader}
+import org.apache.spark.TaskContext
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
+import org.apache.spark.sql.catalyst.expressions.{JoinedRow, UnsafeRow}
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+import java.net.URI
+import scala.collection.JavaConverters._
+
+/**
+ * This class is an extension of ParquetFileFormat from Spark SQL.
+ * The file split, record reader, record reader iterator are customized to read Hudi MOR table.
+ */
+class HoodieRealtimeInputFormat extends ParquetFileFormat {

Review comment:
       This is similar to the Hive file split we have. If we split one parquet file into two splits, we will do the log scanning for the same logfile twice. 

##########
File path: hudi-spark/src/main/scala/org/apache/hudi/SnapshotRelation.scala
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.table.{HoodieTableMetaClient, TableSchemaResolver}
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.hadoop.{HoodieParquetInputFormat, HoodieROTablePathFilter}
+import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.table.HoodieTable
+
+import org.apache.hadoop.mapred.JobConf
+import org.apache.log4j.LogManager
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{Row, SQLContext}
+import org.apache.spark.sql.sources.{BaseRelation, TableScan}
+import org.apache.spark.sql.types.StructType
+
+import java.util
+import scala.collection.JavaConverters._
+
+/**
+ * This is the Spark DataSourceV1 relation to read Hudi MOR table.
+ * @param sqlContext
+ * @param basePath
+ * @param optParams
+ * @param userSchema
+ */
+class SnapshotRelation(val sqlContext: SQLContext,
+                       val basePath: String,
+                       val optParams: Map[String, String],
+                       val userSchema: StructType) extends BaseRelation with TableScan {
+
+  private val log = LogManager.getLogger(classOf[SnapshotRelation])
+  private val conf = sqlContext.sparkContext.hadoopConfiguration
+
+  // Set config for listStatus() in HoodieParquetInputFormat
+  conf.setClass(
+    "mapreduce.input.pathFilter.class",
+    classOf[HoodieROTablePathFilter],
+    classOf[org.apache.hadoop.fs.PathFilter])
+  conf.setStrings("mapreduce.input.fileinputformat.inputdir", basePath)
+  conf.setStrings("mapreduce.input.fileinputformat.input.dir.recursive", "true")
+
+  private val HoodieInputFormat = new HoodieParquetInputFormat
+  HoodieInputFormat.setConf(conf)
+  private val fileStatus = HoodieInputFormat.listStatus(new JobConf(conf))
+  private val fileGroup = HoodieRealtimeInputFormatUtils.groupLogsByBaseFile(conf, util.Arrays.stream(fileStatus)).asScala
+
+  // Split the file group to: parquet file without a matching log file, parquet file need to merge with log files
+  private val parquetWithoutLogPaths: List[String] = fileGroup.filter(p => p._2.size() == 0).keys.toList
+  private val fileWithLogMap: Map[String, String] = fileGroup.filter(p => p._2.size() > 0).map{ case(k, v) => (k, v.asScala.toList.mkString(","))}.toMap
+
+  if (log.isDebugEnabled) {
+    log.debug("All parquet files" + fileStatus.map(s => s.getPath.toString).mkString(","))
+    log.debug("ParquetWithoutLogPaths" + parquetWithoutLogPaths.mkString(","))
+    log.debug("ParquetWithLogPaths" + fileWithLogMap.map(m => s"${m._1}:${m._2}").mkString(","))
+  }
+
+  // Add log file map to options
+  private val finalOps = optParams ++ fileWithLogMap

Review comment:
       I agree here. Other possible implementations could be:
   - Add it to Hadoop config, but it could cause the same issue as using the option hashmap.
   - Concatenate the logPath to the parquetPath, then split path in the executor before doing the file split.
   - Search log based on parquetPath on each executor. This could put pressure on the name node as well.
   
   I personally prefer the second way if we can efficiently handle all the file listing on the driver side. I will see if it's possible to implement it.




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



[GitHub] [hudi] garyli1019 commented on a change in pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
garyli1019 commented on a change in pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#discussion_r438274470



##########
File path: hudi-spark/src/main/java/org/apache/hudi/realtime/HoodieRealtimeParquetRecordReader.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.realtime;
+
+import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.parquet.CorruptDeltaByteArrays;
+import org.apache.parquet.ParquetReadOptions;
+import org.apache.parquet.column.Encoding;
+import org.apache.parquet.filter2.compat.FilterCompat;
+import org.apache.parquet.filter2.compat.FilterCompat.Filter;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.ParquetRecordReader;
+import org.apache.parquet.hadoop.api.ReadSupport;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.FileMetaData;
+import org.apache.parquet.hadoop.util.ContextUtil;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.hadoop.util.counters.BenchmarkCounter;
+import org.apache.parquet.io.ParquetDecodingException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.apache.parquet.hadoop.ParquetInputFormat.SPLIT_FILES;
+
+/**
+ * Custom implementation of org.apache.parquet.hadoop.ParquetRecordReader.
+ * This class is a wrapper class. The real reader is the internalReader.
+ *
+ * @see ParquetRecordReader
+ *
+ * @param <T> type of the materialized records
+ */
+public class HoodieRealtimeParquetRecordReader<T> extends RecordReader<Void, T> {

Review comment:
       I am not sure if Hive can support `mapreduce.RecordReader`. Seems like we use `mapred` because Hive use it. 




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



[GitHub] [hudi] umehrot2 edited a comment on pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
umehrot2 edited a comment on pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#issuecomment-643568699


   Like @vinothchandar I do agree with the **high level approach** here, and thanks for putting out this PR 👍  However, I would highly recommend both of you to check out https://github.com/apache/hudi/pull/1702/ which is along similar lines, and solves some of the issues I see in this PR:
   
   - Here we are instantiating another datasource/relation i.e. `HoodieRealtimeFileFormat` and `spark parquet` relation within `Snapshot relation` which has overheads associated with it, like spark having to form index again by listing the paths passed to `HoodieRealtimeFileFormat` and `spark parquet` relations to be able to instantiate them.
   
   - We are re-using the `ParquetFileFormat` reader and all of its functionalities like **vectorized reading** , **predicate pushdown**, **column pruning** without having to copy the over and maintain it internally.
   
   - We do not have to pass the expensive `map from parquet to log files` to each task. Instead it gives complete control over what goes into each task partition, and we send only the file and its corresponding mapping (in our case `external data file`, and in this case `log file`) over to the task. It is the very use to **RDD** interface to have that kind of control over the datasource we are building.
   
   Happy to have more in-depth discussion on this and help get this to completion.


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



[GitHub] [hudi] vinothchandar commented on a change in pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on a change in pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#discussion_r454749227



##########
File path: hudi-spark/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetRealtimeFileFormat.scala
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.spark.sql.execution.datasources.parquet
+
+import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapred.{FileSplit, JobConf}
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
+import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType}
+import org.apache.parquet.filter2.compat.FilterCompat
+import org.apache.parquet.filter2.predicate.FilterApi
+import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS
+import org.apache.parquet.hadoop.{ParquetFileReader, ParquetInputFormat, ParquetRecordReader}
+import org.apache.spark.TaskContext
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
+import org.apache.spark.sql.catalyst.expressions.{JoinedRow, UnsafeRow}
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+import java.net.URI
+import scala.collection.JavaConverters._
+
+/**
+ * This class is an extension of ParquetFileFormat from Spark SQL.
+ * The file split, record reader, record reader iterator are customized to read Hudi MOR table.
+ */
+class HoodieParquetRealtimeFileFormat extends ParquetFileFormat {

Review comment:
       @garyli1019 it should be possible to call `super.buildReader..()` get the `Iterator[InternalRow]` for the parquet base file alone and then use the class above to merge right? 
   It will avoid a ton of code from this file.. 




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



[GitHub] [hudi] vinothchandar commented on pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#issuecomment-661660112


   @garyli1019 should we close this in favor of #1848 ? 


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



[GitHub] [hudi] garyli1019 commented on a change in pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
garyli1019 commented on a change in pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#discussion_r445305619



##########
File path: hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala
##########
@@ -57,8 +57,7 @@ class DefaultSource extends RelationProvider
     if (path.isEmpty) {
       throw new HoodieException("'path' must be specified.")
     }
-
-    if (parameters(QUERY_TYPE_OPT_KEY).equals(QUERY_TYPE_SNAPSHOT_OPT_VAL)) {
+        if (parameters(QUERY_TYPE_OPT_KEY).equals(QUERY_TYPE_READ_OPTIMIZED_OPT_VAL)) {
       // this is just effectively RO view only, where `path` can contain a mix of

Review comment:
       This is hard without a path handler. Now we tell the user to add glob pattern to `load(basePath + "/*/*/*/*")` for COW table and the glob will be in the `basePath`. The Spark default `DataSource.apply().resolveRelation()` is able to handle the glob, but our custom relation not able to handle this. This is why our incremental relation requires `.load(basePath)` only. Udit's PR has this path handler so we will have a unified place to handle all the paths. 
   I think we can change the default option to `READ_OPTIMIZED`, so the user side has no impact. Currently spark datasource only supports snapshot on COW anyway, which is the same as READ_OPTIMIZED. We can switch back later with the path handler https://github.com/apache/hudi/pull/1702/files#diff-683cf2c70477ed6cc0a484a2ae494999R72




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



[GitHub] [hudi] garyli1019 commented on a change in pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
garyli1019 commented on a change in pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#discussion_r445306684



##########
File path: hudi-spark/src/main/scala/org/apache/hudi/SnapshotRelation.scala
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.table.{HoodieTableMetaClient, TableSchemaResolver}
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.hadoop.{HoodieParquetInputFormat, HoodieROTablePathFilter}
+import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.table.HoodieTable
+
+import org.apache.hadoop.mapred.JobConf
+import org.apache.log4j.LogManager
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{Row, SQLContext}
+import org.apache.spark.sql.sources.{BaseRelation, TableScan}
+import org.apache.spark.sql.types.StructType
+
+import java.util
+import scala.collection.JavaConverters._
+
+/**
+ * This is the Spark DataSourceV1 relation to read Hudi MOR table.

Review comment:
       This is not as efficient as the default `DataSource.apply().resolveRelation()` yet, due to the filter pushdown and column pruning is not supported yet. But we will get there soon...




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



[GitHub] [hudi] garyli1019 commented on a change in pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
garyli1019 commented on a change in pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#discussion_r438269270



##########
File path: hudi-spark/src/main/java/org/apache/hudi/realtime/HoodieRealtimeParquetRecordReader.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.realtime;
+
+import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.parquet.CorruptDeltaByteArrays;
+import org.apache.parquet.ParquetReadOptions;
+import org.apache.parquet.column.Encoding;
+import org.apache.parquet.filter2.compat.FilterCompat;
+import org.apache.parquet.filter2.compat.FilterCompat.Filter;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.ParquetRecordReader;
+import org.apache.parquet.hadoop.api.ReadSupport;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.FileMetaData;
+import org.apache.parquet.hadoop.util.ContextUtil;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.hadoop.util.counters.BenchmarkCounter;
+import org.apache.parquet.io.ParquetDecodingException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.apache.parquet.hadoop.ParquetInputFormat.SPLIT_FILES;
+
+/**
+ * Custom implementation of org.apache.parquet.hadoop.ParquetRecordReader.
+ * This class is a wrapper class. The real reader is the internalReader.
+ *
+ * @see ParquetRecordReader
+ *
+ * @param <T> type of the materialized records
+ */
+public class HoodieRealtimeParquetRecordReader<T> extends RecordReader<Void, T> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(HoodieRealtimeParquetRecordReader.class);
+  public final CompactedRealtimeParquetReader<T> internalReader;
+
+  /**
+   * @param readSupport Object which helps reads files of the given type, e.g. Thrift, Avro.
+   */
+  public HoodieRealtimeParquetRecordReader(ReadSupport<T> readSupport, HoodieRealtimeFileSplit split, JobConf job)
+      throws IOException {
+    this(readSupport, FilterCompat.NOOP, split, job);
+  }
+
+  /**
+   * @param readSupport Object which helps reads files of the given type, e.g. Thrift, Avro.
+   * @param filter for filtering individual records
+   */
+  public HoodieRealtimeParquetRecordReader(ReadSupport<T> readSupport, Filter filter, HoodieRealtimeFileSplit split, JobConf job)
+      throws IOException {
+    this.internalReader = new CompactedRealtimeParquetReader<T>(readSupport, filter, split, job);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void close() throws IOException {
+    internalReader.close();
+  }
+
+  /**
+   * always returns null.
+   */
+  @Override
+  public Void getCurrentKey() throws IOException, InterruptedException {
+    return null;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public T getCurrentValue() throws IOException,
+      InterruptedException {
+    return internalReader.getCurrentValue();
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public float getProgress() throws IOException, InterruptedException {
+    return internalReader.getProgress();
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void initialize(InputSplit inputSplit, TaskAttemptContext context)
+      throws IOException, InterruptedException {
+
+    if (ContextUtil.hasCounterMethod(context)) {
+      BenchmarkCounter.initCounterFromContext(context);
+    } else {
+      LOG.error(
+          String.format("Can not initialize counter because the class '%s' does not have a '.getCounterMethod'",
+              context.getClass().getCanonicalName()));
+    }
+
+    initializeInternalReader((HoodieRealtimeFileSplit) inputSplit, ContextUtil.getConfiguration(context));
+  }
+
+  public void initialize(InputSplit inputSplit, Configuration configuration, Reporter reporter)
+      throws IOException, InterruptedException {
+    BenchmarkCounter.initCounterFromReporter(reporter,configuration);
+    initializeInternalReader((HoodieRealtimeFileSplit) inputSplit, configuration);
+  }
+
+  private void initializeInternalReader(HoodieRealtimeFileSplit split, Configuration configuration) throws IOException {
+    Path path = split.getPath();
+    ParquetReadOptions.Builder optionsBuilder = HadoopReadOptions.builder(configuration);
+    optionsBuilder.withRange(split.getStart(), split.getStart() + split.getLength());
+
+    // open a reader with the metadata filter
+    ParquetFileReader reader = ParquetFileReader.open(
+        HadoopInputFile.fromPath(path, configuration), optionsBuilder.build());
+    if (!reader.getRowGroups().isEmpty()) {
+      checkDeltaByteArrayProblem(
+          reader.getFooter().getFileMetaData(), configuration,
+          reader.getRowGroups().get(0));
+    }
+
+    internalReader.initialize(reader, configuration);
+  }
+
+  private void checkDeltaByteArrayProblem(FileMetaData meta, Configuration conf, BlockMetaData block) {
+    if (conf.getBoolean(SPLIT_FILES, true)) {

Review comment:
       yes, this is from `org.apache.parquet.hadoop.ParquetRecordReader`. Many codes here are private so I have to move the whole thing over.




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



[GitHub] [hudi] garyli1019 commented on a change in pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
garyli1019 commented on a change in pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#discussion_r438271840



##########
File path: hudi-spark/src/main/java/org/apache/hudi/realtime/HoodieRealtimeParquetRecordReader.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.realtime;
+
+import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.parquet.CorruptDeltaByteArrays;
+import org.apache.parquet.ParquetReadOptions;
+import org.apache.parquet.column.Encoding;
+import org.apache.parquet.filter2.compat.FilterCompat;
+import org.apache.parquet.filter2.compat.FilterCompat.Filter;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.ParquetRecordReader;
+import org.apache.parquet.hadoop.api.ReadSupport;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.FileMetaData;
+import org.apache.parquet.hadoop.util.ContextUtil;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.hadoop.util.counters.BenchmarkCounter;
+import org.apache.parquet.io.ParquetDecodingException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.apache.parquet.hadoop.ParquetInputFormat.SPLIT_FILES;
+
+/**
+ * Custom implementation of org.apache.parquet.hadoop.ParquetRecordReader.
+ * This class is a wrapper class. The real reader is the internalReader.
+ *
+ * @see ParquetRecordReader
+ *
+ * @param <T> type of the materialized records
+ */
+public class HoodieRealtimeParquetRecordReader<T> extends RecordReader<Void, T> {

Review comment:
       We don't have much flexibility here if we wanna take advantage of Spark internal optimization. Even if I change the generic type to `UnsafeRow`, WholeStageCodeGen won't be able to use this class, which makes me not able to handle the merging inside the `RecordReader`, but luckily I can do it in the `Iterator`. So I need to strictly follow other data source implementation patterns inside Spark.




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



[GitHub] [hudi] codecov-commenter edited a comment on pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#issuecomment-643095877


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/1722?src=pr&el=h1) Report
   > Merging [#1722](https://codecov.io/gh/apache/hudi/pull/1722?src=pr&el=desc) into [master](https://codecov.io/gh/apache/hudi/commit/8919be6a5d8038db7265bfd7459d72fbd545f133&el=desc) will **decrease** coverage by `3.03%`.
   > The diff coverage is `1.44%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/1722/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/1722?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #1722      +/-   ##
   ============================================
   - Coverage     62.82%   59.79%   -3.04%     
   - Complexity     3437     3610     +173     
   ============================================
     Files           401      439      +38     
     Lines         17091    19183    +2092     
     Branches       1698     1946     +248     
   ============================================
   + Hits          10737    11470     +733     
   - Misses         5623     6924    +1301     
   - Partials        731      789      +58     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | #hudicli | `68.45% <ø> (?)` | `1430.00 <ø> (?)` | |
   | #hudiclient | `79.24% <ø> (ø)` | `1258.00 <ø> (ø)` | |
   | #hudicommon | `54.29% <ø> (+0.04%)` | `1486.00 <ø> (+1.00)` | |
   | #hudihadoopmr | `39.36% <ø> (ø)` | `163.00 <ø> (ø)` | |
   | #hudihivesync | `72.25% <ø> (ø)` | `121.00 <ø> (ø)` | |
   | #hudispark | `36.39% <1.44%> (-7.83%)` | `76.00 <0.00> (ø)` | |
   | #huditimelineservice | `63.47% <ø> (ø)` | `47.00 <ø> (ø)` | |
   | #hudiutilities | `73.75% <ø> (ø)` | `287.00 <ø> (ø)` | |
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/1722?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [.../main/scala/org/apache/hudi/SnapshotRelation.scala](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2h1ZGkvU25hcHNob3RSZWxhdGlvbi5zY2FsYQ==) | `0.00% <0.00%> (ø)` | `0.00 <0.00> (?)` | |
   | [...urces/parquet/HoodieMergedParquetRowIterator.scala](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL3NwYXJrL3NxbC9leGVjdXRpb24vZGF0YXNvdXJjZXMvcGFycXVldC9Ib29kaWVNZXJnZWRQYXJxdWV0Um93SXRlcmF0b3Iuc2NhbGE=) | `0.00% <0.00%> (ø)` | `0.00 <0.00> (?)` | |
   | [...rces/parquet/HoodieParquetRealtimeFileFormat.scala](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL3NwYXJrL3NxbC9leGVjdXRpb24vZGF0YXNvdXJjZXMvcGFycXVldC9Ib29kaWVQYXJxdWV0UmVhbHRpbWVGaWxlRm9ybWF0LnNjYWxh) | `0.00% <0.00%> (ø)` | `0.00 <0.00> (?)` | |
   | [...src/main/scala/org/apache/hudi/DefaultSource.scala](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2h1ZGkvRGVmYXVsdFNvdXJjZS5zY2FsYQ==) | `65.71% <33.33%> (-4.88%)` | `7.00 <0.00> (ø)` | |
   | [...main/scala/org/apache/hudi/DataSourceOptions.scala](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2h1ZGkvRGF0YVNvdXJjZU9wdGlvbnMuc2NhbGE=) | `93.54% <100.00%> (ø)` | `0.00 <0.00> (ø)` | |
   | [.../main/scala/org/apache/hudi/cli/SparkHelpers.scala](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree#diff-aHVkaS1jbGkvc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9odWRpL2NsaS9TcGFya0hlbHBlcnMuc2NhbGE=) | `0.00% <0.00%> (ø)` | `0.00% <0.00%> (?%)` | |
   | [...src/main/java/org/apache/hudi/cli/TableHeader.java](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree#diff-aHVkaS1jbGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY2xpL1RhYmxlSGVhZGVyLmphdmE=) | `77.77% <0.00%> (ø)` | `5.00% <0.00%> (?%)` | |
   | [...org/apache/hudi/cli/commands/RollbacksCommand.java](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree#diff-aHVkaS1jbGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY2xpL2NvbW1hbmRzL1JvbGxiYWNrc0NvbW1hbmQuamF2YQ==) | `94.00% <0.00%> (ø)` | `8.00% <0.00%> (?%)` | |
   | [...ava/org/apache/hudi/cli/commands/UtilsCommand.java](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree#diff-aHVkaS1jbGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY2xpL2NvbW1hbmRzL1V0aWxzQ29tbWFuZC5qYXZh) | `100.00% <0.00%> (ø)` | `3.00% <0.00%> (?%)` | |
   | ... and [34 more](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree-more) | |
   


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



[GitHub] [hudi] vinothchandar commented on a change in pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on a change in pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#discussion_r450169252



##########
File path: hudi-spark/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetRealtimeFileFormat.scala
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.spark.sql.execution.datasources.parquet
+
+import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapred.{FileSplit, JobConf}
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
+import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType}
+import org.apache.parquet.filter2.compat.FilterCompat
+import org.apache.parquet.filter2.predicate.FilterApi
+import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS
+import org.apache.parquet.hadoop.{ParquetFileReader, ParquetInputFormat, ParquetRecordReader}
+import org.apache.spark.TaskContext
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
+import org.apache.spark.sql.catalyst.expressions.{JoinedRow, UnsafeRow}
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+import java.net.URI
+import scala.collection.JavaConverters._
+
+/**
+ * This class is an extension of ParquetFileFormat from Spark SQL.
+ * The file split, record reader, record reader iterator are customized to read Hudi MOR table.
+ */
+class HoodieParquetRealtimeFileFormat extends ParquetFileFormat {

Review comment:
       still mulling if this file can be simpler.. working on it.. 
   
   heads up : if we have to re-use code, we need to attribute in NOTICE/LICENSE as well.




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



[GitHub] [hudi] garyli1019 commented on a change in pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
garyli1019 commented on a change in pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#discussion_r438277002



##########
File path: hudi-spark/src/main/java/org/apache/hudi/realtime/AbstractRealtimeParquetReader.java
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.realtime;
+
+import org.apache.hudi.common.table.log.LogReaderUtils;
+import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit;
+import org.apache.hudi.hadoop.config.HoodieRealtimeConfig;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.ParquetReadOptions;
+import org.apache.parquet.avro.AvroSchemaConverter;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.filter2.compat.FilterCompat;
+import org.apache.parquet.filter2.compat.FilterCompat.Filter;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.UnmaterializableRecordCounter;
+import org.apache.parquet.hadoop.api.InitContext;
+import org.apache.parquet.hadoop.api.ReadSupport;
+import org.apache.parquet.hadoop.metadata.FileMetaData;
+import org.apache.parquet.hadoop.util.counters.BenchmarkCounter;
+import org.apache.parquet.io.ColumnIOFactory;
+import org.apache.parquet.io.MessageColumnIO;
+import org.apache.parquet.io.api.RecordMaterializer;
+import org.apache.parquet.schema.MessageType;
+import org.mortbay.log.Log;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.parquet.Preconditions.checkNotNull;
+import static org.apache.parquet.hadoop.ParquetInputFormat.RECORD_FILTERING_ENABLED;
+import static org.apache.parquet.hadoop.ParquetInputFormat.STRICT_TYPE_CHECKING;
+
+/**
+ * This class is customized from org.apache.parquet.hadoop.InternalParquetRecordReader combining with AbstractRealtimeRecordReader.

Review comment:
       agree. The problem here is this class is private like many other codes here. I will go back and see if I can warp more things up.




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



[GitHub] [hudi] garyli1019 commented on a change in pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
garyli1019 commented on a change in pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#discussion_r454762535



##########
File path: hudi-spark/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetRealtimeFileFormat.scala
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.spark.sql.execution.datasources.parquet
+
+import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapred.{FileSplit, JobConf}
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
+import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType}
+import org.apache.parquet.filter2.compat.FilterCompat
+import org.apache.parquet.filter2.predicate.FilterApi
+import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS
+import org.apache.parquet.hadoop.{ParquetFileReader, ParquetInputFormat, ParquetRecordReader}
+import org.apache.spark.TaskContext
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
+import org.apache.spark.sql.catalyst.expressions.{JoinedRow, UnsafeRow}
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+import java.net.URI
+import scala.collection.JavaConverters._
+
+/**
+ * This class is an extension of ParquetFileFormat from Spark SQL.
+ * The file split, record reader, record reader iterator are customized to read Hudi MOR table.
+ */
+class HoodieParquetRealtimeFileFormat extends ParquetFileFormat {

Review comment:
       I think this could come with a schema issue. `super.buildReader..()` could use a vectorized reader that only read a few columns and right now I didn't implement merging `Avro` with `InternalRow` with different schema. Right now we can only merge row by row with the same schema.
   We will get rid of this `FileFormat` approach. The RDD approach is cleaner. https://github.com/apache/hudi/pull/1702/files#diff-809772c649e85ffb321055d9871e37e0R75




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



[GitHub] [hudi] vinothchandar commented on a change in pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on a change in pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#discussion_r454670023



##########
File path: hudi-spark/src/main/scala/org/apache/hudi/DataSourceOptions.scala
##########
@@ -65,7 +66,7 @@ object DataSourceReadOptions {
     * This eases migration from old configs to new configs.
     */
   def translateViewTypesToQueryTypes(optParams: Map[String, String]) : Map[String, String] = {
-    val translation = Map(VIEW_TYPE_READ_OPTIMIZED_OPT_VAL -> QUERY_TYPE_SNAPSHOT_OPT_VAL,
+    val translation = Map(VIEW_TYPE_READ_OPTIMIZED_OPT_VAL -> QUERY_TYPE_READ_OPTIMIZED_OPT_VAL,

Review comment:
       > f they are using VIEW_TYPE_READ_OPTIMIZED_OPT_VAL(deprecated) on MOR in their code, after upgrade to the next release, the code will run snapshot query instead of RO query.
   
   we have been logging warning for sometime on the use of the deprecated configs. and so I think its fair to do the right thing here moving forward and call this out in the release notes. Let me push some changes.. 




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



[GitHub] [hudi] garyli1019 commented on a change in pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
garyli1019 commented on a change in pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#discussion_r450390920



##########
File path: hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala
##########
@@ -58,26 +60,28 @@ class DefaultSource extends RelationProvider
       throw new HoodieException("'path' must be specified.")
     }
 
+    // Try to create hoodie table meta client from the give path
+    // TODO: Smarter path handling
+    val metaClient = try {
+      val conf = sqlContext.sparkContext.hadoopConfiguration
+      Option(new HoodieTableMetaClient(conf, path.get, true))

Review comment:
       At this point we have:
   - RO, Snapshot query for COW: Support glob and basePath
   - Snapshot for MOR: only support basePath
   - Incremental: Only support basePath
   
   What I am trying to do here is:
   - If the `path` contains glob, fall back to RO. This is the current behavior. Create metaClient will throw an Exception but handled below.
   - If the `path` is basePath, we create the metaClient. If COW table, go RO relation. If MOR, go snapshot relation.

##########
File path: hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala
##########
@@ -123,4 +127,25 @@ class DefaultSource extends RelationProvider
   }
 
   override def shortName(): String = "hudi"
+
+  private def getReadOptimizedView(sqlContext: SQLContext,

Review comment:
       sure, will do

##########
File path: hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala
##########
@@ -58,26 +60,28 @@ class DefaultSource extends RelationProvider
       throw new HoodieException("'path' must be specified.")
     }
 
+    // Try to create hoodie table meta client from the give path
+    // TODO: Smarter path handling
+    val metaClient = try {
+      val conf = sqlContext.sparkContext.hadoopConfiguration
+      Option(new HoodieTableMetaClient(conf, path.get, true))
+    } catch {
+      case e: HoodieException => Option.empty

Review comment:
       I used this as a flag that the `path` is not basePath. This is a temporary solution to not change the query behavior.
   This will be handled better with: https://github.com/apache/hudi/pull/1702/files#diff-9a21766ebf794414f94b302bcb968f41R31

##########
File path: hudi-spark/src/main/scala/org/apache/hudi/DataSourceOptions.scala
##########
@@ -65,7 +66,7 @@ object DataSourceReadOptions {
     * This eases migration from old configs to new configs.
     */
   def translateViewTypesToQueryTypes(optParams: Map[String, String]) : Map[String, String] = {
-    val translation = Map(VIEW_TYPE_READ_OPTIMIZED_OPT_VAL -> QUERY_TYPE_SNAPSHOT_OPT_VAL,
+    val translation = Map(VIEW_TYPE_READ_OPTIMIZED_OPT_VAL -> QUERY_TYPE_READ_OPTIMIZED_OPT_VAL,

Review comment:
       Sorry, my previous comments are confusing, let me rephrase.
   What I trying to do here is to not change the query behavior. Since before we don't support snapshot query for MOR, so RO and snapshot query type will behave the same regardless of its COW or MOR. 
   If we don't change this mapping, the user will have different behavior after upgrade to the next release. If they are using `VIEW_TYPE_READ_OPTIMIZED_OPT_VAL(deprecated)` on MOR in their code, after upgrade to the next release, the code will run snapshot query instead of RO query. This could give users surprise even this key was deprecated.

##########
File path: hudi-spark/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetRealtimeFileFormat.scala
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.spark.sql.execution.datasources.parquet
+
+import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapred.{FileSplit, JobConf}
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
+import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType}
+import org.apache.parquet.filter2.compat.FilterCompat
+import org.apache.parquet.filter2.predicate.FilterApi
+import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS
+import org.apache.parquet.hadoop.{ParquetFileReader, ParquetInputFormat, ParquetRecordReader}
+import org.apache.spark.TaskContext
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
+import org.apache.spark.sql.catalyst.expressions.{JoinedRow, UnsafeRow}
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+import java.net.URI
+import scala.collection.JavaConverters._
+
+/**
+ * This class is an extension of ParquetFileFormat from Spark SQL.
+ * The file split, record reader, record reader iterator are customized to read Hudi MOR table.
+ */
+class HoodieParquetRealtimeFileFormat extends ParquetFileFormat {

Review comment:
       If we use the `FileFormat` approach, we probably can't avoid copy some Spark code. For datasource V2, we need to copy more code since Spark 3 use `case class` for all the `FileFormat`
   I will try to use udit's `RDD` approach https://github.com/apache/hudi/pull/1702/files#diff-809772c649e85ffb321055d9871e37e0R39
   I think that's doable. In that approach, we can get rid of this, but need to try that after his PR merged. Will need to reuse many code from that PR.




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



[GitHub] [hudi] codecov-commenter edited a comment on pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#issuecomment-643095877






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



[GitHub] [hudi] garyli1019 commented on a change in pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
garyli1019 commented on a change in pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#discussion_r438490392



##########
File path: hudi-spark/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieRealtimeInputFormat.scala
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.spark.sql.execution.datasources.parquet
+
+import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit
+import org.apache.hudi.realtime.HoodieRealtimeParquetRecordReader
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapred.{FileSplit, JobConf}
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
+import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType}
+import org.apache.parquet.filter2.compat.FilterCompat
+import org.apache.parquet.filter2.predicate.FilterApi
+import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS
+import org.apache.parquet.hadoop.{ParquetFileReader, ParquetInputFormat}
+import org.apache.spark.TaskContext
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
+import org.apache.spark.sql.catalyst.expressions.{JoinedRow, UnsafeRow}
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+import java.net.URI
+import scala.collection.JavaConverters._
+
+/**
+ * This class is an extension of ParquetFileFormat from Spark SQL.
+ * The file split, record reader, record reader iterator are customized to read Hudi MOR table.
+ */
+class HoodieRealtimeInputFormat extends ParquetFileFormat {
+  //TODO: Better usage of this short name.
+  override def shortName(): String = "hudi.realtime"
+  override def toString(): String = "hudi.realtime"
+
+  override def buildReaderWithPartitionValues(sparkSession: SparkSession,
+                                               dataSchema: StructType,
+                                               partitionSchema: StructType,
+                                               requiredSchema: StructType,
+                                               filters: Seq[Filter],
+                                               options: Map[String, String],
+                                               hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = {
+    hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName)

Review comment:
       Do you mean use `ParquetInputFormat` as superclass? We have to use `ParquetFileFormat` from Spark.




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



[GitHub] [hudi] garyli1019 commented on a change in pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
garyli1019 commented on a change in pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#discussion_r450584388



##########
File path: hudi-spark/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieMergedParquetRowIterator.scala
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.spark.sql.execution.datasources.parquet
+
+import org.apache.hadoop.mapred.JobConf
+import org.apache.hudi.common.fs.FSUtils
+import org.apache.hudi.common.table.log.{HoodieMergedLogRecordScanner, LogReaderUtils}
+import org.apache.hudi.hadoop.config.HoodieRealtimeConfig
+import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit
+import org.apache.parquet.hadoop.ParquetRecordReader
+import org.apache.avro.Schema
+import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.HOODIE_RECORD_KEY_COL_POS
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.avro.{AvroDeserializer, SchemaConverters}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, UnsafeRow}
+import org.apache.spark.sql.types.StructType
+
+import org.apache.hudi.common.model.HoodieRecord
+import org.apache.hudi.common.model.HoodieRecordPayload
+
+import java.io.Closeable
+import java.util
+import scala.util.Try
+
+/**
+ * This class is the iterator for Hudi MOR table.
+ * Log files are scanned on initialization.
+ * This iterator will read the parquet file first and skip the record if it present in the log file.
+ * Then read the log file.
+ * Custom payload is not supported yet. This combining logic is matching with [OverwriteWithLatestAvroPayload]
+ * @param rowReader ParquetRecordReader
+ */
+class HoodieMergedParquetRowIterator(private[this] var rowReader: ParquetRecordReader[UnsafeRow],

Review comment:
       https://github.com/apache/spark/blob/branch-2.4/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/RecordReaderIterator.scala#L32




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



[GitHub] [hudi] garyli1019 commented on a change in pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
garyli1019 commented on a change in pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#discussion_r450584271



##########
File path: hudi-spark/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetRealtimeFileFormat.scala
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.spark.sql.execution.datasources.parquet
+
+import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapred.{FileSplit, JobConf}
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
+import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType}
+import org.apache.parquet.filter2.compat.FilterCompat
+import org.apache.parquet.filter2.predicate.FilterApi
+import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS
+import org.apache.parquet.hadoop.{ParquetFileReader, ParquetInputFormat, ParquetRecordReader}
+import org.apache.spark.TaskContext
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
+import org.apache.spark.sql.catalyst.expressions.{JoinedRow, UnsafeRow}
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+import java.net.URI
+import scala.collection.JavaConverters._
+
+/**
+ * This class is an extension of ParquetFileFormat from Spark SQL.
+ * The file split, record reader, record reader iterator are customized to read Hudi MOR table.
+ */
+class HoodieParquetRealtimeFileFormat extends ParquetFileFormat {

Review comment:
       https://github.com/apache/spark/blob/branch-2.4/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala#L295




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



[GitHub] [hudi] garyli1019 closed pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
garyli1019 closed pull request #1722:
URL: https://github.com/apache/hudi/pull/1722


   


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



[GitHub] [hudi] garyli1019 commented on a change in pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
garyli1019 commented on a change in pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#discussion_r438261535



##########
File path: hudi-spark/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieRealtimeInputFormat.scala
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.spark.sql.execution.datasources.parquet
+
+import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit
+import org.apache.hudi.realtime.HoodieRealtimeParquetRecordReader
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapred.{FileSplit, JobConf}
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
+import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType}
+import org.apache.parquet.filter2.compat.FilterCompat
+import org.apache.parquet.filter2.predicate.FilterApi
+import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS
+import org.apache.parquet.hadoop.{ParquetFileReader, ParquetInputFormat}
+import org.apache.spark.TaskContext
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
+import org.apache.spark.sql.catalyst.expressions.{JoinedRow, UnsafeRow}
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+import java.net.URI
+import scala.collection.JavaConverters._
+
+/**
+ * This class is an extension of ParquetFileFormat from Spark SQL.
+ * The file split, record reader, record reader iterator are customized to read Hudi MOR table.
+ */
+class HoodieRealtimeInputFormat extends ParquetFileFormat {

Review comment:
       right. some methods are private to the package




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



[GitHub] [hudi] garyli1019 commented on a change in pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
garyli1019 commented on a change in pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#discussion_r454694800



##########
File path: hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala
##########
@@ -58,26 +60,28 @@ class DefaultSource extends RelationProvider
       throw new HoodieException("'path' must be specified.")
     }
 
+    // Try to create hoodie table meta client from the give path
+    // TODO: Smarter path handling
+    val metaClient = try {
+      val conf = sqlContext.sparkContext.hadoopConfiguration
+      Option(new HoodieTableMetaClient(conf, path.get, true))

Review comment:
       Udit's PR has this path handling. Should we merge part of his PR first? https://github.com/apache/hudi/pull/1702/files#diff-9a21766ebf794414f94b302bcb968f41R31
   With this, we can handle user to `.load(basePath)` or `.load(basePath + "/*/*")` for COW, MOR and incremental.




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



[GitHub] [hudi] vinothchandar commented on a change in pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on a change in pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#discussion_r454670467



##########
File path: hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala
##########
@@ -58,26 +60,28 @@ class DefaultSource extends RelationProvider
       throw new HoodieException("'path' must be specified.")
     }
 
+    // Try to create hoodie table meta client from the give path
+    // TODO: Smarter path handling
+    val metaClient = try {
+      val conf = sqlContext.sparkContext.hadoopConfiguration
+      Option(new HoodieTableMetaClient(conf, path.get, true))

Review comment:
       >Snapshot for MOR: only support basePath
   
   Let me think about this more. We need to support some form of globbing for MOR/Snapshot query. 




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



[GitHub] [hudi] garyli1019 commented on a change in pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
garyli1019 commented on a change in pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#discussion_r444623018



##########
File path: hudi-spark/src/main/scala/org/apache/hudi/DataSourceOptions.scala
##########
@@ -48,7 +49,7 @@ object DataSourceReadOptions {
   val QUERY_TYPE_SNAPSHOT_OPT_VAL = "snapshot"
   val QUERY_TYPE_READ_OPTIMIZED_OPT_VAL = "read_optimized"
   val QUERY_TYPE_INCREMENTAL_OPT_VAL = "incremental"
-  val DEFAULT_QUERY_TYPE_OPT_VAL: String = QUERY_TYPE_SNAPSHOT_OPT_VAL
+  val DEFAULT_QUERY_TYPE_OPT_VAL: String = QUERY_TYPE_READ_OPTIMIZED_OPT_VAL

Review comment:
       This is like incremental relation. We need the base path to get the hudi table view. We need a smarter path handler to handle the glob path and udit's pr have this.

##########
File path: hudi-spark/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieRealtimeInputFormat.scala
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.spark.sql.execution.datasources.parquet
+
+import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapred.{FileSplit, JobConf}
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
+import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType}
+import org.apache.parquet.filter2.compat.FilterCompat
+import org.apache.parquet.filter2.predicate.FilterApi
+import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS
+import org.apache.parquet.hadoop.{ParquetFileReader, ParquetInputFormat, ParquetRecordReader}
+import org.apache.spark.TaskContext
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
+import org.apache.spark.sql.catalyst.expressions.{JoinedRow, UnsafeRow}
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+import java.net.URI
+import scala.collection.JavaConverters._
+
+/**
+ * This class is an extension of ParquetFileFormat from Spark SQL.
+ * The file split, record reader, record reader iterator are customized to read Hudi MOR table.
+ */
+class HoodieRealtimeInputFormat extends ParquetFileFormat {
+  //TODO: Better usage of this short name.
+  override def shortName(): String = "hudi.snapshot"
+  override def toString(): String = "hudi.snapshot"
+
+  override def buildReaderWithPartitionValues(sparkSession: SparkSession,
+                                              dataSchema: StructType,
+                                              partitionSchema: StructType,
+                                              requiredSchema: StructType,
+                                              filters: Seq[Filter], options: Map[String, String],
+                                              hadoopConf: Configuration): (PartitionedFile) =>
+    Iterator[InternalRow] = {
+    hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName)
+    hadoopConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, requiredSchema.json)
+    hadoopConf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, requiredSchema.json)
+    hadoopConf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key,
+      sparkSession.sessionState.conf.sessionLocalTimeZone)
+    hadoopConf.setBoolean(SQLConf.CASE_SENSITIVE.key,
+      sparkSession.sessionState.conf.caseSensitiveAnalysis)
+
+    ParquetWriteSupport.setSchema(requiredSchema, hadoopConf)
+
+    // Sets flags for `ParquetToSparkSchemaConverter`
+    hadoopConf.setBoolean(
+      SQLConf.PARQUET_BINARY_AS_STRING.key,
+      sparkSession.sessionState.conf.isParquetBinaryAsString)
+    hadoopConf.setBoolean(
+      SQLConf.PARQUET_INT96_AS_TIMESTAMP.key,
+      sparkSession.sessionState.conf.isParquetINT96AsTimestamp)
+
+    val broadcastedHadoopConf =
+      sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf))
+
+    // TODO: if you move this into the closure it reverts to the default values.
+    // If true, enable using the custom RecordReader for parquet. This only works for
+    // a subset of the types (no complex types).
+    //val resultSchema = StructType(partitionSchema.fields ++ requiredSchema.fields)
+    val sqlConf = sparkSession.sessionState.conf
+    val enableRecordFilter: Boolean = sqlConf.parquetRecordFilterEnabled
+    val timestampConversion: Boolean = sqlConf.isParquetINT96TimestampConversion
+    val enableParquetFilterPushDown: Boolean = sqlConf.parquetFilterPushDown
+    // Whole stage codegen (PhysicalRDD) is able to deal with batches directly
+    //val returningBatch = supportBatch(sparkSession, resultSchema)
+    val pushDownDate = sqlConf.parquetFilterPushDownDate
+    val pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp
+    val pushDownDecimal = sqlConf.parquetFilterPushDownDecimal
+    val pushDownStringStartWith = sqlConf.parquetFilterPushDownStringStartWith
+    val pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold
+    val isCaseSensitive = sqlConf.caseSensitiveAnalysis
+
+    (file: PartitionedFile) => {
+      assert(file.partitionValues.numFields == partitionSchema.size)
+
+      val sharedConf = broadcastedHadoopConf.value.value
+      val fileSplit =
+        new FileSplit(new Path(new URI(file.filePath)), file.start, file.length, new Array[String](0))

Review comment:
       Could be only a portion of a file. From spark description `A part (i.e. "block") of a single file that should be read, along with partition column values that need to be prepended to each row.`

##########
File path: hudi-spark/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetRecordReaderIterator.scala
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.spark.sql.execution.datasources.parquet

Review comment:
       correct

##########
File path: hudi-spark/src/main/scala/org/apache/hudi/DataSourceOptions.scala
##########
@@ -65,7 +66,7 @@ object DataSourceReadOptions {
     * This eases migration from old configs to new configs.
     */
   def translateViewTypesToQueryTypes(optParams: Map[String, String]) : Map[String, String] = {
-    val translation = Map(VIEW_TYPE_READ_OPTIMIZED_OPT_VAL -> QUERY_TYPE_SNAPSHOT_OPT_VAL,
+    val translation = Map(VIEW_TYPE_READ_OPTIMIZED_OPT_VAL -> QUERY_TYPE_READ_OPTIMIZED_OPT_VAL,

Review comment:
       I got confused by the naming sometimes...
   So for COW table, snapshot view = read optimized view
   for MOR, snapshot view and read optimized view are different things.
   With bootstrap, we will have one more view. 
   Can we call `read optimized view -> parquet only(including bootstrap)` `snapshot view -> parquet(with bootstrap) merge with log` regardless of table type?

##########
File path: hudi-spark/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetRecordReaderIterator.scala
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.spark.sql.execution.datasources.parquet
+
+import org.apache.hadoop.mapred.JobConf
+import org.apache.hudi.common.fs.FSUtils
+import org.apache.hudi.common.table.log.{HoodieMergedLogRecordScanner, LogReaderUtils}
+import org.apache.hudi.hadoop.config.HoodieRealtimeConfig
+import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit
+import org.apache.parquet.hadoop.ParquetRecordReader
+
+import org.apache.avro.Schema
+import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.HOODIE_RECORD_KEY_COL_POS
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.avro.{AvroDeserializer, SchemaConverters}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, UnsafeRow}
+import org.apache.spark.sql.types.StructType
+import java.io.Closeable
+import java.util
+
+import scala.util.Try
+
+/**
+ * This class is the iterator for Hudi MOR table.
+ * Log files are scanned on initialization.
+ * This iterator will read the parquet file first and skip the record if it present in the log file.
+ * Then read the log file.
+ * Custom payload is not supported yet. This combining logic is matching with [OverwriteWithLatestAvroPayload]
+ * @param rowReader HoodieRealtimeParquetRecordReader
+ */
+class HoodieParquetRecordReaderIterator(private[this] var rowReader: ParquetRecordReader[UnsafeRow],
+                                        private[this] val split: HoodieRealtimeFileSplit,
+                                        private[this] val jobConf: JobConf) extends Iterator[UnsafeRow] with Closeable with Logging {
+  private[this] var havePair = false
+  private[this] var finished = false
+  private[this] var parquetFinished = false
+
+  private[this] var deltaRecordMap: util.Map[String,org.apache.hudi.common.model
+  .HoodieRecord[_ <: org.apache.hudi.common.model.HoodieRecordPayload[_ <: org.apache.hudi.common.model.HoodieRecordPayload[_ <: AnyRef]]]] = _

Review comment:
       good catch

##########
File path: hudi-spark/src/main/scala/org/apache/hudi/SnapshotRelation.scala
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.table.{HoodieTableMetaClient, TableSchemaResolver}
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.hadoop.{HoodieParquetInputFormat, HoodieROTablePathFilter}
+import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.table.HoodieTable
+
+import org.apache.hadoop.mapred.JobConf
+import org.apache.log4j.LogManager
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{Row, SQLContext}
+import org.apache.spark.sql.sources.{BaseRelation, TableScan}
+import org.apache.spark.sql.types.StructType
+
+import java.util
+import scala.collection.JavaConverters._
+
+/**
+ * This is the Spark DataSourceV1 relation to read Hudi MOR table.
+ * @param sqlContext
+ * @param basePath
+ * @param optParams
+ * @param userSchema
+ */
+class SnapshotRelation(val sqlContext: SQLContext,
+                       val basePath: String,
+                       val optParams: Map[String, String],
+                       val userSchema: StructType) extends BaseRelation with TableScan {
+
+  private val log = LogManager.getLogger(classOf[SnapshotRelation])
+  private val conf = sqlContext.sparkContext.hadoopConfiguration
+
+  // Set config for listStatus() in HoodieParquetInputFormat
+  // TODO(garyli): Switch to bootstrap file listing methods
+  conf.setClass(
+    "mapreduce.input.pathFilter.class",
+    classOf[HoodieROTablePathFilter],
+    classOf[org.apache.hadoop.fs.PathFilter])
+  conf.setStrings("mapreduce.input.fileinputformat.inputdir", basePath)
+  conf.setStrings("mapreduce.input.fileinputformat.input.dir.recursive", "true")
+
+  private val HoodieInputFormat = new HoodieParquetInputFormat
+  HoodieInputFormat.setConf(conf)
+  private val fileStatus = HoodieInputFormat.listStatus(new JobConf(conf))
+  private val fileGroup = HoodieRealtimeInputFormatUtils.groupLogsByBaseFile(conf, util.Arrays.stream(fileStatus)).asScala
+
+  // Split the file group to: parquet file without a matching log file, parquet file need to merge with log files
+  private val parquetWithoutLogPaths: List[String] = fileGroup.filter(p => p._2.size() == 0).keys.toList
+  private val fileWithLogMap: Map[String, String] = fileGroup.filter(p => p._2.size() > 0).map{ case(k, v) => (k, v.asScala.toList.mkString(","))}.toMap
+
+  if (log.isDebugEnabled) {
+    log.debug("All parquet files" + fileStatus.map(s => s.getPath.toString).mkString(","))
+    log.debug("ParquetWithoutLogPaths" + parquetWithoutLogPaths.mkString(","))
+    log.debug("ParquetWithLogPaths" + fileWithLogMap.map(m => s"${m._1}:${m._2}").mkString(","))
+  }
+
+  // Add log file map to options
+  private val finalOps = optParams ++ fileWithLogMap
+
+  // Load Hudi metadata
+  val metaClient = new HoodieTableMetaClient(conf, basePath, true)
+  private val hoodieTable = HoodieTable.create(metaClient, HoodieWriteConfig.newBuilder().withPath(basePath).build(), conf)
+
+  private val commitTimeline = hoodieTable.getMetaClient.getCommitsAndCompactionTimeline
+  if (commitTimeline.empty()) {

Review comment:
       good point.

##########
File path: hudi-spark/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetRecordReaderIterator.scala
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.spark.sql.execution.datasources.parquet
+
+import org.apache.hadoop.mapred.JobConf
+import org.apache.hudi.common.fs.FSUtils
+import org.apache.hudi.common.table.log.{HoodieMergedLogRecordScanner, LogReaderUtils}
+import org.apache.hudi.hadoop.config.HoodieRealtimeConfig
+import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit
+import org.apache.parquet.hadoop.ParquetRecordReader
+
+import org.apache.avro.Schema
+import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.HOODIE_RECORD_KEY_COL_POS
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.avro.{AvroDeserializer, SchemaConverters}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, UnsafeRow}
+import org.apache.spark.sql.types.StructType
+import java.io.Closeable
+import java.util
+
+import scala.util.Try
+
+/**
+ * This class is the iterator for Hudi MOR table.
+ * Log files are scanned on initialization.
+ * This iterator will read the parquet file first and skip the record if it present in the log file.
+ * Then read the log file.
+ * Custom payload is not supported yet. This combining logic is matching with [OverwriteWithLatestAvroPayload]
+ * @param rowReader HoodieRealtimeParquetRecordReader
+ */
+class HoodieParquetRecordReaderIterator(private[this] var rowReader: ParquetRecordReader[UnsafeRow],
+                                        private[this] val split: HoodieRealtimeFileSplit,
+                                        private[this] val jobConf: JobConf) extends Iterator[UnsafeRow] with Closeable with Logging {
+  private[this] var havePair = false
+  private[this] var finished = false
+  private[this] var parquetFinished = false
+
+  private[this] var deltaRecordMap: util.Map[String,org.apache.hudi.common.model
+  .HoodieRecord[_ <: org.apache.hudi.common.model.HoodieRecordPayload[_ <: org.apache.hudi.common.model.HoodieRecordPayload[_ <: AnyRef]]]] = _
+  private[this] var deltaRecordKeys: util.Set[String] = _
+  private[this] var deltaIter: util.Iterator[String] = _
+  private[this] var avroSchema: Schema = _
+  private[this] var sparkTypes: StructType = _
+  private[this] var converter: AvroDeserializer = _
+
+  // SPARK-23457 Register a task completion lister before `initialization`.

Review comment:
       yea, doesn't really make sense to put it here

##########
File path: hudi-spark/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetRecordReaderIterator.scala
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.spark.sql.execution.datasources.parquet
+
+import org.apache.avro.Schema
+import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.HOODIE_RECORD_KEY_COL_POS
+import org.apache.hudi.realtime.HoodieRealtimeParquetRecordReader
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.avro.{AvroDeserializer, SchemaConverters}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, UnsafeRow}
+import org.apache.spark.sql.types.StructType
+import java.io.Closeable
+import java.util
+
+/**
+ * This class is the iterator for Hudi MOR table.
+ * This iterator will read the parquet file first and skip the record if it present in the log file.
+ * Then read the log file.
+ * Custom payload is not supported yet. This combining logic is matching with [OverwriteWithLatestAvroPayload]

Review comment:
       This goes back to the MapReduce v1 vs v2 API. Hive use V1 and Spark use V2. Spark also warp V2(from parquet) into its own record reader. I couldn't find a way to make Hive and Spark share one record reader. The vectorized reader would be another story as well.

##########
File path: hudi-spark/src/main/scala/org/apache/hudi/SnapshotRelation.scala
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.table.{HoodieTableMetaClient, TableSchemaResolver}
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.hadoop.{HoodieParquetInputFormat, HoodieROTablePathFilter}
+import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.table.HoodieTable
+
+import org.apache.hadoop.mapred.JobConf
+import org.apache.log4j.LogManager
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{Row, SQLContext}
+import org.apache.spark.sql.sources.{BaseRelation, TableScan}
+import org.apache.spark.sql.types.StructType
+
+import java.util
+import scala.collection.JavaConverters._
+
+/**
+ * This is the Spark DataSourceV1 relation to read Hudi MOR table.
+ * @param sqlContext
+ * @param basePath
+ * @param optParams
+ * @param userSchema
+ */
+class SnapshotRelation(val sqlContext: SQLContext,
+                       val basePath: String,
+                       val optParams: Map[String, String],
+                       val userSchema: StructType) extends BaseRelation with TableScan {
+
+  private val log = LogManager.getLogger(classOf[SnapshotRelation])
+  private val conf = sqlContext.sparkContext.hadoopConfiguration
+
+  // Set config for listStatus() in HoodieParquetInputFormat
+  // TODO(garyli): Switch to bootstrap file listing methods
+  conf.setClass(
+    "mapreduce.input.pathFilter.class",
+    classOf[HoodieROTablePathFilter],
+    classOf[org.apache.hadoop.fs.PathFilter])
+  conf.setStrings("mapreduce.input.fileinputformat.inputdir", basePath)
+  conf.setStrings("mapreduce.input.fileinputformat.input.dir.recursive", "true")
+
+  private val HoodieInputFormat = new HoodieParquetInputFormat
+  HoodieInputFormat.setConf(conf)
+  private val fileStatus = HoodieInputFormat.listStatus(new JobConf(conf))
+  private val fileGroup = HoodieRealtimeInputFormatUtils.groupLogsByBaseFile(conf, util.Arrays.stream(fileStatus)).asScala
+
+  // Split the file group to: parquet file without a matching log file, parquet file need to merge with log files
+  private val parquetWithoutLogPaths: List[String] = fileGroup.filter(p => p._2.size() == 0).keys.toList
+  private val fileWithLogMap: Map[String, String] = fileGroup.filter(p => p._2.size() > 0).map{ case(k, v) => (k, v.asScala.toList.mkString(","))}.toMap
+
+  if (log.isDebugEnabled) {
+    log.debug("All parquet files" + fileStatus.map(s => s.getPath.toString).mkString(","))
+    log.debug("ParquetWithoutLogPaths" + parquetWithoutLogPaths.mkString(","))
+    log.debug("ParquetWithLogPaths" + fileWithLogMap.map(m => s"${m._1}:${m._2}").mkString(","))
+  }
+
+  // Add log file map to options
+  private val finalOps = optParams ++ fileWithLogMap
+
+  // Load Hudi metadata
+  val metaClient = new HoodieTableMetaClient(conf, basePath, true)
+  private val hoodieTable = HoodieTable.create(metaClient, HoodieWriteConfig.newBuilder().withPath(basePath).build(), conf)
+
+  private val commitTimeline = hoodieTable.getMetaClient.getCommitsAndCompactionTimeline
+  if (commitTimeline.empty()) {
+    throw new HoodieException("No Valid Hudi timeline exists")
+  }
+  private val completedCommitTimeline = hoodieTable.getMetaClient.getCommitsTimeline.filterCompletedInstants()
+  private val lastInstant = completedCommitTimeline.lastInstant().get()
+  conf.setStrings("hoodie.realtime.last.commit", lastInstant.getTimestamp)
+
+  // 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)
+  }
+
+  override def schema: StructType = latestSchema
+
+  override def buildScan(): RDD[Row] = {
+    if (fileWithLogMap.isEmpty) {
+      sqlContext
+        .read
+        .options(finalOps)
+        .schema(schema)
+        .format("parquet")
+        .load(parquetWithoutLogPaths:_*)
+        .toDF()
+        .rdd

Review comment:
       remove `toDF()` worked as well. This will likely be optimized by Spark. Forgot why I put it here. Probably following the incremental relation implementation.




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



[GitHub] [hudi] garyli1019 commented on a change in pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
garyli1019 commented on a change in pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#discussion_r438265070



##########
File path: hudi-spark/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetRecordReaderIterator.scala
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.spark.sql.execution.datasources.parquet
+
+import org.apache.avro.Schema
+import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.HOODIE_RECORD_KEY_COL_POS
+import org.apache.hudi.realtime.HoodieRealtimeParquetRecordReader
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.avro.{AvroDeserializer, SchemaConverters}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, UnsafeRow}
+import org.apache.spark.sql.types.StructType
+import java.io.Closeable
+import java.util
+
+/**
+ * This class is the iterator for Hudi MOR table.
+ * This iterator will read the parquet file first and skip the record if it present in the log file.
+ * Then read the log file.
+ * Custom payload is not supported yet. This combining logic is matching with [OverwriteWithLatestAvroPayload]

Review comment:
       I am trying to break it into smaller tasks. I will definitely follow up with the custom payload support.
   This would require some type conversion. Parquet -> UnsafeRow -> Avro -> merge -> InternalRow -> UnsafeRow. This can come together with the type conversion optimization work.




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



[GitHub] [hudi] vinothchandar commented on a change in pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on a change in pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#discussion_r450165106



##########
File path: hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala
##########
@@ -58,26 +60,28 @@ class DefaultSource extends RelationProvider
       throw new HoodieException("'path' must be specified.")
     }
 
+    // Try to create hoodie table meta client from the give path
+    // TODO: Smarter path handling
+    val metaClient = try {
+      val conf = sqlContext.sparkContext.hadoopConfiguration
+      Option(new HoodieTableMetaClient(conf, path.get, true))

Review comment:
       would n't be problematic if `path` is a glob and not the actual basePath of the table? COW/Snapshot query can for e.g do this and I think we should handle the same for MOR as well.

##########
File path: hudi-spark/src/main/scala/org/apache/hudi/DataSourceOptions.scala
##########
@@ -65,7 +66,7 @@ object DataSourceReadOptions {
     * This eases migration from old configs to new configs.
     */
   def translateViewTypesToQueryTypes(optParams: Map[String, String]) : Map[String, String] = {
-    val translation = Map(VIEW_TYPE_READ_OPTIMIZED_OPT_VAL -> QUERY_TYPE_SNAPSHOT_OPT_VAL,
+    val translation = Map(VIEW_TYPE_READ_OPTIMIZED_OPT_VAL -> QUERY_TYPE_READ_OPTIMIZED_OPT_VAL,

Review comment:
       No.. there are no more views.. we did a renaming exercise to clear things up as "query types" .. with that there should be no confusion.. our docs are consistent with this as well..  On COW there is in fact no RO view.. so this change has to be done differently, if you need for MOR.. 

##########
File path: hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala
##########
@@ -58,26 +60,28 @@ class DefaultSource extends RelationProvider
       throw new HoodieException("'path' must be specified.")
     }
 
+    // Try to create hoodie table meta client from the give path
+    // TODO: Smarter path handling
+    val metaClient = try {
+      val conf = sqlContext.sparkContext.hadoopConfiguration
+      Option(new HoodieTableMetaClient(conf, path.get, true))
+    } catch {
+      case e: HoodieException => Option.empty

Review comment:
       can just error out there?

##########
File path: hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala
##########
@@ -123,4 +127,25 @@ class DefaultSource extends RelationProvider
   }
 
   override def shortName(): String = "hudi"
+
+  private def getReadOptimizedView(sqlContext: SQLContext,

Review comment:
       we can rename to something like `getFilteredBaseFileRelation()`. Again, don't want to bring back view nomenclature into the code. 




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



[GitHub] [hudi] vinothchandar commented on pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#issuecomment-646429952


   Just finished reviewing bootstrap pr which blocks udits pr :). Will read up and review both your and udits or shortly


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



[GitHub] [hudi] garyli1019 commented on a change in pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
garyli1019 commented on a change in pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#discussion_r438268212



##########
File path: hudi-spark/src/main/scala/org/apache/hudi/DataSourceOptions.scala
##########
@@ -48,7 +49,7 @@ object DataSourceReadOptions {
   val QUERY_TYPE_SNAPSHOT_OPT_VAL = "snapshot"
   val QUERY_TYPE_READ_OPTIMIZED_OPT_VAL = "read_optimized"
   val QUERY_TYPE_INCREMENTAL_OPT_VAL = "incremental"
-  val DEFAULT_QUERY_TYPE_OPT_VAL: String = QUERY_TYPE_SNAPSHOT_OPT_VAL
+  val DEFAULT_QUERY_TYPE_OPT_VAL: String = QUERY_TYPE_READ_OPTIMIZED_OPT_VAL

Review comment:
       will change back




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



[GitHub] [hudi] garyli1019 commented on a change in pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
garyli1019 commented on a change in pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#discussion_r445318518



##########
File path: hudi-spark/src/main/scala/org/apache/hudi/SnapshotRelation.scala
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.table.{HoodieTableMetaClient, TableSchemaResolver}
+import org.apache.hudi.common.table.timeline.HoodieTimeline
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.hadoop.{HoodieParquetInputFormat, HoodieROTablePathFilter}
+import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.table.HoodieTable
+
+import org.apache.hadoop.mapred.JobConf
+import org.apache.log4j.LogManager
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{DataFrame, Row, SQLContext}
+import org.apache.spark.sql.sources.{BaseRelation, Filter, PrunedFilteredScan}
+import org.apache.spark.sql.types.StructType
+
+import java.util
+import scala.collection.JavaConverters._
+
+/**
+ * This is the Spark DataSourceV1 relation to read Hudi MOR table.
+ * @param sqlContext
+ * @param basePath
+ * @param optParams
+ * @param userSchema
+ */
+class SnapshotRelation(val sqlContext: SQLContext,
+                       val basePath: String,
+                       val optParams: Map[String, String],
+                       val userSchema: StructType) extends BaseRelation with PrunedFilteredScan {
+
+  private val log = LogManager.getLogger(classOf[SnapshotRelation])
+  private val conf = sqlContext.sparkContext.hadoopConfiguration
+
+  // Load Hudi metadata
+  val metaClient = new HoodieTableMetaClient(conf, basePath, true)
+  private val hoodieTable = HoodieTable.create(metaClient, HoodieWriteConfig.newBuilder().withPath(basePath).build(), conf)
+  private val commitTimeline = hoodieTable.getMetaClient.getCommitsAndCompactionTimeline
+  if (commitTimeline.empty()) {
+    throw new HoodieException("No Valid Hudi timeline exists")
+  }
+  private val completedCommitTimeline = hoodieTable.getMetaClient.getCommitsTimeline.filterCompletedInstants()
+  private val lastInstant = completedCommitTimeline.lastInstant().get()
+
+  // Set config for listStatus() in HoodieParquetInputFormat
+  conf.setClass(
+    "mapreduce.input.pathFilter.class",
+    classOf[HoodieROTablePathFilter],
+    classOf[org.apache.hadoop.fs.PathFilter])
+  conf.setStrings("mapreduce.input.fileinputformat.inputdir", basePath)
+  conf.setStrings("mapreduce.input.fileinputformat.input.dir.recursive", "true")
+  conf.setStrings("hoodie.realtime.last.commit", lastInstant.getTimestamp)
+
+  private val hoodieInputFormat = new HoodieParquetInputFormat
+  hoodieInputFormat.setConf(conf)
+
+  // List all parquet files
+  private val fileStatus = hoodieInputFormat.listStatus(new JobConf(conf))
+
+  val (parquetPaths, parquetWithLogPaths) = if (lastInstant.getAction.equals(HoodieTimeline.COMMIT_ACTION)
+    || lastInstant.getAction.equals(HoodieTimeline.COMPACTION_ACTION)) {
+    (fileStatus.map(f => f.getPath.toString).toList, Map.empty[String, String])
+  } else {
+    val fileGroups = HoodieRealtimeInputFormatUtils.groupLogsByBaseFile(conf, util.Arrays.stream(fileStatus)).asScala
+    // Split the file group to: parquet file without a matching log file, parquet file need to merge with log files
+    val parquetPaths: List[String] = fileGroups.filter(p => p._2.size() == 0).keys.toList
+    val parquetWithLogPaths: Map[String, String] = fileGroups
+      .filter(p => p._2.size() > 0)
+      .map{ case(k, v) => (k, v.asScala.toList.mkString(","))}
+      .toMap
+    (parquetPaths, parquetWithLogPaths)
+  }
+
+  if (log.isDebugEnabled) {
+    log.debug("Stand alone parquet files: \n" + parquetPaths.mkString("\n"))
+    log.debug("Parquet files that have matching log files: \n" + parquetWithLogPaths.map(m => s"${m._1}:${m._2}").mkString("\n"))
+  }
+
+  // Add log file map to options
+  private val finalOps = optParams ++ parquetWithLogPaths
+
+  // 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)
+  }
+
+  override def schema: StructType = latestSchema
+
+  override def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = {

Review comment:
       Follow up: https://issues.apache.org/jira/browse/HUDI-1050




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



[GitHub] [hudi] vinothchandar commented on pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#issuecomment-658501011


   I am fine with doing that.. not sure if thats more work for @umehrot2  .. wdyt ?
   
   @bvaradar in general, can we get more of the bootstrap landed and work on the follow ups vs having these large PRs pending out there this long.. We kind of have a traffic gridlock here. 


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



[GitHub] [hudi] codecov-commenter edited a comment on pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#issuecomment-643095877


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/1722?src=pr&el=h1) Report
   > Merging [#1722](https://codecov.io/gh/apache/hudi/pull/1722?src=pr&el=desc) into [master](https://codecov.io/gh/apache/hudi/commit/37838cea6094ddc66191df42e8b2c84f132d1623&el=desc) will **decrease** coverage by `0.24%`.
   > The diff coverage is `0.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/1722/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/1722?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #1722      +/-   ##
   ============================================
   - Coverage     18.16%   17.91%   -0.25%     
   + Complexity      860      859       -1     
   ============================================
     Files           352      355       +3     
     Lines         15411    15613     +202     
     Branches       1525     1550      +25     
   ============================================
   - Hits           2799     2797       -2     
   - Misses        12254    12458     +204     
     Partials        358      358              
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/1722?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...main/scala/org/apache/hudi/DataSourceOptions.scala](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2h1ZGkvRGF0YVNvdXJjZU9wdGlvbnMuc2NhbGE=) | `68.13% <0.00%> (ø)` | `0.00 <0.00> (ø)` | |
   | [...src/main/scala/org/apache/hudi/DefaultSource.scala](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2h1ZGkvRGVmYXVsdFNvdXJjZS5zY2FsYQ==) | `0.00% <0.00%> (ø)` | `0.00 <0.00> (ø)` | |
   | [.../main/scala/org/apache/hudi/SnapshotRelation.scala](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2h1ZGkvU25hcHNob3RSZWxhdGlvbi5zY2FsYQ==) | `0.00% <0.00%> (ø)` | `0.00 <0.00> (?)` | |
   | [...es/parquet/HoodieParquetRecordReaderIterator.scala](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL3NwYXJrL3NxbC9leGVjdXRpb24vZGF0YXNvdXJjZXMvcGFycXVldC9Ib29kaWVQYXJxdWV0UmVjb3JkUmVhZGVySXRlcmF0b3Iuc2NhbGE=) | `0.00% <0.00%> (ø)` | `0.00 <0.00> (?)` | |
   | [...atasources/parquet/HoodieRealtimeInputFormat.scala](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL3NwYXJrL3NxbC9leGVjdXRpb24vZGF0YXNvdXJjZXMvcGFycXVldC9Ib29kaWVSZWFsdGltZUlucHV0Rm9ybWF0LnNjYWxh) | `0.00% <0.00%> (ø)` | `0.00 <0.00> (?)` | |
   | [...apache/hudi/common/fs/HoodieWrapperFileSystem.java](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL2ZzL0hvb2RpZVdyYXBwZXJGaWxlU3lzdGVtLmphdmE=) | `21.98% <0.00%> (-0.71%)` | `28.00% <0.00%> (-1.00%)` | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/hudi/pull/1722?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/hudi/pull/1722?src=pr&el=footer). Last update [37838ce...96f8c36](https://codecov.io/gh/apache/hudi/pull/1722?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


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



[GitHub] [hudi] bvaradar commented on pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
bvaradar commented on pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#issuecomment-658506263


   @garyli1019 @vinothchandar : Yes, I am planning to address the bootstrap PR comments and also give review comments for @umehrot2  changes by this weekend. @umehrot2 :  I know this is not ideal and apologies for not able to give a review quickly. 


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



[GitHub] [hudi] garyli1019 commented on a change in pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
garyli1019 commented on a change in pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#discussion_r438489438



##########
File path: hudi-spark/src/main/scala/org/apache/hudi/DataSourceOptions.scala
##########
@@ -48,7 +49,7 @@ object DataSourceReadOptions {
   val QUERY_TYPE_SNAPSHOT_OPT_VAL = "snapshot"
   val QUERY_TYPE_READ_OPTIMIZED_OPT_VAL = "read_optimized"
   val QUERY_TYPE_INCREMENTAL_OPT_VAL = "incremental"
-  val DEFAULT_QUERY_TYPE_OPT_VAL: String = QUERY_TYPE_SNAPSHOT_OPT_VAL
+  val DEFAULT_QUERY_TYPE_OPT_VAL: String = QUERY_TYPE_READ_OPTIMIZED_OPT_VAL

Review comment:
       This might give some surprise to the user if we keep the snapshot as default. 
   If the user don't change their code `spark.read.format("org.apache.hudi").load(path, "/*/*/*/*")` then they will get an error.




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



[GitHub] [hudi] vinothchandar commented on pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#issuecomment-658757520


   @garyli1019 actually @umehrot2 's approach there is wrapping the FileFormat as opposed to extending, which is great as well.. If you can wait, we can land the bootstrap changes and rework this on top. But, I feel what you will be doing here is a very different merge than @umehrot2 's PR and the code to wrap ParquetFileFormat is not that large..  So you can also in parallel just proceed? Do you just want the two util classes landed? SparkUtils/TablePathUtils?
   
   your call. lmk what you think. 
   
   
   


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



[GitHub] [hudi] vinothchandar commented on a change in pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on a change in pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#discussion_r444220031



##########
File path: hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala
##########
@@ -57,8 +57,7 @@ class DefaultSource extends RelationProvider
     if (path.isEmpty) {
       throw new HoodieException("'path' must be specified.")
     }
-
-    if (parameters(QUERY_TYPE_OPT_KEY).equals(QUERY_TYPE_SNAPSHOT_OPT_VAL)) {
+        if (parameters(QUERY_TYPE_OPT_KEY).equals(QUERY_TYPE_READ_OPTIMIZED_OPT_VAL)) {
       // this is just effectively RO view only, where `path` can contain a mix of

Review comment:
       Nit:indentation 
   
   This changes behavior for cow/snapshot. I feel we should have two separate methods - `createCopyOnWriteRelation` (No change in behavior) and `createMergeOnReadRelation` (changes for this poor)
   

##########
File path: hudi-spark/src/main/scala/org/apache/hudi/DataSourceOptions.scala
##########
@@ -65,7 +66,7 @@ object DataSourceReadOptions {
     * This eases migration from old configs to new configs.
     */
   def translateViewTypesToQueryTypes(optParams: Map[String, String]) : Map[String, String] = {
-    val translation = Map(VIEW_TYPE_READ_OPTIMIZED_OPT_VAL -> QUERY_TYPE_SNAPSHOT_OPT_VAL,
+    val translation = Map(VIEW_TYPE_READ_OPTIMIZED_OPT_VAL -> QUERY_TYPE_READ_OPTIMIZED_OPT_VAL,

Review comment:
       Don’t think this change is necessary, right? RO view does map to snapshot query for cow. We may need to have two maps for cow and mor

##########
File path: hudi-spark/src/main/scala/org/apache/hudi/SnapshotRelation.scala
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.table.{HoodieTableMetaClient, TableSchemaResolver}
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.hadoop.{HoodieParquetInputFormat, HoodieROTablePathFilter}
+import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.table.HoodieTable
+
+import org.apache.hadoop.mapred.JobConf
+import org.apache.log4j.LogManager
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{Row, SQLContext}
+import org.apache.spark.sql.sources.{BaseRelation, TableScan}
+import org.apache.spark.sql.types.StructType
+
+import java.util
+import scala.collection.JavaConverters._
+
+/**
+ * This is the Spark DataSourceV1 relation to read Hudi MOR table.
+ * @param sqlContext
+ * @param basePath
+ * @param optParams
+ * @param userSchema
+ */
+class SnapshotRelation(val sqlContext: SQLContext,
+                       val basePath: String,
+                       val optParams: Map[String, String],
+                       val userSchema: StructType) extends BaseRelation with TableScan {
+
+  private val log = LogManager.getLogger(classOf[SnapshotRelation])
+  private val conf = sqlContext.sparkContext.hadoopConfiguration
+
+  // Set config for listStatus() in HoodieParquetInputFormat
+  // TODO(garyli): Switch to bootstrap file listing methods
+  conf.setClass(
+    "mapreduce.input.pathFilter.class",
+    classOf[HoodieROTablePathFilter],
+    classOf[org.apache.hadoop.fs.PathFilter])
+  conf.setStrings("mapreduce.input.fileinputformat.inputdir", basePath)
+  conf.setStrings("mapreduce.input.fileinputformat.input.dir.recursive", "true")
+
+  private val HoodieInputFormat = new HoodieParquetInputFormat
+  HoodieInputFormat.setConf(conf)
+  private val fileStatus = HoodieInputFormat.listStatus(new JobConf(conf))
+  private val fileGroup = HoodieRealtimeInputFormatUtils.groupLogsByBaseFile(conf, util.Arrays.stream(fileStatus)).asScala
+
+  // Split the file group to: parquet file without a matching log file, parquet file need to merge with log files
+  private val parquetWithoutLogPaths: List[String] = fileGroup.filter(p => p._2.size() == 0).keys.toList
+  private val fileWithLogMap: Map[String, String] = fileGroup.filter(p => p._2.size() > 0).map{ case(k, v) => (k, v.asScala.toList.mkString(","))}.toMap

Review comment:
       Consistent naming wrt l69

##########
File path: hudi-spark/src/main/scala/org/apache/hudi/SnapshotRelation.scala
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.table.{HoodieTableMetaClient, TableSchemaResolver}
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.hadoop.{HoodieParquetInputFormat, HoodieROTablePathFilter}
+import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.table.HoodieTable
+
+import org.apache.hadoop.mapred.JobConf
+import org.apache.log4j.LogManager
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{Row, SQLContext}
+import org.apache.spark.sql.sources.{BaseRelation, TableScan}
+import org.apache.spark.sql.types.StructType
+
+import java.util
+import scala.collection.JavaConverters._
+
+/**
+ * This is the Spark DataSourceV1 relation to read Hudi MOR table.
+ * @param sqlContext
+ * @param basePath
+ * @param optParams
+ * @param userSchema
+ */
+class SnapshotRelation(val sqlContext: SQLContext,
+                       val basePath: String,
+                       val optParams: Map[String, String],
+                       val userSchema: StructType) extends BaseRelation with TableScan {
+
+  private val log = LogManager.getLogger(classOf[SnapshotRelation])
+  private val conf = sqlContext.sparkContext.hadoopConfiguration
+
+  // Set config for listStatus() in HoodieParquetInputFormat
+  // TODO(garyli): Switch to bootstrap file listing methods
+  conf.setClass(
+    "mapreduce.input.pathFilter.class",
+    classOf[HoodieROTablePathFilter],
+    classOf[org.apache.hadoop.fs.PathFilter])
+  conf.setStrings("mapreduce.input.fileinputformat.inputdir", basePath)
+  conf.setStrings("mapreduce.input.fileinputformat.input.dir.recursive", "true")
+
+  private val HoodieInputFormat = new HoodieParquetInputFormat
+  HoodieInputFormat.setConf(conf)
+  private val fileStatus = HoodieInputFormat.listStatus(new JobConf(conf))
+  private val fileGroup = HoodieRealtimeInputFormatUtils.groupLogsByBaseFile(conf, util.Arrays.stream(fileStatus)).asScala
+
+  // Split the file group to: parquet file without a matching log file, parquet file need to merge with log files
+  private val parquetWithoutLogPaths: List[String] = fileGroup.filter(p => p._2.size() == 0).keys.toList
+  private val fileWithLogMap: Map[String, String] = fileGroup.filter(p => p._2.size() > 0).map{ case(k, v) => (k, v.asScala.toList.mkString(","))}.toMap
+
+  if (log.isDebugEnabled) {
+    log.debug("All parquet files" + fileStatus.map(s => s.getPath.toString).mkString(","))
+    log.debug("ParquetWithoutLogPaths" + parquetWithoutLogPaths.mkString(","))

Review comment:
       Can we have variable names and comments more generically talking about base and log files

##########
File path: hudi-spark/src/main/scala/org/apache/hudi/DataSourceOptions.scala
##########
@@ -48,7 +49,7 @@ object DataSourceReadOptions {
   val QUERY_TYPE_SNAPSHOT_OPT_VAL = "snapshot"
   val QUERY_TYPE_READ_OPTIMIZED_OPT_VAL = "read_optimized"
   val QUERY_TYPE_INCREMENTAL_OPT_VAL = "incremental"
-  val DEFAULT_QUERY_TYPE_OPT_VAL: String = QUERY_TYPE_SNAPSHOT_OPT_VAL
+  val DEFAULT_QUERY_TYPE_OPT_VAL: String = QUERY_TYPE_READ_OPTIMIZED_OPT_VAL

Review comment:
       For cow, snapshot is the default anyway.
   Why would globing like that error out? 

##########
File path: hudi-spark/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetRecordReaderIterator.scala
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.spark.sql.execution.datasources.parquet
+
+import org.apache.hadoop.mapred.JobConf
+import org.apache.hudi.common.fs.FSUtils
+import org.apache.hudi.common.table.log.{HoodieMergedLogRecordScanner, LogReaderUtils}
+import org.apache.hudi.hadoop.config.HoodieRealtimeConfig
+import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit
+import org.apache.parquet.hadoop.ParquetRecordReader
+
+import org.apache.avro.Schema
+import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.HOODIE_RECORD_KEY_COL_POS
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.avro.{AvroDeserializer, SchemaConverters}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, UnsafeRow}
+import org.apache.spark.sql.types.StructType
+import java.io.Closeable
+import java.util
+
+import scala.util.Try
+
+/**
+ * This class is the iterator for Hudi MOR table.
+ * Log files are scanned on initialization.
+ * This iterator will read the parquet file first and skip the record if it present in the log file.
+ * Then read the log file.
+ * Custom payload is not supported yet. This combining logic is matching with [OverwriteWithLatestAvroPayload]
+ * @param rowReader HoodieRealtimeParquetRecordReader
+ */
+class HoodieParquetRecordReaderIterator(private[this] var rowReader: ParquetRecordReader[UnsafeRow],
+                                        private[this] val split: HoodieRealtimeFileSplit,
+                                        private[this] val jobConf: JobConf) extends Iterator[UnsafeRow] with Closeable with Logging {
+  private[this] var havePair = false
+  private[this] var finished = false
+  private[this] var parquetFinished = false
+
+  private[this] var deltaRecordMap: util.Map[String,org.apache.hudi.common.model
+  .HoodieRecord[_ <: org.apache.hudi.common.model.HoodieRecordPayload[_ <: org.apache.hudi.common.model.HoodieRecordPayload[_ <: AnyRef]]]] = _

Review comment:
       why the fully qualified class names ?

##########
File path: hudi-spark/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieRealtimeInputFormat.scala
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.spark.sql.execution.datasources.parquet
+
+import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapred.{FileSplit, JobConf}
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
+import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType}
+import org.apache.parquet.filter2.compat.FilterCompat
+import org.apache.parquet.filter2.predicate.FilterApi
+import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS
+import org.apache.parquet.hadoop.{ParquetFileReader, ParquetInputFormat, ParquetRecordReader}
+import org.apache.spark.TaskContext
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
+import org.apache.spark.sql.catalyst.expressions.{JoinedRow, UnsafeRow}
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+import java.net.URI
+import scala.collection.JavaConverters._
+
+/**
+ * This class is an extension of ParquetFileFormat from Spark SQL.
+ * The file split, record reader, record reader iterator are customized to read Hudi MOR table.
+ */
+class HoodieRealtimeInputFormat extends ParquetFileFormat {

Review comment:
       @garyli1019 true.. but if the logs are small w.r.t parquet base, this might be okay actually.. For workloads with high write amplification, people tend to have much smaller file sizes anyway.. We should support the row group level parallel splitting for sure, otherwise, performance would be an issue.. btw Hive achieves this today.. We wrap each file split into a realtime file split.. 

##########
File path: hudi-spark/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieRealtimeInputFormat.scala
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.spark.sql.execution.datasources.parquet
+
+import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapred.{FileSplit, JobConf}
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
+import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType}
+import org.apache.parquet.filter2.compat.FilterCompat
+import org.apache.parquet.filter2.predicate.FilterApi
+import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS
+import org.apache.parquet.hadoop.{ParquetFileReader, ParquetInputFormat, ParquetRecordReader}
+import org.apache.spark.TaskContext
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
+import org.apache.spark.sql.catalyst.expressions.{JoinedRow, UnsafeRow}
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+import java.net.URI
+import scala.collection.JavaConverters._
+
+/**
+ * This class is an extension of ParquetFileFormat from Spark SQL.
+ * The file split, record reader, record reader iterator are customized to read Hudi MOR table.
+ */
+class HoodieRealtimeInputFormat extends ParquetFileFormat {
+  //TODO: Better usage of this short name.
+  override def shortName(): String = "hudi.snapshot"

Review comment:
       let's get rid of this for now?  we don't really plan to expose this to the users right?

##########
File path: hudi-spark/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieRealtimeInputFormat.scala
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.spark.sql.execution.datasources.parquet
+
+import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapred.{FileSplit, JobConf}
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
+import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType}
+import org.apache.parquet.filter2.compat.FilterCompat
+import org.apache.parquet.filter2.predicate.FilterApi
+import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS
+import org.apache.parquet.hadoop.{ParquetFileReader, ParquetInputFormat, ParquetRecordReader}
+import org.apache.spark.TaskContext
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
+import org.apache.spark.sql.catalyst.expressions.{JoinedRow, UnsafeRow}
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+import java.net.URI
+import scala.collection.JavaConverters._
+
+/**
+ * This class is an extension of ParquetFileFormat from Spark SQL.
+ * The file split, record reader, record reader iterator are customized to read Hudi MOR table.
+ */
+class HoodieRealtimeInputFormat extends ParquetFileFormat {
+  //TODO: Better usage of this short name.
+  override def shortName(): String = "hudi.snapshot"
+  override def toString(): String = "hudi.snapshot"
+
+  override def buildReaderWithPartitionValues(sparkSession: SparkSession,
+                                              dataSchema: StructType,
+                                              partitionSchema: StructType,
+                                              requiredSchema: StructType,
+                                              filters: Seq[Filter], options: Map[String, String],
+                                              hadoopConf: Configuration): (PartitionedFile) =>
+    Iterator[InternalRow] = {
+    hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName)
+    hadoopConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, requiredSchema.json)
+    hadoopConf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, requiredSchema.json)
+    hadoopConf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key,
+      sparkSession.sessionState.conf.sessionLocalTimeZone)
+    hadoopConf.setBoolean(SQLConf.CASE_SENSITIVE.key,
+      sparkSession.sessionState.conf.caseSensitiveAnalysis)
+
+    ParquetWriteSupport.setSchema(requiredSchema, hadoopConf)
+
+    // Sets flags for `ParquetToSparkSchemaConverter`
+    hadoopConf.setBoolean(
+      SQLConf.PARQUET_BINARY_AS_STRING.key,
+      sparkSession.sessionState.conf.isParquetBinaryAsString)
+    hadoopConf.setBoolean(
+      SQLConf.PARQUET_INT96_AS_TIMESTAMP.key,
+      sparkSession.sessionState.conf.isParquetINT96AsTimestamp)
+
+    val broadcastedHadoopConf =
+      sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf))
+
+    // TODO: if you move this into the closure it reverts to the default values.
+    // If true, enable using the custom RecordReader for parquet. This only works for
+    // a subset of the types (no complex types).
+    //val resultSchema = StructType(partitionSchema.fields ++ requiredSchema.fields)
+    val sqlConf = sparkSession.sessionState.conf
+    val enableRecordFilter: Boolean = sqlConf.parquetRecordFilterEnabled
+    val timestampConversion: Boolean = sqlConf.isParquetINT96TimestampConversion
+    val enableParquetFilterPushDown: Boolean = sqlConf.parquetFilterPushDown
+    // Whole stage codegen (PhysicalRDD) is able to deal with batches directly
+    //val returningBatch = supportBatch(sparkSession, resultSchema)
+    val pushDownDate = sqlConf.parquetFilterPushDownDate
+    val pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp
+    val pushDownDecimal = sqlConf.parquetFilterPushDownDecimal
+    val pushDownStringStartWith = sqlConf.parquetFilterPushDownStringStartWith
+    val pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold
+    val isCaseSensitive = sqlConf.caseSensitiveAnalysis
+
+    (file: PartitionedFile) => {
+      assert(file.partitionValues.numFields == partitionSchema.size)
+
+      val sharedConf = broadcastedHadoopConf.value.value
+      val fileSplit =
+        new FileSplit(new Path(new URI(file.filePath)), file.start, file.length, new Array[String](0))

Review comment:
       this is a single parquet file?

##########
File path: hudi-spark/src/main/scala/org/apache/hudi/SnapshotRelation.scala
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.table.{HoodieTableMetaClient, TableSchemaResolver}
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.hadoop.{HoodieParquetInputFormat, HoodieROTablePathFilter}
+import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.table.HoodieTable
+
+import org.apache.hadoop.mapred.JobConf
+import org.apache.log4j.LogManager
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{Row, SQLContext}
+import org.apache.spark.sql.sources.{BaseRelation, TableScan}
+import org.apache.spark.sql.types.StructType
+
+import java.util
+import scala.collection.JavaConverters._
+
+/**
+ * This is the Spark DataSourceV1 relation to read Hudi MOR table.

Review comment:
       Actually this is what we use for both cow and mor.. see comment above. Would be best to keep behavior for cow same

##########
File path: hudi-spark/src/main/scala/org/apache/hudi/SnapshotRelation.scala
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.table.{HoodieTableMetaClient, TableSchemaResolver}
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.hadoop.{HoodieParquetInputFormat, HoodieROTablePathFilter}
+import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.table.HoodieTable
+
+import org.apache.hadoop.mapred.JobConf
+import org.apache.log4j.LogManager
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{Row, SQLContext}
+import org.apache.spark.sql.sources.{BaseRelation, TableScan}
+import org.apache.spark.sql.types.StructType
+
+import java.util
+import scala.collection.JavaConverters._
+
+/**
+ * This is the Spark DataSourceV1 relation to read Hudi MOR table.
+ * @param sqlContext
+ * @param basePath
+ * @param optParams
+ * @param userSchema
+ */
+class SnapshotRelation(val sqlContext: SQLContext,
+                       val basePath: String,
+                       val optParams: Map[String, String],
+                       val userSchema: StructType) extends BaseRelation with TableScan {
+
+  private val log = LogManager.getLogger(classOf[SnapshotRelation])
+  private val conf = sqlContext.sparkContext.hadoopConfiguration
+
+  // Set config for listStatus() in HoodieParquetInputFormat
+  // TODO(garyli): Switch to bootstrap file listing methods

Review comment:
       Assuming this pr will land first. Let’s remove todo and file a follow on jira ?

##########
File path: hudi-spark/src/main/scala/org/apache/hudi/SnapshotRelation.scala
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.table.{HoodieTableMetaClient, TableSchemaResolver}
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.hadoop.{HoodieParquetInputFormat, HoodieROTablePathFilter}
+import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.table.HoodieTable
+
+import org.apache.hadoop.mapred.JobConf
+import org.apache.log4j.LogManager
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{Row, SQLContext}
+import org.apache.spark.sql.sources.{BaseRelation, TableScan}
+import org.apache.spark.sql.types.StructType
+
+import java.util
+import scala.collection.JavaConverters._
+
+/**
+ * This is the Spark DataSourceV1 relation to read Hudi MOR table.
+ * @param sqlContext
+ * @param basePath
+ * @param optParams
+ * @param userSchema
+ */
+class SnapshotRelation(val sqlContext: SQLContext,
+                       val basePath: String,
+                       val optParams: Map[String, String],
+                       val userSchema: StructType) extends BaseRelation with TableScan {
+
+  private val log = LogManager.getLogger(classOf[SnapshotRelation])
+  private val conf = sqlContext.sparkContext.hadoopConfiguration
+
+  // Set config for listStatus() in HoodieParquetInputFormat
+  // TODO(garyli): Switch to bootstrap file listing methods
+  conf.setClass(
+    "mapreduce.input.pathFilter.class",
+    classOf[HoodieROTablePathFilter],
+    classOf[org.apache.hadoop.fs.PathFilter])
+  conf.setStrings("mapreduce.input.fileinputformat.inputdir", basePath)
+  conf.setStrings("mapreduce.input.fileinputformat.input.dir.recursive", "true")
+
+  private val HoodieInputFormat = new HoodieParquetInputFormat
+  HoodieInputFormat.setConf(conf)
+  private val fileStatus = HoodieInputFormat.listStatus(new JobConf(conf))
+  private val fileGroup = HoodieRealtimeInputFormatUtils.groupLogsByBaseFile(conf, util.Arrays.stream(fileStatus)).asScala

Review comment:
       Rename:fileGroups

##########
File path: hudi-spark/src/main/scala/org/apache/hudi/SnapshotRelation.scala
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.table.{HoodieTableMetaClient, TableSchemaResolver}
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.hadoop.{HoodieParquetInputFormat, HoodieROTablePathFilter}
+import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.table.HoodieTable
+
+import org.apache.hadoop.mapred.JobConf
+import org.apache.log4j.LogManager
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{Row, SQLContext}
+import org.apache.spark.sql.sources.{BaseRelation, TableScan}
+import org.apache.spark.sql.types.StructType
+
+import java.util
+import scala.collection.JavaConverters._
+
+/**
+ * This is the Spark DataSourceV1 relation to read Hudi MOR table.
+ * @param sqlContext
+ * @param basePath
+ * @param optParams
+ * @param userSchema
+ */
+class SnapshotRelation(val sqlContext: SQLContext,
+                       val basePath: String,
+                       val optParams: Map[String, String],
+                       val userSchema: StructType) extends BaseRelation with TableScan {
+
+  private val log = LogManager.getLogger(classOf[SnapshotRelation])
+  private val conf = sqlContext.sparkContext.hadoopConfiguration
+
+  // Set config for listStatus() in HoodieParquetInputFormat
+  // TODO(garyli): Switch to bootstrap file listing methods
+  conf.setClass(
+    "mapreduce.input.pathFilter.class",
+    classOf[HoodieROTablePathFilter],
+    classOf[org.apache.hadoop.fs.PathFilter])
+  conf.setStrings("mapreduce.input.fileinputformat.inputdir", basePath)
+  conf.setStrings("mapreduce.input.fileinputformat.input.dir.recursive", "true")
+
+  private val HoodieInputFormat = new HoodieParquetInputFormat

Review comment:
       CamelCase Naming ?

##########
File path: hudi-spark/src/main/scala/org/apache/hudi/SnapshotRelation.scala
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.table.{HoodieTableMetaClient, TableSchemaResolver}
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.hadoop.{HoodieParquetInputFormat, HoodieROTablePathFilter}
+import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.table.HoodieTable
+
+import org.apache.hadoop.mapred.JobConf
+import org.apache.log4j.LogManager
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{Row, SQLContext}
+import org.apache.spark.sql.sources.{BaseRelation, TableScan}
+import org.apache.spark.sql.types.StructType
+
+import java.util
+import scala.collection.JavaConverters._
+
+/**
+ * This is the Spark DataSourceV1 relation to read Hudi MOR table.
+ * @param sqlContext
+ * @param basePath
+ * @param optParams
+ * @param userSchema
+ */
+class SnapshotRelation(val sqlContext: SQLContext,
+                       val basePath: String,
+                       val optParams: Map[String, String],
+                       val userSchema: StructType) extends BaseRelation with TableScan {
+
+  private val log = LogManager.getLogger(classOf[SnapshotRelation])
+  private val conf = sqlContext.sparkContext.hadoopConfiguration
+
+  // Set config for listStatus() in HoodieParquetInputFormat
+  // TODO(garyli): Switch to bootstrap file listing methods
+  conf.setClass(
+    "mapreduce.input.pathFilter.class",
+    classOf[HoodieROTablePathFilter],
+    classOf[org.apache.hadoop.fs.PathFilter])
+  conf.setStrings("mapreduce.input.fileinputformat.inputdir", basePath)
+  conf.setStrings("mapreduce.input.fileinputformat.input.dir.recursive", "true")
+
+  private val HoodieInputFormat = new HoodieParquetInputFormat
+  HoodieInputFormat.setConf(conf)
+  private val fileStatus = HoodieInputFormat.listStatus(new JobConf(conf))
+  private val fileGroup = HoodieRealtimeInputFormatUtils.groupLogsByBaseFile(conf, util.Arrays.stream(fileStatus)).asScala
+
+  // Split the file group to: parquet file without a matching log file, parquet file need to merge with log files
+  private val parquetWithoutLogPaths: List[String] = fileGroup.filter(p => p._2.size() == 0).keys.toList
+  private val fileWithLogMap: Map[String, String] = fileGroup.filter(p => p._2.size() > 0).map{ case(k, v) => (k, v.asScala.toList.mkString(","))}.toMap
+
+  if (log.isDebugEnabled) {
+    log.debug("All parquet files" + fileStatus.map(s => s.getPath.toString).mkString(","))
+    log.debug("ParquetWithoutLogPaths" + parquetWithoutLogPaths.mkString(","))
+    log.debug("ParquetWithLogPaths" + fileWithLogMap.map(m => s"${m._1}:${m._2}").mkString(","))
+  }
+
+  // Add log file map to options
+  private val finalOps = optParams ++ fileWithLogMap
+
+  // Load Hudi metadata
+  val metaClient = new HoodieTableMetaClient(conf, basePath, true)
+  private val hoodieTable = HoodieTable.create(metaClient, HoodieWriteConfig.newBuilder().withPath(basePath).build(), conf)
+
+  private val commitTimeline = hoodieTable.getMetaClient.getCommitsAndCompactionTimeline
+  if (commitTimeline.empty()) {

Review comment:
       Should these checks happen before we list above? Also should we return an empty data set opposed to erroring others are no commits

##########
File path: hudi-spark/src/main/scala/org/apache/hudi/SnapshotRelation.scala
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.table.{HoodieTableMetaClient, TableSchemaResolver}
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.hadoop.{HoodieParquetInputFormat, HoodieROTablePathFilter}
+import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.table.HoodieTable
+
+import org.apache.hadoop.mapred.JobConf
+import org.apache.log4j.LogManager
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{Row, SQLContext}
+import org.apache.spark.sql.sources.{BaseRelation, TableScan}
+import org.apache.spark.sql.types.StructType
+
+import java.util
+import scala.collection.JavaConverters._
+
+/**
+ * This is the Spark DataSourceV1 relation to read Hudi MOR table.
+ * @param sqlContext
+ * @param basePath
+ * @param optParams
+ * @param userSchema
+ */
+class SnapshotRelation(val sqlContext: SQLContext,
+                       val basePath: String,
+                       val optParams: Map[String, String],
+                       val userSchema: StructType) extends BaseRelation with TableScan {
+
+  private val log = LogManager.getLogger(classOf[SnapshotRelation])
+  private val conf = sqlContext.sparkContext.hadoopConfiguration
+
+  // Set config for listStatus() in HoodieParquetInputFormat
+  conf.setClass(
+    "mapreduce.input.pathFilter.class",
+    classOf[HoodieROTablePathFilter],
+    classOf[org.apache.hadoop.fs.PathFilter])
+  conf.setStrings("mapreduce.input.fileinputformat.inputdir", basePath)
+  conf.setStrings("mapreduce.input.fileinputformat.input.dir.recursive", "true")
+
+  private val HoodieInputFormat = new HoodieParquetInputFormat
+  HoodieInputFormat.setConf(conf)
+  private val fileStatus = HoodieInputFormat.listStatus(new JobConf(conf))
+  private val fileGroup = HoodieRealtimeInputFormatUtils.groupLogsByBaseFile(conf, util.Arrays.stream(fileStatus)).asScala
+
+  // Split the file group to: parquet file without a matching log file, parquet file need to merge with log files
+  private val parquetWithoutLogPaths: List[String] = fileGroup.filter(p => p._2.size() == 0).keys.toList
+  private val fileWithLogMap: Map[String, String] = fileGroup.filter(p => p._2.size() > 0).map{ case(k, v) => (k, v.asScala.toList.mkString(","))}.toMap
+
+  if (log.isDebugEnabled) {
+    log.debug("All parquet files" + fileStatus.map(s => s.getPath.toString).mkString(","))
+    log.debug("ParquetWithoutLogPaths" + parquetWithoutLogPaths.mkString(","))
+    log.debug("ParquetWithLogPaths" + fileWithLogMap.map(m => s"${m._1}:${m._2}").mkString(","))
+  }
+
+  // Add log file map to options
+  private val finalOps = optParams ++ fileWithLogMap

Review comment:
       We should refrain from listing other executors. @umehrot2  do you have any suggestions here? 
   FWIW the amount of data would be proportional to the amount of partitions globbed which should not be as bad?

##########
File path: hudi-spark/src/main/scala/org/apache/hudi/SnapshotRelation.scala
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.table.{HoodieTableMetaClient, TableSchemaResolver}
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.hadoop.{HoodieParquetInputFormat, HoodieROTablePathFilter}
+import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.table.HoodieTable
+
+import org.apache.hadoop.mapred.JobConf
+import org.apache.log4j.LogManager
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{Row, SQLContext}
+import org.apache.spark.sql.sources.{BaseRelation, TableScan}
+import org.apache.spark.sql.types.StructType
+
+import java.util
+import scala.collection.JavaConverters._
+
+/**
+ * This is the Spark DataSourceV1 relation to read Hudi MOR table.
+ * @param sqlContext
+ * @param basePath
+ * @param optParams
+ * @param userSchema
+ */
+class SnapshotRelation(val sqlContext: SQLContext,
+                       val basePath: String,
+                       val optParams: Map[String, String],
+                       val userSchema: StructType) extends BaseRelation with TableScan {
+
+  private val log = LogManager.getLogger(classOf[SnapshotRelation])
+  private val conf = sqlContext.sparkContext.hadoopConfiguration
+
+  // Set config for listStatus() in HoodieParquetInputFormat
+  // TODO(garyli): Switch to bootstrap file listing methods
+  conf.setClass(
+    "mapreduce.input.pathFilter.class",
+    classOf[HoodieROTablePathFilter],
+    classOf[org.apache.hadoop.fs.PathFilter])
+  conf.setStrings("mapreduce.input.fileinputformat.inputdir", basePath)
+  conf.setStrings("mapreduce.input.fileinputformat.input.dir.recursive", "true")
+
+  private val HoodieInputFormat = new HoodieParquetInputFormat
+  HoodieInputFormat.setConf(conf)
+  private val fileStatus = HoodieInputFormat.listStatus(new JobConf(conf))
+  private val fileGroup = HoodieRealtimeInputFormatUtils.groupLogsByBaseFile(conf, util.Arrays.stream(fileStatus)).asScala
+
+  // Split the file group to: parquet file without a matching log file, parquet file need to merge with log files
+  private val parquetWithoutLogPaths: List[String] = fileGroup.filter(p => p._2.size() == 0).keys.toList
+  private val fileWithLogMap: Map[String, String] = fileGroup.filter(p => p._2.size() > 0).map{ case(k, v) => (k, v.asScala.toList.mkString(","))}.toMap
+
+  if (log.isDebugEnabled) {
+    log.debug("All parquet files" + fileStatus.map(s => s.getPath.toString).mkString(","))
+    log.debug("ParquetWithoutLogPaths" + parquetWithoutLogPaths.mkString(","))
+    log.debug("ParquetWithLogPaths" + fileWithLogMap.map(m => s"${m._1}:${m._2}").mkString(","))
+  }
+
+  // Add log file map to options
+  private val finalOps = optParams ++ fileWithLogMap
+
+  // Load Hudi metadata
+  val metaClient = new HoodieTableMetaClient(conf, basePath, true)
+  private val hoodieTable = HoodieTable.create(metaClient, HoodieWriteConfig.newBuilder().withPath(basePath).build(), conf)
+
+  private val commitTimeline = hoodieTable.getMetaClient.getCommitsAndCompactionTimeline
+  if (commitTimeline.empty()) {
+    throw new HoodieException("No Valid Hudi timeline exists")
+  }
+  private val completedCommitTimeline = hoodieTable.getMetaClient.getCommitsTimeline.filterCompletedInstants()
+  private val lastInstant = completedCommitTimeline.lastInstant().get()
+  conf.setStrings("hoodie.realtime.last.commit", lastInstant.getTimestamp)
+
+  // 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)
+  }
+
+  override def schema: StructType = latestSchema
+
+  override def buildScan(): RDD[Row] = {
+    if (fileWithLogMap.isEmpty) {
+      sqlContext
+        .read
+        .options(finalOps)
+        .schema(schema)
+        .format("parquet")
+        .load(parquetWithoutLogPaths:_*)
+        .toDF()
+        .rdd

Review comment:
       Need to understand this better.. do you see toDF().rdd causing additional overhead, compared to regular spark.read.parquet()

##########
File path: hudi-spark/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieRealtimeInputFormat.scala
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.spark.sql.execution.datasources.parquet
+
+import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapred.{FileSplit, JobConf}
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
+import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType}
+import org.apache.parquet.filter2.compat.FilterCompat
+import org.apache.parquet.filter2.predicate.FilterApi
+import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS
+import org.apache.parquet.hadoop.{ParquetFileReader, ParquetInputFormat, ParquetRecordReader}
+import org.apache.spark.TaskContext
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
+import org.apache.spark.sql.catalyst.expressions.{JoinedRow, UnsafeRow}
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+import java.net.URI
+import scala.collection.JavaConverters._
+
+/**
+ * This class is an extension of ParquetFileFormat from Spark SQL.
+ * The file split, record reader, record reader iterator are customized to read Hudi MOR table.
+ */
+class HoodieRealtimeInputFormat extends ParquetFileFormat {

Review comment:
       Rename: HoodieParquetRealtimeFileFormat denoting this is specifically handling parquet base files

##########
File path: hudi-spark/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetRecordReaderIterator.scala
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.spark.sql.execution.datasources.parquet
+
+import org.apache.avro.Schema
+import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.HOODIE_RECORD_KEY_COL_POS
+import org.apache.hudi.realtime.HoodieRealtimeParquetRecordReader
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.avro.{AvroDeserializer, SchemaConverters}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, UnsafeRow}
+import org.apache.spark.sql.types.StructType
+import java.io.Closeable
+import java.util
+
+/**
+ * This class is the iterator for Hudi MOR table.
+ * This iterator will read the parquet file first and skip the record if it present in the log file.
+ * Then read the log file.
+ * Custom payload is not supported yet. This combining logic is matching with [OverwriteWithLatestAvroPayload]

Review comment:
       sg. We can also add new methods to Payload interface to avoid the conversion.. and merges rows directly. Let’s file a follow on

##########
File path: hudi-spark/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetRecordReaderIterator.scala
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.spark.sql.execution.datasources.parquet
+
+import org.apache.hadoop.mapred.JobConf
+import org.apache.hudi.common.fs.FSUtils
+import org.apache.hudi.common.table.log.{HoodieMergedLogRecordScanner, LogReaderUtils}
+import org.apache.hudi.hadoop.config.HoodieRealtimeConfig
+import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit
+import org.apache.parquet.hadoop.ParquetRecordReader
+
+import org.apache.avro.Schema
+import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.HOODIE_RECORD_KEY_COL_POS
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.avro.{AvroDeserializer, SchemaConverters}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, UnsafeRow}
+import org.apache.spark.sql.types.StructType
+import java.io.Closeable
+import java.util
+
+import scala.util.Try
+
+/**
+ * This class is the iterator for Hudi MOR table.
+ * Log files are scanned on initialization.
+ * This iterator will read the parquet file first and skip the record if it present in the log file.
+ * Then read the log file.
+ * Custom payload is not supported yet. This combining logic is matching with [OverwriteWithLatestAvroPayload]
+ * @param rowReader HoodieRealtimeParquetRecordReader
+ */
+class HoodieParquetRecordReaderIterator(private[this] var rowReader: ParquetRecordReader[UnsafeRow],
+                                        private[this] val split: HoodieRealtimeFileSplit,
+                                        private[this] val jobConf: JobConf) extends Iterator[UnsafeRow] with Closeable with Logging {
+  private[this] var havePair = false
+  private[this] var finished = false
+  private[this] var parquetFinished = false
+
+  private[this] var deltaRecordMap: util.Map[String,org.apache.hudi.common.model
+  .HoodieRecord[_ <: org.apache.hudi.common.model.HoodieRecordPayload[_ <: org.apache.hudi.common.model.HoodieRecordPayload[_ <: AnyRef]]]] = _
+  private[this] var deltaRecordKeys: util.Set[String] = _
+  private[this] var deltaIter: util.Iterator[String] = _
+  private[this] var avroSchema: Schema = _
+  private[this] var sparkTypes: StructType = _
+  private[this] var converter: AvroDeserializer = _
+
+  // SPARK-23457 Register a task completion lister before `initialization`.

Review comment:
       does this comment make sense in this context?

##########
File path: hudi-spark/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetRecordReaderIterator.scala
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.spark.sql.execution.datasources.parquet

Review comment:
       why does this have to be under this package? so we can subclass `ParquetFileFormat`? 

##########
File path: hudi-spark/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetRecordReaderIterator.scala
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.spark.sql.execution.datasources.parquet
+
+import org.apache.hadoop.mapred.JobConf
+import org.apache.hudi.common.fs.FSUtils
+import org.apache.hudi.common.table.log.{HoodieMergedLogRecordScanner, LogReaderUtils}
+import org.apache.hudi.hadoop.config.HoodieRealtimeConfig
+import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit
+import org.apache.parquet.hadoop.ParquetRecordReader
+
+import org.apache.avro.Schema
+import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.HOODIE_RECORD_KEY_COL_POS
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.avro.{AvroDeserializer, SchemaConverters}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, UnsafeRow}
+import org.apache.spark.sql.types.StructType
+import java.io.Closeable
+import java.util
+
+import scala.util.Try
+
+/**
+ * This class is the iterator for Hudi MOR table.
+ * Log files are scanned on initialization.
+ * This iterator will read the parquet file first and skip the record if it present in the log file.
+ * Then read the log file.
+ * Custom payload is not supported yet. This combining logic is matching with [OverwriteWithLatestAvroPayload]
+ * @param rowReader HoodieRealtimeParquetRecordReader
+ */
+class HoodieParquetRecordReaderIterator(private[this] var rowReader: ParquetRecordReader[UnsafeRow],

Review comment:
       rename to `HoodieMergedParquetRowIterator` ?

##########
File path: hudi-spark/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetRecordReaderIterator.scala
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.spark.sql.execution.datasources.parquet
+
+import org.apache.avro.Schema
+import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.HOODIE_RECORD_KEY_COL_POS
+import org.apache.hudi.realtime.HoodieRealtimeParquetRecordReader
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.avro.{AvroDeserializer, SchemaConverters}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, UnsafeRow}
+import org.apache.spark.sql.types.StructType
+import java.io.Closeable
+import java.util
+
+/**
+ * This class is the iterator for Hudi MOR table.
+ * This iterator will read the parquet file first and skip the record if it present in the log file.
+ * Then read the log file.
+ * Custom payload is not supported yet. This combining logic is matching with [OverwriteWithLatestAvroPayload]

Review comment:
       Also we have re-implemented the merge by hand again in this class.. I was trying to explore if we can reuse the existing HoodieRecordReader classes by templatizing for `Row` instead of `ArrayWritable` .. That's atleast the more longer term option to pursue.. 

##########
File path: hudi-spark/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieRealtimeInputFormat.scala
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.spark.sql.execution.datasources.parquet
+
+import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapred.{FileSplit, JobConf}
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
+import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType}
+import org.apache.parquet.filter2.compat.FilterCompat
+import org.apache.parquet.filter2.predicate.FilterApi
+import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS
+import org.apache.parquet.hadoop.{ParquetFileReader, ParquetInputFormat, ParquetRecordReader}
+import org.apache.spark.TaskContext
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
+import org.apache.spark.sql.catalyst.expressions.{JoinedRow, UnsafeRow}
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+import java.net.URI
+import scala.collection.JavaConverters._
+
+/**
+ * This class is an extension of ParquetFileFormat from Spark SQL.
+ * The file split, record reader, record reader iterator are customized to read Hudi MOR table.
+ */
+class HoodieRealtimeInputFormat extends ParquetFileFormat {
+  //TODO: Better usage of this short name.
+  override def shortName(): String = "hudi.snapshot"
+  override def toString(): String = "hudi.snapshot"
+
+  override def buildReaderWithPartitionValues(sparkSession: SparkSession,
+                                              dataSchema: StructType,
+                                              partitionSchema: StructType,
+                                              requiredSchema: StructType,
+                                              filters: Seq[Filter], options: Map[String, String],
+                                              hadoopConf: Configuration): (PartitionedFile) =>
+    Iterator[InternalRow] = {
+    hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName)
+    hadoopConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, requiredSchema.json)
+    hadoopConf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, requiredSchema.json)
+    hadoopConf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key,
+      sparkSession.sessionState.conf.sessionLocalTimeZone)
+    hadoopConf.setBoolean(SQLConf.CASE_SENSITIVE.key,
+      sparkSession.sessionState.conf.caseSensitiveAnalysis)
+
+    ParquetWriteSupport.setSchema(requiredSchema, hadoopConf)
+
+    // Sets flags for `ParquetToSparkSchemaConverter`
+    hadoopConf.setBoolean(

Review comment:
       can we avoid all these copied code from ParquetFileFormat by wrapping or overriding? This should be doable?
   
   I am concerned about the amount of borrowed code in this file..




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



[GitHub] [hudi] garyli1019 commented on pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
garyli1019 commented on pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#issuecomment-658842626


   @vinothchandar I agree we should use @umehrot2 RDD approach. 
   >So you can also in parallel just proceed?
   
   Yes, I will change this PR in parallel. 
   >Do you just want the two util classes landed? SparkUtils/TablePathUtils?
   
   Yes, @umehrot2 do you mind making a separate PR for these two classes?
   


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



[GitHub] [hudi] umehrot2 commented on pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
umehrot2 commented on pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#issuecomment-643568699


   Like @vinothchandar I do agree with the **high level approach** here, and thanks for putting out this PR 👍  However, I would highly recommend both of you to check out https://github.com/apache/hudi/pull/1702/ which is along similar lines, and solves some of the issues I see in this PR:
   
   - Here we are instantiating another datasource/relation i.e. `HoodieRealtimeFileFormat` and `spark parquet` relation which has overheads associated with it, like spark having to form index again by listing the paths passed to the datasource.
   
   - We are re-using the `ParquetFileFormat` reader and all of its functionalities like **vectorized reading** , **predicate pushdown**, **column pruning** without having to copy the over and maintain it internally.
   
   - We do not have to pass the expensive `map from parquet to log files` to each task. Instead it gives complete control over what goes into each task partition, and we send only the file and its corresponding mapping (in our case `external data file`, and in this case `log file`) over to the task. It is the very use to **RDD** interface to have that kind of control over the datasource we are building.
   
   Happy to have more in-depth discussion on this and help get this to completion.


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



[GitHub] [hudi] vinothchandar commented on a change in pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on a change in pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#discussion_r438078737



##########
File path: hudi-spark/src/main/scala/org/apache/hudi/RealtimeRelation.scala
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.table.{HoodieTableMetaClient, TableSchemaResolver}
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.hadoop.{HoodieParquetInputFormat, HoodieROTablePathFilter}
+import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.table.HoodieTable
+import org.apache.hadoop.mapred.JobConf
+import org.apache.log4j.LogManager
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{Row, SQLContext}
+import org.apache.spark.sql.sources.{BaseRelation, TableScan}
+import org.apache.spark.sql.types.StructType
+
+import java.util
+import scala.collection.JavaConverters._
+
+/**
+ * This is the Spark DataSourceV1 relation to read Hudi MOR table.
+ * @param sqlContext
+ * @param basePath
+ * @param optParams
+ * @param userSchema
+ */
+class RealtimeRelation(val sqlContext: SQLContext,

Review comment:
       IIUC COW/Snapshot also comes here.. So let's rename this to `SnapshotRelation`. 

##########
File path: hudi-spark/src/main/scala/org/apache/hudi/RealtimeRelation.scala
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.table.{HoodieTableMetaClient, TableSchemaResolver}
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.hadoop.{HoodieParquetInputFormat, HoodieROTablePathFilter}
+import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.table.HoodieTable
+import org.apache.hadoop.mapred.JobConf
+import org.apache.log4j.LogManager
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{Row, SQLContext}
+import org.apache.spark.sql.sources.{BaseRelation, TableScan}
+import org.apache.spark.sql.types.StructType
+
+import java.util
+import scala.collection.JavaConverters._
+
+/**
+ * This is the Spark DataSourceV1 relation to read Hudi MOR table.
+ * @param sqlContext
+ * @param basePath
+ * @param optParams
+ * @param userSchema
+ */
+class RealtimeRelation(val sqlContext: SQLContext,
+                       val basePath: String,
+                       val optParams: Map[String, String],
+                       val userSchema: StructType) extends BaseRelation with TableScan {
+
+  private val log = LogManager.getLogger(classOf[RealtimeRelation])
+  private val conf = sqlContext.sparkContext.hadoopConfiguration
+
+  // Set config for listStatus() in HoodieParquetInputFormat
+  conf.setClass(
+    "mapreduce.input.pathFilter.class",
+    classOf[HoodieROTablePathFilter],
+    classOf[org.apache.hadoop.fs.PathFilter])
+  conf.setStrings("mapreduce.input.fileinputformat.inputdir", basePath)
+  conf.setStrings("mapreduce.input.fileinputformat.input.dir.recursive", "true")
+
+  private val HoodieInputFormat = new HoodieParquetInputFormat
+  HoodieInputFormat.setConf(conf)
+  private val fileStatus = HoodieInputFormat.listStatus(new JobConf(conf))
+  log.debug("All parquet files" + fileStatus.map(s => s.getPath.toString).mkString(","))

Review comment:
       fence with with a `if log.isDebugEnabled()` check?

##########
File path: hudi-spark/src/main/scala/org/apache/hudi/DataSourceOptions.scala
##########
@@ -48,7 +49,7 @@ object DataSourceReadOptions {
   val QUERY_TYPE_SNAPSHOT_OPT_VAL = "snapshot"
   val QUERY_TYPE_READ_OPTIMIZED_OPT_VAL = "read_optimized"
   val QUERY_TYPE_INCREMENTAL_OPT_VAL = "incremental"
-  val DEFAULT_QUERY_TYPE_OPT_VAL: String = QUERY_TYPE_SNAPSHOT_OPT_VAL
+  val DEFAULT_QUERY_TYPE_OPT_VAL: String = QUERY_TYPE_READ_OPTIMIZED_OPT_VAL

Review comment:
       default has to be SNAPSHOT.. why are we changing this..

##########
File path: hudi-spark/src/main/scala/org/apache/hudi/RealtimeRelation.scala
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.table.{HoodieTableMetaClient, TableSchemaResolver}
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.hadoop.{HoodieParquetInputFormat, HoodieROTablePathFilter}
+import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.table.HoodieTable
+import org.apache.hadoop.mapred.JobConf
+import org.apache.log4j.LogManager
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{Row, SQLContext}
+import org.apache.spark.sql.sources.{BaseRelation, TableScan}
+import org.apache.spark.sql.types.StructType
+
+import java.util
+import scala.collection.JavaConverters._
+
+/**
+ * This is the Spark DataSourceV1 relation to read Hudi MOR table.
+ * @param sqlContext
+ * @param basePath
+ * @param optParams
+ * @param userSchema
+ */
+class RealtimeRelation(val sqlContext: SQLContext,
+                       val basePath: String,
+                       val optParams: Map[String, String],
+                       val userSchema: StructType) extends BaseRelation with TableScan {
+
+  private val log = LogManager.getLogger(classOf[RealtimeRelation])
+  private val conf = sqlContext.sparkContext.hadoopConfiguration
+
+  // Set config for listStatus() in HoodieParquetInputFormat
+  conf.setClass(
+    "mapreduce.input.pathFilter.class",
+    classOf[HoodieROTablePathFilter],
+    classOf[org.apache.hadoop.fs.PathFilter])
+  conf.setStrings("mapreduce.input.fileinputformat.inputdir", basePath)
+  conf.setStrings("mapreduce.input.fileinputformat.input.dir.recursive", "true")
+
+  private val HoodieInputFormat = new HoodieParquetInputFormat
+  HoodieInputFormat.setConf(conf)
+  private val fileStatus = HoodieInputFormat.listStatus(new JobConf(conf))
+  log.debug("All parquet files" + fileStatus.map(s => s.getPath.toString).mkString(","))
+  private val fileGroup = HoodieRealtimeInputFormatUtils.groupLogsByBaseFile(conf, util.Arrays.stream(fileStatus)).asScala
+
+  // Split the file group to: parquet file without a matching log file, parquet file need to merge with log files
+  private val parquetWithoutLogPaths: List[String] = fileGroup.filter(p => p._2.size() == 0).keys.toList
+  private val fileWithLogMap: Map[String, String] = fileGroup.filter(p => p._2.size() > 0).map{ case(k, v) => (k, v.asScala.toList.mkString(","))}.toMap
+  log.debug("ParquetWithoutLogPaths" + parquetWithoutLogPaths.mkString(","))
+  log.debug("ParquetWithLogPaths" + fileWithLogMap.map(m => s"${m._1}:${m._2}").mkString(","))
+
+  // Add log file map to options
+  private val finalOps = optParams ++ fileWithLogMap
+
+  // Load Hudi metadata
+  val metaClient = new HoodieTableMetaClient(conf, basePath, true)
+  private val hoodieTable = HoodieTable.create(metaClient, HoodieWriteConfig.newBuilder().withPath(basePath).build(), conf)
+
+  private val commitTimeline = hoodieTable.getMetaClient.getCommitsAndCompactionTimeline
+  if (commitTimeline.empty()) {
+    throw new HoodieException("No Valid Hudi timeline exists")
+  }
+  private val completedCommitTimeline = hoodieTable.getMetaClient.getCommitsTimeline.filterCompletedInstants()
+  private val lastInstant = completedCommitTimeline.lastInstant().get()
+  conf.setStrings("hoodie.realtime.last.commit", lastInstant.getTimestamp)
+
+  // 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)
+  }
+
+  override def schema: StructType = latestSchema
+
+  override def buildScan(): RDD[Row] = {
+    // Read parquet file doesn't have matching log file to merge as normal parquet
+    val regularParquet = sqlContext
+        .read
+        .options(finalOps)
+        .schema(schema)
+        .format("parquet")
+        .load(parquetWithoutLogPaths:_*)
+        .toDF()
+    // Hudi parquet files needed to merge with log file
+    sqlContext

Review comment:
       can we short circuit in cases where `fileWithLogMap` is empty? we will avoid the planning phases for the second datasource.

##########
File path: hudi-spark/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieRealtimeInputFormat.scala
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.spark.sql.execution.datasources.parquet
+
+import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit
+import org.apache.hudi.realtime.HoodieRealtimeParquetRecordReader
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapred.{FileSplit, JobConf}
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
+import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType}
+import org.apache.parquet.filter2.compat.FilterCompat
+import org.apache.parquet.filter2.predicate.FilterApi
+import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS
+import org.apache.parquet.hadoop.{ParquetFileReader, ParquetInputFormat}
+import org.apache.spark.TaskContext
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
+import org.apache.spark.sql.catalyst.expressions.{JoinedRow, UnsafeRow}
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+import java.net.URI
+import scala.collection.JavaConverters._
+
+/**
+ * This class is an extension of ParquetFileFormat from Spark SQL.
+ * The file split, record reader, record reader iterator are customized to read Hudi MOR table.
+ */
+class HoodieRealtimeInputFormat extends ParquetFileFormat {
+  //TODO: Better usage of this short name.
+  override def shortName(): String = "hudi.realtime"
+  override def toString(): String = "hudi.realtime"
+
+  override def buildReaderWithPartitionValues(sparkSession: SparkSession,
+                                               dataSchema: StructType,
+                                               partitionSchema: StructType,
+                                               requiredSchema: StructType,
+                                               filters: Seq[Filter],
+                                               options: Map[String, String],
+                                               hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = {
+    hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName)

Review comment:
       can we get away by delegating this to be super class? 

##########
File path: hudi-spark/src/main/java/org/apache/hudi/realtime/HoodieRealtimeParquetRecordReader.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.realtime;
+
+import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.parquet.CorruptDeltaByteArrays;
+import org.apache.parquet.ParquetReadOptions;
+import org.apache.parquet.column.Encoding;
+import org.apache.parquet.filter2.compat.FilterCompat;
+import org.apache.parquet.filter2.compat.FilterCompat.Filter;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.ParquetRecordReader;
+import org.apache.parquet.hadoop.api.ReadSupport;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.FileMetaData;
+import org.apache.parquet.hadoop.util.ContextUtil;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.hadoop.util.counters.BenchmarkCounter;
+import org.apache.parquet.io.ParquetDecodingException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.apache.parquet.hadoop.ParquetInputFormat.SPLIT_FILES;
+
+/**
+ * Custom implementation of org.apache.parquet.hadoop.ParquetRecordReader.
+ * This class is a wrapper class. The real reader is the internalReader.
+ *
+ * @see ParquetRecordReader
+ *
+ * @param <T> type of the materialized records
+ */
+public class HoodieRealtimeParquetRecordReader<T> extends RecordReader<Void, T> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(HoodieRealtimeParquetRecordReader.class);
+  public final CompactedRealtimeParquetReader<T> internalReader;
+
+  /**
+   * @param readSupport Object which helps reads files of the given type, e.g. Thrift, Avro.
+   */
+  public HoodieRealtimeParquetRecordReader(ReadSupport<T> readSupport, HoodieRealtimeFileSplit split, JobConf job)
+      throws IOException {
+    this(readSupport, FilterCompat.NOOP, split, job);
+  }
+
+  /**
+   * @param readSupport Object which helps reads files of the given type, e.g. Thrift, Avro.
+   * @param filter for filtering individual records
+   */
+  public HoodieRealtimeParquetRecordReader(ReadSupport<T> readSupport, Filter filter, HoodieRealtimeFileSplit split, JobConf job)
+      throws IOException {
+    this.internalReader = new CompactedRealtimeParquetReader<T>(readSupport, filter, split, job);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void close() throws IOException {
+    internalReader.close();
+  }
+
+  /**
+   * always returns null.
+   */
+  @Override
+  public Void getCurrentKey() throws IOException, InterruptedException {
+    return null;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public T getCurrentValue() throws IOException,
+      InterruptedException {
+    return internalReader.getCurrentValue();
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public float getProgress() throws IOException, InterruptedException {
+    return internalReader.getProgress();
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void initialize(InputSplit inputSplit, TaskAttemptContext context)
+      throws IOException, InterruptedException {
+
+    if (ContextUtil.hasCounterMethod(context)) {
+      BenchmarkCounter.initCounterFromContext(context);
+    } else {
+      LOG.error(
+          String.format("Can not initialize counter because the class '%s' does not have a '.getCounterMethod'",
+              context.getClass().getCanonicalName()));
+    }
+
+    initializeInternalReader((HoodieRealtimeFileSplit) inputSplit, ContextUtil.getConfiguration(context));
+  }
+
+  public void initialize(InputSplit inputSplit, Configuration configuration, Reporter reporter)
+      throws IOException, InterruptedException {
+    BenchmarkCounter.initCounterFromReporter(reporter,configuration);
+    initializeInternalReader((HoodieRealtimeFileSplit) inputSplit, configuration);
+  }
+
+  private void initializeInternalReader(HoodieRealtimeFileSplit split, Configuration configuration) throws IOException {
+    Path path = split.getPath();
+    ParquetReadOptions.Builder optionsBuilder = HadoopReadOptions.builder(configuration);
+    optionsBuilder.withRange(split.getStart(), split.getStart() + split.getLength());
+
+    // open a reader with the metadata filter
+    ParquetFileReader reader = ParquetFileReader.open(
+        HadoopInputFile.fromPath(path, configuration), optionsBuilder.build());
+    if (!reader.getRowGroups().isEmpty()) {
+      checkDeltaByteArrayProblem(
+          reader.getFooter().getFileMetaData(), configuration,
+          reader.getRowGroups().get(0));
+    }
+
+    internalReader.initialize(reader, configuration);
+  }
+
+  private void checkDeltaByteArrayProblem(FileMetaData meta, Configuration conf, BlockMetaData block) {
+    if (conf.getBoolean(SPLIT_FILES, true)) {

Review comment:
       is this adapted and reused from somewhere else ?

##########
File path: hudi-spark/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieRealtimeInputFormat.scala
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.spark.sql.execution.datasources.parquet
+
+import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit
+import org.apache.hudi.realtime.HoodieRealtimeParquetRecordReader
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapred.{FileSplit, JobConf}
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
+import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType}
+import org.apache.parquet.filter2.compat.FilterCompat
+import org.apache.parquet.filter2.predicate.FilterApi
+import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS
+import org.apache.parquet.hadoop.{ParquetFileReader, ParquetInputFormat}
+import org.apache.spark.TaskContext
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
+import org.apache.spark.sql.catalyst.expressions.{JoinedRow, UnsafeRow}
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+import java.net.URI
+import scala.collection.JavaConverters._
+
+/**
+ * This class is an extension of ParquetFileFormat from Spark SQL.
+ * The file split, record reader, record reader iterator are customized to read Hudi MOR table.
+ */
+class HoodieRealtimeInputFormat extends ParquetFileFormat {
+  //TODO: Better usage of this short name.
+  override def shortName(): String = "hudi.realtime"
+  override def toString(): String = "hudi.realtime"
+
+  override def buildReaderWithPartitionValues(sparkSession: SparkSession,
+                                               dataSchema: StructType,

Review comment:
       nit: indentation

##########
File path: hudi-spark/src/main/java/org/apache/hudi/realtime/HoodieRealtimeParquetRecordReader.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.realtime;
+
+import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.parquet.CorruptDeltaByteArrays;
+import org.apache.parquet.ParquetReadOptions;
+import org.apache.parquet.column.Encoding;
+import org.apache.parquet.filter2.compat.FilterCompat;
+import org.apache.parquet.filter2.compat.FilterCompat.Filter;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.ParquetRecordReader;
+import org.apache.parquet.hadoop.api.ReadSupport;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.FileMetaData;
+import org.apache.parquet.hadoop.util.ContextUtil;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.hadoop.util.counters.BenchmarkCounter;
+import org.apache.parquet.io.ParquetDecodingException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.apache.parquet.hadoop.ParquetInputFormat.SPLIT_FILES;
+
+/**
+ * Custom implementation of org.apache.parquet.hadoop.ParquetRecordReader.
+ * This class is a wrapper class. The real reader is the internalReader.
+ *
+ * @see ParquetRecordReader
+ *
+ * @param <T> type of the materialized records
+ */
+public class HoodieRealtimeParquetRecordReader<T> extends RecordReader<Void, T> {

Review comment:
       so the main difference here, is the use of `mapred.RecordReader` right.. Did you check hadoop-mr could instead be updated with `mapreduce.RecordReader`

##########
File path: hudi-spark/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetRecordReaderIterator.scala
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.spark.sql.execution.datasources.parquet
+
+import org.apache.avro.Schema
+import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.HOODIE_RECORD_KEY_COL_POS
+import org.apache.hudi.realtime.HoodieRealtimeParquetRecordReader
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.avro.{AvroDeserializer, SchemaConverters}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, UnsafeRow}
+import org.apache.spark.sql.types.StructType
+import java.io.Closeable
+import java.util
+
+/**
+ * This class is the iterator for Hudi MOR table.
+ * This iterator will read the parquet file first and skip the record if it present in the log file.
+ * Then read the log file.
+ * Custom payload is not supported yet. This combining logic is matching with [OverwriteWithLatestAvroPayload]

Review comment:
       custom payload is kind of an important thing, to support out of box.. any complications in supporting that?

##########
File path: hudi-spark/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieRealtimeInputFormat.scala
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.spark.sql.execution.datasources.parquet
+
+import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit
+import org.apache.hudi.realtime.HoodieRealtimeParquetRecordReader
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapred.{FileSplit, JobConf}
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
+import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType}
+import org.apache.parquet.filter2.compat.FilterCompat
+import org.apache.parquet.filter2.predicate.FilterApi
+import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS
+import org.apache.parquet.hadoop.{ParquetFileReader, ParquetInputFormat}
+import org.apache.spark.TaskContext
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
+import org.apache.spark.sql.catalyst.expressions.{JoinedRow, UnsafeRow}
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+import java.net.URI
+import scala.collection.JavaConverters._
+
+/**
+ * This class is an extension of ParquetFileFormat from Spark SQL.
+ * The file split, record reader, record reader iterator are customized to read Hudi MOR table.
+ */
+class HoodieRealtimeInputFormat extends ParquetFileFormat {

Review comment:
       and I guess you have it here in a differnt package to be able to extend ParquetFileFormat? 

##########
File path: hudi-spark/src/main/scala/org/apache/hudi/RealtimeRelation.scala
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.table.{HoodieTableMetaClient, TableSchemaResolver}
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.hadoop.{HoodieParquetInputFormat, HoodieROTablePathFilter}
+import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.table.HoodieTable
+import org.apache.hadoop.mapred.JobConf
+import org.apache.log4j.LogManager
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{Row, SQLContext}
+import org.apache.spark.sql.sources.{BaseRelation, TableScan}
+import org.apache.spark.sql.types.StructType
+
+import java.util
+import scala.collection.JavaConverters._
+
+/**
+ * This is the Spark DataSourceV1 relation to read Hudi MOR table.
+ * @param sqlContext
+ * @param basePath
+ * @param optParams
+ * @param userSchema
+ */
+class RealtimeRelation(val sqlContext: SQLContext,
+                       val basePath: String,
+                       val optParams: Map[String, String],
+                       val userSchema: StructType) extends BaseRelation with TableScan {
+
+  private val log = LogManager.getLogger(classOf[RealtimeRelation])
+  private val conf = sqlContext.sparkContext.hadoopConfiguration
+
+  // Set config for listStatus() in HoodieParquetInputFormat
+  conf.setClass(
+    "mapreduce.input.pathFilter.class",
+    classOf[HoodieROTablePathFilter],
+    classOf[org.apache.hadoop.fs.PathFilter])
+  conf.setStrings("mapreduce.input.fileinputformat.inputdir", basePath)
+  conf.setStrings("mapreduce.input.fileinputformat.input.dir.recursive", "true")
+
+  private val HoodieInputFormat = new HoodieParquetInputFormat
+  HoodieInputFormat.setConf(conf)
+  private val fileStatus = HoodieInputFormat.listStatus(new JobConf(conf))
+  log.debug("All parquet files" + fileStatus.map(s => s.getPath.toString).mkString(","))
+  private val fileGroup = HoodieRealtimeInputFormatUtils.groupLogsByBaseFile(conf, util.Arrays.stream(fileStatus)).asScala
+
+  // Split the file group to: parquet file without a matching log file, parquet file need to merge with log files
+  private val parquetWithoutLogPaths: List[String] = fileGroup.filter(p => p._2.size() == 0).keys.toList
+  private val fileWithLogMap: Map[String, String] = fileGroup.filter(p => p._2.size() > 0).map{ case(k, v) => (k, v.asScala.toList.mkString(","))}.toMap
+  log.debug("ParquetWithoutLogPaths" + parquetWithoutLogPaths.mkString(","))
+  log.debug("ParquetWithLogPaths" + fileWithLogMap.map(m => s"${m._1}:${m._2}").mkString(","))
+
+  // Add log file map to options
+  private val finalOps = optParams ++ fileWithLogMap
+
+  // Load Hudi metadata
+  val metaClient = new HoodieTableMetaClient(conf, basePath, true)
+  private val hoodieTable = HoodieTable.create(metaClient, HoodieWriteConfig.newBuilder().withPath(basePath).build(), conf)
+
+  private val commitTimeline = hoodieTable.getMetaClient.getCommitsAndCompactionTimeline
+  if (commitTimeline.empty()) {
+    throw new HoodieException("No Valid Hudi timeline exists")
+  }
+  private val completedCommitTimeline = hoodieTable.getMetaClient.getCommitsTimeline.filterCompletedInstants()
+  private val lastInstant = completedCommitTimeline.lastInstant().get()
+  conf.setStrings("hoodie.realtime.last.commit", lastInstant.getTimestamp)
+
+  // 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)
+  }
+
+  override def schema: StructType = latestSchema
+
+  override def buildScan(): RDD[Row] = {
+    // Read parquet file doesn't have matching log file to merge as normal parquet
+    val regularParquet = sqlContext
+        .read
+        .options(finalOps)
+        .schema(schema)
+        .format("parquet")
+        .load(parquetWithoutLogPaths:_*)
+        .toDF()
+    // Hudi parquet files needed to merge with log file
+    sqlContext
+      .read
+      .options(finalOps)
+      .schema(schema)
+      .format("org.apache.spark.sql.execution.datasources.parquet.HoodieRealtimeInputFormat")
+      .load(fileWithLogMap.keys.toList:_*)
+      .toDF()
+      .union(regularParquet)
+      .rdd

Review comment:
       this can be hurting performance? the .rdd conversion.. 

##########
File path: hudi-spark/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieRealtimeInputFormat.scala
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.spark.sql.execution.datasources.parquet
+
+import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit
+import org.apache.hudi.realtime.HoodieRealtimeParquetRecordReader
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapred.{FileSplit, JobConf}
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
+import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType}
+import org.apache.parquet.filter2.compat.FilterCompat
+import org.apache.parquet.filter2.predicate.FilterApi
+import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS
+import org.apache.parquet.hadoop.{ParquetFileReader, ParquetInputFormat}
+import org.apache.spark.TaskContext
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
+import org.apache.spark.sql.catalyst.expressions.{JoinedRow, UnsafeRow}
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+import java.net.URI
+import scala.collection.JavaConverters._
+
+/**
+ * This class is an extension of ParquetFileFormat from Spark SQL.
+ * The file split, record reader, record reader iterator are customized to read Hudi MOR table.
+ */
+class HoodieRealtimeInputFormat extends ParquetFileFormat {
+  //TODO: Better usage of this short name.
+  override def shortName(): String = "hudi.realtime"
+  override def toString(): String = "hudi.realtime"
+
+  override def buildReaderWithPartitionValues(sparkSession: SparkSession,
+                                               dataSchema: StructType,
+                                               partitionSchema: StructType,
+                                               requiredSchema: StructType,
+                                               filters: Seq[Filter],
+                                               options: Map[String, String],
+                                               hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = {
+    hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName)
+    hadoopConf.set(
+      ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA,
+      requiredSchema.json)
+    hadoopConf.set(
+      ParquetWriteSupport.SPARK_ROW_SCHEMA,
+      requiredSchema.json)
+    hadoopConf.set(
+      SQLConf.SESSION_LOCAL_TIMEZONE.key,
+      sparkSession.sessionState.conf.sessionLocalTimeZone)
+    hadoopConf.setBoolean(
+      SQLConf.CASE_SENSITIVE.key,
+      sparkSession.sessionState.conf.caseSensitiveAnalysis)
+
+    ParquetWriteSupport.setSchema(requiredSchema, hadoopConf)
+
+    // Sets flags for `ParquetToSparkSchemaConverter`
+    hadoopConf.setBoolean(
+      SQLConf.PARQUET_BINARY_AS_STRING.key,
+      sparkSession.sessionState.conf.isParquetBinaryAsString)
+    hadoopConf.setBoolean(
+      SQLConf.PARQUET_INT96_AS_TIMESTAMP.key,
+      sparkSession.sessionState.conf.isParquetINT96AsTimestamp)
+
+    val broadcastedHadoopConf =
+      sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf))
+
+    // TODO: if you move this into the closure it reverts to the default values.
+    // If true, enable using the custom RecordReader for parquet. This only works for
+    // a subset of the types (no complex types).
+    val resultSchema = StructType(partitionSchema.fields ++ requiredSchema.fields)
+    val sqlConf = sparkSession.sessionState.conf
+    val enableRecordFilter: Boolean = sqlConf.parquetRecordFilterEnabled
+    val timestampConversion: Boolean = sqlConf.isParquetINT96TimestampConversion
+    val enableParquetFilterPushDown: Boolean = sqlConf.parquetFilterPushDown
+    // Whole stage codegen (PhysicalRDD) is able to deal with batches directly
+    val returningBatch = supportBatch(sparkSession, resultSchema)
+    val pushDownDate = sqlConf.parquetFilterPushDownDate
+    val pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp
+    val pushDownDecimal = sqlConf.parquetFilterPushDownDecimal
+    val pushDownStringStartWith = sqlConf.parquetFilterPushDownStringStartWith
+    val pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold
+    val isCaseSensitive = sqlConf.caseSensitiveAnalysis
+
+    (file: PartitionedFile) => {
+      assert(file.partitionValues.numFields == partitionSchema.size)
+
+      val sharedConf = broadcastedHadoopConf.value.value
+      val fileSplit =
+        new FileSplit(new Path(new URI(file.filePath)), file.start, file.length, new Array[String](0))
+      val filePath = fileSplit.getPath
+
+      val basePath = sharedConf.get("mapreduce.input.fileinputformat.inputdir")
+      val maxCommitTime = sharedConf.get("hoodie.realtime.last.commit")
+      // Read the log file path from the option
+      val logPathStr = options.getOrElse(fileSplit.getPath.toString, "").split(",")
+      log.debug(s"fileSplit.getPath in HoodieRealtimeInputFormat: ${fileSplit.getPath} and ${fileSplit.getPath.getName}")
+      log.debug(s"logPath in HoodieRealtimeInputFormat: ${logPathStr.toString}")
+      val hoodieRealtimeFileSplit = new HoodieRealtimeFileSplit(fileSplit, basePath, logPathStr.toList.asJava, maxCommitTime)
+
+      lazy val footerFileMetaData =
+        ParquetFileReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData
+      // Try to push down filters when filter push-down is enabled.
+      val pushed = if (enableParquetFilterPushDown) {
+        val parquetSchema = footerFileMetaData.getSchema
+        val parquetFilters = new ParquetFilters(pushDownDate, pushDownTimestamp, pushDownDecimal,
+          pushDownStringStartWith, pushDownInFilterThreshold, isCaseSensitive)
+        filters
+          // Collects all converted Parquet filter predicates. Notice that not all predicates can be
+          // converted (`ParquetFilters.createFilter` returns an `Option`). That's why a `flatMap`
+          // is used here.
+          .flatMap(parquetFilters.createFilter(parquetSchema, _))
+          .reduceOption(FilterApi.and)
+      } else {
+        None
+      }
+
+      // PARQUET_INT96_TIMESTAMP_CONVERSION says to apply timezone conversions to int96 timestamps'
+      // *only* if the file was created by something other than "parquet-mr", so check the actual
+      // writer here for this file.  We have to do this per-file, as each file in the table may
+      // have different writers.
+      // Define isCreatedByParquetMr as function to avoid unnecessary parquet footer reads.
+      def isCreatedByParquetMr: Boolean =
+        footerFileMetaData.getCreatedBy().startsWith("parquet-mr")
+
+      val convertTz =
+        if (timestampConversion && !isCreatedByParquetMr) {
+          Some(DateTimeUtils.getTimeZone(sharedConf.get(SQLConf.SESSION_LOCAL_TIMEZONE.key)))
+        } else {
+          None
+        }
+
+      val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0)
+      val hadoopAttemptContext =
+        new TaskAttemptContextImpl(broadcastedHadoopConf.value.value, attemptId)
+
+      // Try to push down filters when filter push-down is enabled.
+      // Notice: This push-down is RowGroups level, not individual records.
+      if (pushed.isDefined) {
+        ParquetInputFormat.setFilterPredicate(hadoopAttemptContext.getConfiguration, pushed.get)
+      }
+      val taskContext = Option(TaskContext.get())
+      //TODO: Support the vectorized reader.
+      logDebug(s"Falling back to parquet-mr")
+      // ParquetRecordReader returns UnsafeRow
+      val reader = if (pushed.isDefined && enableRecordFilter) {
+        val parquetFilter = FilterCompat.get(pushed.get, null)
+        new HoodieRealtimeParquetRecordReader[UnsafeRow](new ParquetReadSupport(convertTz), parquetFilter, hoodieRealtimeFileSplit, new JobConf(sharedConf))
+      } else {
+        new HoodieRealtimeParquetRecordReader[UnsafeRow](new ParquetReadSupport(convertTz), hoodieRealtimeFileSplit, new JobConf(sharedConf))
+      }
+      val iter = new HoodieParquetRecordReaderIterator(reader)
+      // SPARK-23457 Register a task completion lister before `initialization`.
+      taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close()))
+      reader.initialize(hoodieRealtimeFileSplit, hadoopAttemptContext)
+      iter.init()
+
+      val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes
+      val joinedRow = new JoinedRow()
+      val appendPartitionColumns = GenerateUnsafeProjection.generate(fullSchema, fullSchema)
+
+      // This is a horrible erasure hack...  if we type the iterator above, then it actually check

Review comment:
       is this codde re-used from some place? 

##########
File path: hudi-spark/src/main/java/org/apache/hudi/realtime/AbstractRealtimeParquetReader.java
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.realtime;
+
+import org.apache.hudi.common.table.log.LogReaderUtils;
+import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit;
+import org.apache.hudi.hadoop.config.HoodieRealtimeConfig;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.ParquetReadOptions;
+import org.apache.parquet.avro.AvroSchemaConverter;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.filter2.compat.FilterCompat;
+import org.apache.parquet.filter2.compat.FilterCompat.Filter;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.UnmaterializableRecordCounter;
+import org.apache.parquet.hadoop.api.InitContext;
+import org.apache.parquet.hadoop.api.ReadSupport;
+import org.apache.parquet.hadoop.metadata.FileMetaData;
+import org.apache.parquet.hadoop.util.counters.BenchmarkCounter;
+import org.apache.parquet.io.ColumnIOFactory;
+import org.apache.parquet.io.MessageColumnIO;
+import org.apache.parquet.io.api.RecordMaterializer;
+import org.apache.parquet.schema.MessageType;
+import org.mortbay.log.Log;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.parquet.Preconditions.checkNotNull;
+import static org.apache.parquet.hadoop.ParquetInputFormat.RECORD_FILTERING_ENABLED;
+import static org.apache.parquet.hadoop.ParquetInputFormat.STRICT_TYPE_CHECKING;
+
+/**
+ * This class is customized from org.apache.parquet.hadoop.InternalParquetRecordReader combining with AbstractRealtimeRecordReader.

Review comment:
       This will become a maintenance issue.. is there a way to wrap this..
   
   (Will review this more deeply)

##########
File path: hudi-spark/src/main/scala/org/apache/hudi/RealtimeRelation.scala
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.table.{HoodieTableMetaClient, TableSchemaResolver}
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.hadoop.{HoodieParquetInputFormat, HoodieROTablePathFilter}
+import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.table.HoodieTable
+import org.apache.hadoop.mapred.JobConf
+import org.apache.log4j.LogManager
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{Row, SQLContext}
+import org.apache.spark.sql.sources.{BaseRelation, TableScan}
+import org.apache.spark.sql.types.StructType
+
+import java.util
+import scala.collection.JavaConverters._
+
+/**
+ * This is the Spark DataSourceV1 relation to read Hudi MOR table.
+ * @param sqlContext
+ * @param basePath
+ * @param optParams
+ * @param userSchema
+ */
+class RealtimeRelation(val sqlContext: SQLContext,
+                       val basePath: String,
+                       val optParams: Map[String, String],
+                       val userSchema: StructType) extends BaseRelation with TableScan {
+
+  private val log = LogManager.getLogger(classOf[RealtimeRelation])
+  private val conf = sqlContext.sparkContext.hadoopConfiguration
+
+  // Set config for listStatus() in HoodieParquetInputFormat
+  conf.setClass(
+    "mapreduce.input.pathFilter.class",
+    classOf[HoodieROTablePathFilter],
+    classOf[org.apache.hadoop.fs.PathFilter])
+  conf.setStrings("mapreduce.input.fileinputformat.inputdir", basePath)
+  conf.setStrings("mapreduce.input.fileinputformat.input.dir.recursive", "true")
+
+  private val HoodieInputFormat = new HoodieParquetInputFormat
+  HoodieInputFormat.setConf(conf)
+  private val fileStatus = HoodieInputFormat.listStatus(new JobConf(conf))
+  log.debug("All parquet files" + fileStatus.map(s => s.getPath.toString).mkString(","))
+  private val fileGroup = HoodieRealtimeInputFormatUtils.groupLogsByBaseFile(conf, util.Arrays.stream(fileStatus)).asScala
+
+  // Split the file group to: parquet file without a matching log file, parquet file need to merge with log files
+  private val parquetWithoutLogPaths: List[String] = fileGroup.filter(p => p._2.size() == 0).keys.toList
+  private val fileWithLogMap: Map[String, String] = fileGroup.filter(p => p._2.size() > 0).map{ case(k, v) => (k, v.asScala.toList.mkString(","))}.toMap
+  log.debug("ParquetWithoutLogPaths" + parquetWithoutLogPaths.mkString(","))
+  log.debug("ParquetWithLogPaths" + fileWithLogMap.map(m => s"${m._1}:${m._2}").mkString(","))
+
+  // Add log file map to options
+  private val finalOps = optParams ++ fileWithLogMap
+
+  // Load Hudi metadata
+  val metaClient = new HoodieTableMetaClient(conf, basePath, true)
+  private val hoodieTable = HoodieTable.create(metaClient, HoodieWriteConfig.newBuilder().withPath(basePath).build(), conf)
+
+  private val commitTimeline = hoodieTable.getMetaClient.getCommitsAndCompactionTimeline
+  if (commitTimeline.empty()) {
+    throw new HoodieException("No Valid Hudi timeline exists")
+  }
+  private val completedCommitTimeline = hoodieTable.getMetaClient.getCommitsTimeline.filterCompletedInstants()
+  private val lastInstant = completedCommitTimeline.lastInstant().get()
+  conf.setStrings("hoodie.realtime.last.commit", lastInstant.getTimestamp)
+
+  // 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)
+  }
+
+  override def schema: StructType = latestSchema
+
+  override def buildScan(): RDD[Row] = {
+    // Read parquet file doesn't have matching log file to merge as normal parquet

Review comment:
       one thing to ensure is COW/Snapshot will use spark's native parquet reader.. I think we push the same path filter above and reuse the spark source here. so should work.. But good to test once with few queries and ensure there is no change in perf




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



[GitHub] [hudi] garyli1019 commented on a change in pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
garyli1019 commented on a change in pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#discussion_r447407892



##########
File path: hudi-spark/src/main/scala/org/apache/hudi/DataSourceOptions.scala
##########
@@ -65,7 +66,7 @@ object DataSourceReadOptions {
     * This eases migration from old configs to new configs.
     */
   def translateViewTypesToQueryTypes(optParams: Map[String, String]) : Map[String, String] = {
-    val translation = Map(VIEW_TYPE_READ_OPTIMIZED_OPT_VAL -> QUERY_TYPE_SNAPSHOT_OPT_VAL,
+    val translation = Map(VIEW_TYPE_READ_OPTIMIZED_OPT_VAL -> QUERY_TYPE_READ_OPTIMIZED_OPT_VAL,

Review comment:
       Let's keep this mapping because we should be able to do RO view on MOR.




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



[GitHub] [hudi] garyli1019 commented on a change in pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
garyli1019 commented on a change in pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#discussion_r438488757



##########
File path: hudi-spark/src/main/scala/org/apache/hudi/RealtimeRelation.scala
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.table.{HoodieTableMetaClient, TableSchemaResolver}
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.hadoop.{HoodieParquetInputFormat, HoodieROTablePathFilter}
+import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.table.HoodieTable
+import org.apache.hadoop.mapred.JobConf
+import org.apache.log4j.LogManager
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{Row, SQLContext}
+import org.apache.spark.sql.sources.{BaseRelation, TableScan}
+import org.apache.spark.sql.types.StructType
+
+import java.util
+import scala.collection.JavaConverters._
+
+/**
+ * This is the Spark DataSourceV1 relation to read Hudi MOR table.
+ * @param sqlContext
+ * @param basePath
+ * @param optParams
+ * @param userSchema
+ */
+class RealtimeRelation(val sqlContext: SQLContext,
+                       val basePath: String,
+                       val optParams: Map[String, String],
+                       val userSchema: StructType) extends BaseRelation with TableScan {
+
+  private val log = LogManager.getLogger(classOf[RealtimeRelation])
+  private val conf = sqlContext.sparkContext.hadoopConfiguration
+
+  // Set config for listStatus() in HoodieParquetInputFormat
+  conf.setClass(
+    "mapreduce.input.pathFilter.class",
+    classOf[HoodieROTablePathFilter],
+    classOf[org.apache.hadoop.fs.PathFilter])
+  conf.setStrings("mapreduce.input.fileinputformat.inputdir", basePath)
+  conf.setStrings("mapreduce.input.fileinputformat.input.dir.recursive", "true")
+
+  private val HoodieInputFormat = new HoodieParquetInputFormat
+  HoodieInputFormat.setConf(conf)
+  private val fileStatus = HoodieInputFormat.listStatus(new JobConf(conf))
+  log.debug("All parquet files" + fileStatus.map(s => s.getPath.toString).mkString(","))
+  private val fileGroup = HoodieRealtimeInputFormatUtils.groupLogsByBaseFile(conf, util.Arrays.stream(fileStatus)).asScala
+
+  // Split the file group to: parquet file without a matching log file, parquet file need to merge with log files
+  private val parquetWithoutLogPaths: List[String] = fileGroup.filter(p => p._2.size() == 0).keys.toList
+  private val fileWithLogMap: Map[String, String] = fileGroup.filter(p => p._2.size() > 0).map{ case(k, v) => (k, v.asScala.toList.mkString(","))}.toMap
+  log.debug("ParquetWithoutLogPaths" + parquetWithoutLogPaths.mkString(","))
+  log.debug("ParquetWithLogPaths" + fileWithLogMap.map(m => s"${m._1}:${m._2}").mkString(","))
+
+  // Add log file map to options
+  private val finalOps = optParams ++ fileWithLogMap
+
+  // Load Hudi metadata
+  val metaClient = new HoodieTableMetaClient(conf, basePath, true)
+  private val hoodieTable = HoodieTable.create(metaClient, HoodieWriteConfig.newBuilder().withPath(basePath).build(), conf)
+
+  private val commitTimeline = hoodieTable.getMetaClient.getCommitsAndCompactionTimeline
+  if (commitTimeline.empty()) {
+    throw new HoodieException("No Valid Hudi timeline exists")
+  }
+  private val completedCommitTimeline = hoodieTable.getMetaClient.getCommitsTimeline.filterCompletedInstants()
+  private val lastInstant = completedCommitTimeline.lastInstant().get()
+  conf.setStrings("hoodie.realtime.last.commit", lastInstant.getTimestamp)
+
+  // 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)
+  }
+
+  override def schema: StructType = latestSchema
+
+  override def buildScan(): RDD[Row] = {
+    // Read parquet file doesn't have matching log file to merge as normal parquet

Review comment:
       Yea we can do this once we have the benchmarking framework ready.




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



[GitHub] [hudi] codecov-commenter commented on pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#issuecomment-643095877


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/1722?src=pr&el=h1) Report
   > Merging [#1722](https://codecov.io/gh/apache/hudi/pull/1722?src=pr&el=desc) into [master](https://codecov.io/gh/apache/hudi/commit/37838cea6094ddc66191df42e8b2c84f132d1623&el=desc) will **decrease** coverage by `0.24%`.
   > The diff coverage is `0.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/1722/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/1722?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #1722      +/-   ##
   ============================================
   - Coverage     18.16%   17.91%   -0.25%     
   + Complexity      860      859       -1     
   ============================================
     Files           352      355       +3     
     Lines         15411    15613     +202     
     Branches       1525     1550      +25     
   ============================================
   - Hits           2799     2797       -2     
   - Misses        12254    12458     +204     
     Partials        358      358              
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/1722?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...main/scala/org/apache/hudi/DataSourceOptions.scala](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2h1ZGkvRGF0YVNvdXJjZU9wdGlvbnMuc2NhbGE=) | `68.13% <0.00%> (ø)` | `0.00 <0.00> (ø)` | |
   | [...src/main/scala/org/apache/hudi/DefaultSource.scala](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2h1ZGkvRGVmYXVsdFNvdXJjZS5zY2FsYQ==) | `0.00% <0.00%> (ø)` | `0.00 <0.00> (ø)` | |
   | [.../main/scala/org/apache/hudi/SnapshotRelation.scala](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2h1ZGkvU25hcHNob3RSZWxhdGlvbi5zY2FsYQ==) | `0.00% <0.00%> (ø)` | `0.00 <0.00> (?)` | |
   | [...es/parquet/HoodieParquetRecordReaderIterator.scala](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL3NwYXJrL3NxbC9leGVjdXRpb24vZGF0YXNvdXJjZXMvcGFycXVldC9Ib29kaWVQYXJxdWV0UmVjb3JkUmVhZGVySXRlcmF0b3Iuc2NhbGE=) | `0.00% <0.00%> (ø)` | `0.00 <0.00> (?)` | |
   | [...atasources/parquet/HoodieRealtimeInputFormat.scala](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL3NwYXJrL3NxbC9leGVjdXRpb24vZGF0YXNvdXJjZXMvcGFycXVldC9Ib29kaWVSZWFsdGltZUlucHV0Rm9ybWF0LnNjYWxh) | `0.00% <0.00%> (ø)` | `0.00 <0.00> (?)` | |
   | [...apache/hudi/common/fs/HoodieWrapperFileSystem.java](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL2ZzL0hvb2RpZVdyYXBwZXJGaWxlU3lzdGVtLmphdmE=) | `21.98% <0.00%> (-0.71%)` | `28.00% <0.00%> (-1.00%)` | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/hudi/pull/1722?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/hudi/pull/1722?src=pr&el=footer). Last update [37838ce...ccb620e](https://codecov.io/gh/apache/hudi/pull/1722?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


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



[GitHub] [hudi] garyli1019 commented on pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
garyli1019 commented on pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#issuecomment-642348677


   Successfully got rid of those `RecordReaders`! @vinothchandar Thanks for the hint!


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



[GitHub] [hudi] codecov-commenter edited a comment on pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#issuecomment-643095877


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/1722?src=pr&el=h1) Report
   > Merging [#1722](https://codecov.io/gh/apache/hudi/pull/1722?src=pr&el=desc) into [master](https://codecov.io/gh/apache/hudi/commit/37838cea6094ddc66191df42e8b2c84f132d1623&el=desc) will **decrease** coverage by `0.24%`.
   > The diff coverage is `0.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/1722/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/1722?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #1722      +/-   ##
   ============================================
   - Coverage     18.16%   17.91%   -0.25%     
   + Complexity      860      859       -1     
   ============================================
     Files           352      355       +3     
     Lines         15411    15613     +202     
     Branches       1525     1550      +25     
   ============================================
   - Hits           2799     2797       -2     
   - Misses        12254    12458     +204     
     Partials        358      358              
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/1722?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...main/scala/org/apache/hudi/DataSourceOptions.scala](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2h1ZGkvRGF0YVNvdXJjZU9wdGlvbnMuc2NhbGE=) | `68.13% <0.00%> (ø)` | `0.00 <0.00> (ø)` | |
   | [...src/main/scala/org/apache/hudi/DefaultSource.scala](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2h1ZGkvRGVmYXVsdFNvdXJjZS5zY2FsYQ==) | `0.00% <0.00%> (ø)` | `0.00 <0.00> (ø)` | |
   | [.../main/scala/org/apache/hudi/SnapshotRelation.scala](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2h1ZGkvU25hcHNob3RSZWxhdGlvbi5zY2FsYQ==) | `0.00% <0.00%> (ø)` | `0.00 <0.00> (?)` | |
   | [...es/parquet/HoodieParquetRecordReaderIterator.scala](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL3NwYXJrL3NxbC9leGVjdXRpb24vZGF0YXNvdXJjZXMvcGFycXVldC9Ib29kaWVQYXJxdWV0UmVjb3JkUmVhZGVySXRlcmF0b3Iuc2NhbGE=) | `0.00% <0.00%> (ø)` | `0.00 <0.00> (?)` | |
   | [...atasources/parquet/HoodieRealtimeInputFormat.scala](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL3NwYXJrL3NxbC9leGVjdXRpb24vZGF0YXNvdXJjZXMvcGFycXVldC9Ib29kaWVSZWFsdGltZUlucHV0Rm9ybWF0LnNjYWxh) | `0.00% <0.00%> (ø)` | `0.00 <0.00> (?)` | |
   | [...apache/hudi/common/fs/HoodieWrapperFileSystem.java](https://codecov.io/gh/apache/hudi/pull/1722/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL2ZzL0hvb2RpZVdyYXBwZXJGaWxlU3lzdGVtLmphdmE=) | `21.98% <0.00%> (-0.71%)` | `28.00% <0.00%> (-1.00%)` | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/hudi/pull/1722?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/hudi/pull/1722?src=pr&el=footer). Last update [37838ce...ccb620e](https://codecov.io/gh/apache/hudi/pull/1722?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


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