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