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


##########
hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/Spark3ParquetSchemaEvolutionUtils.scala:
##########
@@ -0,0 +1,194 @@
+/*
+ * 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
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hudi.client.utils.SparkInternalSchemaConverter
+import org.apache.hudi.common.fs.FSUtils
+import org.apache.hudi.common.util
+import org.apache.hudi.common.util.InternalSchemaCache
+import org.apache.hudi.common.util.StringUtils.isNullOrEmpty
+import org.apache.hudi.common.util.collection.Pair
+import org.apache.hudi.internal.schema.InternalSchema
+import org.apache.hudi.internal.schema.action.InternalSchemaMerger
+import org.apache.hudi.internal.schema.utils.{InternalSchemaUtils, SerDeHelper}
+import org.apache.parquet.hadoop.metadata.FileMetaData
+import 
org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Cast, 
UnsafeProjection}
+import 
org.apache.spark.sql.execution.datasources.Spark3ParquetSchemaEvolutionUtils.pruneInternalSchema
+import 
org.apache.spark.sql.execution.datasources.parquet.{HoodieParquetFileFormatHelper,
 ParquetReadSupport}
+import org.apache.spark.sql.sources._
+import org.apache.spark.sql.types.{AtomicType, DataType, StructField, 
StructType}
+
+import scala.collection.convert.ImplicitConversions.`collection 
AsScalaIterable`
+
+abstract class Spark3ParquetSchemaEvolutionUtils(sharedConf: Configuration,

Review Comment:
   Thoughts for follow-ups in separate PRs.  I see the schema evolution related 
logic is invoked per reader/file, but I think part of the logic is based on the 
information at the table level, e.g., the internal schema of the table.  Is it 
possible to pass in such info to the reader instead of deriving them per file 
group?



##########
hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/Spark3ParquetSchemaEvolutionUtils.scala:
##########
@@ -0,0 +1,194 @@
+/*
+ * 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
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hudi.client.utils.SparkInternalSchemaConverter
+import org.apache.hudi.common.fs.FSUtils
+import org.apache.hudi.common.util
+import org.apache.hudi.common.util.InternalSchemaCache
+import org.apache.hudi.common.util.StringUtils.isNullOrEmpty
+import org.apache.hudi.common.util.collection.Pair
+import org.apache.hudi.internal.schema.InternalSchema
+import org.apache.hudi.internal.schema.action.InternalSchemaMerger
+import org.apache.hudi.internal.schema.utils.{InternalSchemaUtils, SerDeHelper}
+import org.apache.parquet.hadoop.metadata.FileMetaData
+import 
org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Cast, 
UnsafeProjection}
+import 
org.apache.spark.sql.execution.datasources.Spark3ParquetSchemaEvolutionUtils.pruneInternalSchema
+import 
org.apache.spark.sql.execution.datasources.parquet.{HoodieParquetFileFormatHelper,
 ParquetReadSupport}
+import org.apache.spark.sql.sources._
+import org.apache.spark.sql.types.{AtomicType, DataType, StructField, 
StructType}
+
+import scala.collection.convert.ImplicitConversions.`collection 
AsScalaIterable`
+
+abstract class Spark3ParquetSchemaEvolutionUtils(sharedConf: Configuration,
+                                                 filePath: Path,
+                                                 requiredSchema: StructType,
+                                                 partitionSchema: StructType) {
+  // Fetch internal schema
+  private lazy val internalSchemaStr: String = 
sharedConf.get(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA)
+
+  private lazy val querySchemaOption: util.Option[InternalSchema] = 
pruneInternalSchema(internalSchemaStr, requiredSchema)
+
+  var shouldUseInternalSchema: Boolean = !isNullOrEmpty(internalSchemaStr) && 
querySchemaOption.isPresent

Review Comment:
   Original logic is:
   ```
         // Fetch internal schema
         val internalSchemaStr = 
sharedConf.get(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA)
         // Internal schema has to be pruned at this point
         val querySchemaOption = SerDeHelper.fromJson(internalSchemaStr)
   
         var shouldUseInternalSchema = !isNullOrEmpty(internalSchemaStr) && 
querySchemaOption.isPresent
   ```
   `querySchemaOption.isPresent` always triggers the lazy evaluation of 
`querySchemaOption` here.  Should the original logic be used?



-- 
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