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



##########
File path: hudi-spark/src/main/scala/org/apache/hudi/SnapshotRelation.scala
##########
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi
+
+import org.apache.hudi.avro.HoodieAvroUtils
+import org.apache.hudi.common.model.HoodieBaseFile
+import org.apache.hudi.common.table.{HoodieTableMetaClient, 
TableSchemaResolver}
+import org.apache.hudi.common.table.view.HoodieTableFileSystemView
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils
+import 
org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils.getMaxCompactionMemoryInBytes
+
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapred.JobConf
+import org.apache.spark.internal.Logging
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
+import org.apache.spark.sql.{Row, SQLContext}
+import org.apache.spark.sql.sources.{BaseRelation, TableScan}
+import org.apache.spark.sql.types.StructType
+
+import scala.collection.JavaConverters._
+
+case class HudiMergeOnReadFileSplit(dataFile: PartitionedFile,
+                                    logPaths: Option[List[String]],
+                                    latestCommit: String,
+                                    tablePath: String,
+                                    maxCompactionMemoryInBytes: Long,
+                                    skipMerge: Boolean)
+
+class SnapshotRelation (val sqlContext: SQLContext,
+                        val optParams: Map[String, String],
+                        val userSchema: StructType,
+                        val globPaths: Seq[Path],
+                        val metaClient: HoodieTableMetaClient)
+  extends BaseRelation with TableScan with Logging{
+
+  private val conf = sqlContext.sparkContext.hadoopConfiguration
+
+  // use schema from latest metadata, if not present, read schema from the 
data file
+  private val latestSchema = {
+    val schemaUtil = new TableSchemaResolver(metaClient)
+    val tableSchema = 
HoodieAvroUtils.createHoodieWriteSchema(schemaUtil.getTableAvroSchemaWithoutMetadataFields)
+    AvroConversionUtils.convertAvroSchemaToStructType(tableSchema)
+  }
+
+  private val skipMerge = optParams.getOrElse(
+    DataSourceReadOptions.REALTIME_SKIP_MERGE_KEY,
+    DataSourceReadOptions.DEFAULT_REALTIME_SKIP_MERGE_VAL).toBoolean
+  private val maxCompactionMemoryInBytes = getMaxCompactionMemoryInBytes(new 
JobConf(conf))
+  private val fileIndex = buildFileIndex()
+
+  override def schema: StructType = latestSchema
+
+  override def needConversion: Boolean = false
+
+  override def buildScan(): RDD[Row] = {
+    val parquetReaderFunction = new 
ParquetFileFormat().buildReaderWithPartitionValues(
+      sparkSession = sqlContext.sparkSession,
+      dataSchema = latestSchema,
+      partitionSchema = StructType(Nil),
+      requiredSchema = latestSchema,
+      filters = Seq.empty,

Review comment:
       We should try to support filter push-down. Again for the more straight 
forward scenarios which I mentioned in my other comment, it should be possible 
to just pass down the user filters in the reader.
   
   However, for Log file merging scenario we may have to take care of the 
following scenario:
   - Reading base file filtered out say `Row X` because of filter push-down.
   - `Row X` had been updated in the log file and has an entry.
   - While merging we need some way to tell that `Row X` should be filtered out 
from log file as well, otherwise we may end up still returning it in the 
result, because based on the merging logic I see that any remaining rows in log 
file which did not have corresponding key in base file are just appended and 
returned in the result.




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

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to