yihua commented on code in PR #9276:
URL: https://github.com/apache/hudi/pull/9276#discussion_r1284800578
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/Iterators.scala:
##########
@@ -83,8 +94,8 @@ class LogFileIterator(logFiles: List[HoodieLogFile],
}
.getOrElse(new TypedProperties())
- protected override val avroSchema: Schema = new
Schema.Parser().parse(requiredSchema.avroSchemaStr)
- protected override val structTypeSchema: StructType =
requiredSchema.structTypeSchema
+ protected override val avroSchema: Schema = requiredAvroSchema
Review Comment:
Is this for performance improvement? Should `requiredSchema:
HoodieTableSchema` still be kept in the constructor for supporting schema
evolution in the future?
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/Iterators.scala:
##########
@@ -181,17 +194,30 @@ class RecordMergingFileIterator(logFiles:
List[HoodieLogFile],
baseFileIterator: Iterator[InternalRow],
readerSchema: StructType,
dataSchema: HoodieTableSchema,
- requiredSchema: HoodieTableSchema,
+ requiredStructTypeSchema: StructType,
+ requiredAvroSchema: Schema,
tableState: HoodieTableState,
config: Configuration)
- extends LogFileIterator(logFiles, partitionPath, dataSchema, requiredSchema,
tableState, config) {
+ extends LogFileIterator(logFiles, partitionPath, dataSchema,
requiredStructTypeSchema, requiredAvroSchema, tableState, config) {
+ def this(logFiles: List[HoodieLogFile],
+ partitionPath: Path,
+ baseFileIterator: Iterator[InternalRow],
+ readerSchema: StructType,
+ dataSchema: HoodieTableSchema,
+ requiredSchema: HoodieTableSchema,
+ tableState: HoodieTableState,
+ config: Configuration) {
+ this(logFiles, partitionPath, baseFileIterator, readerSchema, dataSchema,
requiredSchema.structTypeSchema,
+ new Schema.Parser().parse(requiredSchema.avroSchemaStr), tableState,
config)
+ }
def this(split: HoodieMergeOnReadFileSplit, baseFileReader: BaseFileReader,
dataSchema: HoodieTableSchema,
requiredSchema: HoodieTableSchema, tableState: HoodieTableState,
config: Configuration) {
this(split.logFiles, getPartitionPath(split),
baseFileReader(split.dataFile.get),
baseFileReader.schema, dataSchema, requiredSchema, tableState, config)
}
+
Review Comment:
nit: remove empty line
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala:
##########
@@ -99,7 +96,7 @@ class SparkHoodieTableFileIndex(spark: SparkSession,
AvroConversionUtils.convertAvroSchemaToStructType(schemaUtil.getTableAvroSchema)
})
- protected lazy val shouldFastBootstrap =
configProperties.getBoolean(DATA_QUERIES_ONLY.key, false)
+ protected lazy val shouldFastBootstrap: Boolean =
configProperties.getBoolean(DATA_QUERIES_ONLY.key, false)
Review Comment:
nit: let's avoid cosmetic changes in this class.
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/NewHoodieParquetFileFormat.scala:
##########
@@ -0,0 +1,353 @@
+/*
+ * 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.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import
org.apache.hudi.DataSourceReadOptions.{REALTIME_PAYLOAD_COMBINE_OPT_VAL,
REALTIME_SKIP_MERGE_OPT_VAL}
+import org.apache.hudi.MergeOnReadSnapshotRelation.createPartitionedFile
+import org.apache.hudi.common.fs.FSUtils
+import org.apache.hudi.common.model.{BaseFile, FileSlice, HoodieLogFile,
HoodieRecord}
+import org.apache.hudi.common.util.ValidationUtils.checkState
+import org.apache.hudi.{HoodieBaseRelation, HoodieSparkUtils,
HoodieTableSchema, HoodieTableState, LogFileIterator,
MergeOnReadSnapshotRelation, PartitionFileSliceMapping,
RecordMergingFileIterator, SkipMergeIterator, SparkAdapterSupport}
+import org.apache.spark.broadcast.Broadcast
+import
org.apache.spark.sql.HoodieCatalystExpressionUtils.generateUnsafeProjection
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.JoinedRow
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.isMetaField
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.{StructField, StructType}
+import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch}
+import org.apache.spark.util.SerializableConfiguration
+
+import scala.collection.mutable
+import scala.jdk.CollectionConverters.asScalaIteratorConverter
+
+class NewHoodieParquetFileFormat(tableState: Broadcast[HoodieTableState],
Review Comment:
Could you add docs to this class to clarify what's new in this Hudi parquet
file format compared to the legacy one?
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/NewHoodieParquetFileFormatUtils.scala:
##########
@@ -0,0 +1,215 @@
+/*
+ * 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.avro.Schema
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapred.JobConf
+import org.apache.hudi.HoodieBaseRelation._
+import org.apache.hudi.HoodieConversionUtils.toScalaOption
+import org.apache.hudi.common.config.ConfigProperty
+import org.apache.hudi.common.fs.FSUtils
+import org.apache.hudi.common.model.HoodieRecord
+import
org.apache.hudi.common.model.HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE_PREFIX
+import org.apache.hudi.common.table.timeline.HoodieTimeline
+import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient,
TableSchemaResolver}
+import org.apache.hudi.common.util.ValidationUtils.checkState
+import org.apache.hudi.common.util.{ConfigUtils, StringUtils}
+import org.apache.hudi.config.HoodieBootstrapConfig.DATA_QUERIES_ONLY
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter
+import org.apache.hudi.internal.schema.{HoodieSchemaException, InternalSchema}
+import org.apache.spark.sql.catalyst.analysis.Resolver
+import
org.apache.spark.sql.execution.datasources.parquet.NewHoodieParquetFileFormat
+import org.apache.spark.sql.execution.datasources.{FileStatusCache,
HadoopFsRelation}
+import org.apache.spark.sql.hudi.HoodieSqlCommonUtils
+import org.apache.spark.sql.sources.BaseRelation
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.{SQLContext, SparkSession}
+
+import scala.collection.JavaConverters._
+import scala.util.{Failure, Success, Try}
+
+class NewHoodieParquetFileFormatUtils(val sqlContext: SQLContext,
+ val metaClient: HoodieTableMetaClient,
+ val optParamsInput: Map[String, String],
+ private val schemaSpec:
Option[StructType]) extends SparkAdapterSupport {
+ protected val sparkSession: SparkSession = sqlContext.sparkSession
+
+ protected val optParams: Map[String, String] = optParamsInput.filter(kv =>
!kv._1.equals(DATA_QUERIES_ONLY.key()))
+ protected def tableName: String = metaClient.getTableConfig.getTableName
+
+ protected lazy val resolver: Resolver =
sparkSession.sessionState.analyzer.resolver
+
+ private lazy val metaFieldNames =
HoodieRecord.HOODIE_META_COLUMNS.asScala.toSet
+
+ protected lazy val fileIndex: HoodieFileIndex =
+ HoodieFileIndex(sparkSession, metaClient, Some(tableStructSchema),
optParams, FileStatusCache.getOrCreate(sparkSession))
+
+ protected lazy val conf: Configuration = new
Configuration(sqlContext.sparkContext.hadoopConfiguration)
+ protected lazy val jobConf = new JobConf(conf)
+
+ protected lazy val tableConfig: HoodieTableConfig = metaClient.getTableConfig
+
+ protected lazy val basePath: Path = metaClient.getBasePathV2
+
+ protected lazy val (tableAvroSchema: Schema, internalSchemaOpt:
Option[InternalSchema]) = {
Review Comment:
Should some of the util methods also used by `HoodieBaseRelation` be
extracted to an independent Util class for code reuse?
--
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.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]