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:
[email protected]


Reply via email to