yihua commented on code in PR #9276:
URL: https://github.com/apache/hudi/pull/9276#discussion_r1284854120


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/LegacyHoodieParquetFileFormat.scala:
##########
@@ -23,12 +23,12 @@ import org.apache.hudi.{DataSourceReadOptions, 
HoodieSparkUtils, SparkAdapterSup
 import org.apache.spark.sql.SparkSession
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.execution.datasources.PartitionedFile
-import 
org.apache.spark.sql.execution.datasources.parquet.HoodieParquetFileFormat.FILE_FORMAT_ID
+import 
org.apache.spark.sql.execution.datasources.parquet.LegacyHoodieParquetFileFormat.FILE_FORMAT_ID
 import org.apache.spark.sql.sources.Filter
 import org.apache.spark.sql.types.{AtomicType, StructType}
 
 
-class HoodieParquetFileFormat extends ParquetFileFormat with 
SparkAdapterSupport {
+class LegacyHoodieParquetFileFormat extends ParquetFileFormat with 
SparkAdapterSupport {

Review Comment:
   Add docs here to link to the new file format implementation so that any 
changes to this format implementation should also reflect in the new file 
format class?



##########
hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24LegacyHoodieParquetFileFormat.scala:
##########
@@ -50,7 +50,7 @@ import java.net.URI
  *   <li>Avoiding appending partition values to the rows read from the data 
file</li>
  * </ol>
  */
-class Spark24HoodieParquetFileFormat(private val shouldAppendPartitionValues: 
Boolean) extends ParquetFileFormat {
+class Spark24LegacyHoodieParquetFileFormat(private val 
shouldAppendPartitionValues: Boolean) extends ParquetFileFormat {

Review Comment:
   Same here for version-specific file format classes: add docs here to link to 
the new file format implementation so that any changes to this format 
implementation should also reflect in the new file format class?



##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala:
##########
@@ -165,7 +165,11 @@ trait SparkAdapter extends Serializable {
   /**
    * Create instance of [[ParquetFileFormat]]
    */
-  def createHoodieParquetFileFormat(appendPartitionValues: Boolean): 
Option[ParquetFileFormat]
+  def createLegacyHoodieParquetFileFormat(appendPartitionValues: Boolean): 
Option[ParquetFileFormat]
+
+  def getFilePath(file: PartitionedFile): Path

Review Comment:
   There is an existing API with the same functionality: 
`HoodieSparkPartitionedFileUtils.getPathFromPartitionedFile`



##########
hudi-spark-datasource/hudi-spark-common/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister:
##########
@@ -17,4 +17,4 @@
 
 
 org.apache.hudi.DefaultSource
-org.apache.spark.sql.execution.datasources.parquet.HoodieParquetFileFormat
\ No newline at end of file
+org.apache.spark.sql.execution.datasources.parquet.LegacyHoodieParquetFileFormat

Review Comment:
   Just curious, I don't have a clear answer.  Since `createRelation` is 
overridden so functionality-wise it's ok.



##########
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:
   Got it.  What I meant is, the new file format should not extend existing 
file format classes or use relation inside directly.  Util methods can still be 
extracted to a common util class so that both `NewHoodieParquetFileFormatUtils` 
and `HoodieBaseRelation` can use this new common util class.  If that takes 
time, we can punt 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.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to