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



##########
File path: 
hudi-spark/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala
##########
@@ -0,0 +1,209 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi
+
+import org.apache.hudi.common.model.{HoodieCommitMetadata, HoodieRecord, 
HoodieTableType}
+import org.apache.hudi.common.table.{HoodieTableMetaClient, 
TableSchemaResolver}
+import org.apache.hudi.exception.HoodieException
+import org.apache.hadoop.fs.{FileStatus, FileSystem, GlobPattern, Path}
+import org.apache.hadoop.mapred.JobConf
+import org.apache.hudi.common.fs.FSUtils
+import org.apache.hudi.common.table.view.HoodieTableFileSystemView
+import 
org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils.getMaxCompactionMemoryInBytes
+import org.apache.log4j.LogManager
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
+import org.apache.spark.sql.sources.{BaseRelation, Filter, GreaterThanOrEqual, 
IsNotNull, LessThanOrEqual, PrunedFilteredScan, TableScan}
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.{Row, SQLContext}
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable.ListBuffer
+
+/**
+  * Experimental.
+  * Relation, that implements the Hoodie incremental view for Merge On Read 
table.
+  *
+  */
+class MergeOnReadIncrementalRelation(val sqlContext: SQLContext,
+                                     val optParams: Map[String, String],
+                                     val userSchema: StructType,
+                                     val metaClient: HoodieTableMetaClient)
+  extends BaseRelation with PrunedFilteredScan {
+
+  private val log = 
LogManager.getLogger(classOf[MergeOnReadIncrementalRelation])
+  private val conf = sqlContext.sparkContext.hadoopConfiguration
+  private val jobConf = new JobConf(conf)
+  private val fs = FSUtils.getFs(metaClient.getBasePath, conf)
+  private val commitTimeline = 
metaClient.getCommitsAndCompactionTimeline.filterCompletedInstants()
+  if (commitTimeline.empty()) {
+    throw new HoodieException("No instants to incrementally pull")
+  }
+  if (!optParams.contains(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY)) {
+    throw new HoodieException(s"Specify the begin instant time to pull from 
using " +
+      s"option ${DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY}")
+  }
+
+  private val lastInstant = commitTimeline.lastInstant().get()
+  private val mergeType = optParams.getOrElse(
+    DataSourceReadOptions.REALTIME_MERGE_OPT_KEY,
+    DataSourceReadOptions.DEFAULT_REALTIME_MERGE_OPT_VAL)
+
+  private val commitsTimelineToReturn = commitTimeline.findInstantsInRange(
+    optParams(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY),
+    optParams.getOrElse(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, 
lastInstant.getTimestamp))
+  log.debug(s"${commitsTimelineToReturn.getInstants.iterator().toList.map(f => 
f.toString).mkString(",")}")
+  private val commitsToReturn = 
commitsTimelineToReturn.getInstants.iterator().toList
+  private val schemaUtil = new TableSchemaResolver(metaClient)
+  private val tableAvroSchema = schemaUtil.getTableAvroSchema
+  private val tableStructSchema = 
AvroConversionUtils.convertAvroSchemaToStructType(tableAvroSchema)
+  private val maxCompactionMemoryInBytes = 
getMaxCompactionMemoryInBytes(jobConf)
+  private val fileIndex = buildFileIndex()
+
+  override def schema: StructType = tableStructSchema
+
+  override def needConversion: Boolean = false
+
+  override def unhandledFilters(filters: Array[Filter]): Array[Filter] = {
+    val isNotNullFilter = IsNotNull(HoodieRecord.COMMIT_TIME_METADATA_FIELD)
+    val largerThanFilter = 
GreaterThanOrEqual(HoodieRecord.COMMIT_TIME_METADATA_FIELD, 
commitsToReturn.head.getTimestamp)
+    val lessThanFilter = 
LessThanOrEqual(HoodieRecord.COMMIT_TIME_METADATA_FIELD, 
commitsToReturn.last.getTimestamp)
+    filters :+isNotNullFilter :+ largerThanFilter :+ lessThanFilter
+  }
+
+  override def buildScan(requiredColumns: Array[String], filters: 
Array[Filter]): RDD[Row] = {
+    log.debug(s"buildScan requiredColumns = ${requiredColumns.mkString(",")}")
+    log.debug(s"buildScan filters = ${filters.mkString(",")}")
+    // config to ensure the push down filter for parquet will be applied.

Review comment:
       This is the tricky part of this PR. We need to force a filter on 
`_hoodie_commit_time`, like what we have 
https://github.com/apache/hudi/blob/master/hudi-spark/src/main/scala/org/apache/hudi/IncrementalRelation.scala#L165.
   
   Currently, I couldn't find an elegant way to force the filter regardless of 
what query the user is running. Spark optimization sometimes will skip scanning 
the file and produce incorrect results(e.g. count()). If we add a filter in 
RDD, then we will get involved in an extra ser/deser step. The elegant way 
might be getting into the Spark planing, but at this point I have no idea about 
how to do it. The easiest way is to ask the user to add a `.filter()` when 
loading the dataset, but we definitely don't wanna do that :) 
   
   My approach here is to force Spark to always scan the Base file and always 
apply the filter:
   - For the Base file, force the pushdown filter and avoid using the default 
ParquetFileFormat reader(baseFileIterator), which will not scan the file when 
the user uses `df.count()` and will produce incorrect results.
   - For log file, no need to filter base on commit time.
   
   So we need to make sure the pushdown filter was on in every run if we keep 
this approach, otherwise the result will be incorrect.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to