garyli1019 commented on a change in pull request #1848: URL: https://github.com/apache/hudi/pull/1848#discussion_r459864183
########## 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, Review comment: After thinking for a while, I think we can handle it this way: #### BaseFileOnly use the user-specified schema base file reader #### Unmerge use the user-specified schema base file reader Convert log record to `InternalRow` then extract the correct schema before exiting the `unMergeFileIterator` Or the other way around. #### Merge Use the full schema base file reader. Merge two records in Avro. Convert to InternalRow then extract the correct schema or the other way around. Since the `InternalRow` need the position to extract the value and the schema could be nested. This could get complicated once nested columns got involved. ---------------------------------------------------------------- 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]
