alexeykudinkin commented on code in PR #6442:
URL: https://github.com/apache/hudi/pull/6442#discussion_r984843702


##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieCatalystPlansUtils.scala:
##########
@@ -90,4 +90,9 @@ trait HoodieCatalystPlansUtils {
   def createInsertInto(table: LogicalPlan, partition: Map[String, 
Option[String]],
                        query: LogicalPlan, overwrite: Boolean, 
ifPartitionNotExists: Boolean): LogicalPlan
 
+  /**
+   * Create Like expression.
+   */
+  def createLike(left: Expression, right: Expression): Expression

Review Comment:
   What do we need this one for?



##########
hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32PlusHoodieParquetFileFormat.scala:
##########
@@ -402,7 +402,7 @@ object Spark32PlusHoodieParquetFileFormat {
   /**
    * NOTE: This method is specific to Spark 3.2.0
    */
-  private def createParquetFilters(args: Any*): ParquetFilters = {
+  def createParquetFilters(args: Any*): ParquetFilters = {

Review Comment:
   Why are we down-grading these?



##########
hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/source/HoodieBatchScanBuilder.scala:
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.hudi.source
+
+import org.apache.hudi.DataSourceReadOptions.QUERY_TYPE
+import org.apache.hudi.exception.HoodieException
+import org.apache.hudi.{DataSourceOptionsHelper, DefaultSource, 
HoodieSparkUtils}
+import org.apache.spark.sql.{SQLContext, SparkSession}
+import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, 
SupportsPushDownFilters, SupportsPushDownRequiredColumns, 
SupportsRuntimeFiltering}
+import org.apache.spark.sql.execution.datasources.HadoopFsRelation
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.StructType
+
+
+class HoodieBatchScanBuilder(spark: SparkSession,
+                             hoodieCatalogTable: HoodieCatalogTable,
+                             options: Map[String, String])
+    extends ScanBuilder with SupportsPushDownFilters with 
SupportsPushDownRequiredColumns {
+  @transient lazy val hadoopConf = {
+    // Hadoop Configurations are case sensitive.
+    spark.sessionState.newHadoopConfWithOptions(options)
+  }
+
+  private var filterExpressions: Option[Expression] = None
+
+  private var filterArrays: Array[Filter] = Array.empty
+
+  private var expectedSchema: StructType= hoodieCatalogTable.tableSchema
+
+  override def build(): Scan = {
+    val relation = new DefaultSource().createRelation(new SQLContext(spark), 
options)
+    relation match {
+      case HadoopFsRelation(location, partitionSchema, dataSchema, _, _, 
options) =>

Review Comment:
   Whole idea of V2 integration is bypassing the V1 concepts that such as 
`HadoopFsRelation` which unfortunately are very limiting for Hudi features.
   
   What's our plan here?



##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala:
##########
@@ -63,6 +66,103 @@ object HoodieSparkUtils extends SparkAdapterSupport with 
SparkVersionsSupport {
     }))
   }
 
+  /**
+   * Convert Filters to Catalyst Expressions and joined by And. If convert 
success return an
+   * Non-Empty Option[Expression],or else return None.
+   */
+  def convertToCatalystExpressions(filters: Seq[Filter],
+                                   tableSchema: StructType): 
Seq[Option[Expression]] = {
+    filters.map(convertToCatalystExpression(_, tableSchema))
+  }
+
+
+  /**
+   * Convert Filters to Catalyst Expressions and joined by And. If convert 
success return an
+   * Non-Empty Option[Expression],or else return None.
+   */
+  def convertToCatalystExpression(filters: Array[Filter],
+                                  tableSchema: StructType): Option[Expression] 
= {
+    val expressions = convertToCatalystExpressions(filters, tableSchema)
+    if (expressions.forall(p => p.isDefined)) {
+      if (expressions.isEmpty) {
+        None
+      } else if (expressions.length == 1) {
+        expressions.head
+      } else {
+        
Some(expressions.map(_.get).reduce(org.apache.spark.sql.catalyst.expressions.And))
+      }
+    } else {
+      None
+    }
+  }
+
+  /**
+   * Convert Filter to Catalyst Expression. If convert success return an 
Non-Empty
+   * Option[Expression],or else return None.
+   */
+  def convertToCatalystExpression(filter: Filter, tableSchema: StructType): 
Option[Expression] = {

Review Comment:
   Let's make sure we're not duplicating utilities we already have (there's 
HoodieCatalystExpressionUtils doing exactly the same thing)



##########
hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/source/SparkBatch.scala:
##########
@@ -0,0 +1,351 @@
+/*
+ * 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.hudi.source
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapred.FileSplit
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
+import org.apache.hadoop.mapreduce.{JobID, RecordReader, TaskAttemptID, 
TaskID, TaskType}
+import org.apache.hudi.HoodieSparkUtils
+import org.apache.hudi.client.utils.SparkInternalSchemaConverter
+import org.apache.hudi.common.fs.FSUtils
+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.action.InternalSchemaMerger
+import org.apache.hudi.internal.schema.utils.SerDeHelper
+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.{ParquetInputFormat, ParquetRecordReader}
+import org.apache.spark.TaskContext
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.connector.read.{Batch, InputPartition, 
PartitionReader, PartitionReaderFactory}
+import org.apache.spark.sql.execution.PartitionedFileUtil
+import org.apache.spark.sql.execution.datasources.parquet.{ParquetFilters, 
ParquetFooterReader, ParquetOptions, ParquetReadSupport, ParquetWriteSupport, 
Spark32PlusDataSourceUtils, Spark32PlusHoodieVectorizedParquetRecordReader, 
VectorizedParquetRecordReader}
+import 
org.apache.spark.sql.execution.datasources.parquet.Spark32PlusHoodieParquetFileFormat.{createParquetFilters,
 createParquetReadSupport, createVectorizedParquetRecordReader, 
pruneInternalSchema, rebuildFilterFromParquet}
+import 
org.apache.spark.sql.execution.datasources.v2.{FilePartitionReaderFactory, 
PartitionReaderWithPartitionValues}
+import org.apache.spark.sql.execution.datasources.{DataSourceUtils, 
FilePartition, PartitionDirectory, PartitionedFile}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.{AtomicType, DataType, StructType}
+import org.apache.spark.util.SerializableConfiguration
+
+import java.net.URI
+
+class SparkBatch(spark: SparkSession,
+                 selectedPartitions: Seq[PartitionDirectory],
+                 partitionSchema: StructType,
+                 requiredSchema: StructType,
+                 filters: Seq[Filter],
+                 options: Map[String, String],
+                 @transient hadoopConf: Configuration) extends Batch with 
Serializable {
+
+  override def planInputPartitions(): Array[InputPartition] = {
+    // TODO support more accurate task planning.
+    val maxSplitBytes = FilePartition.maxSplitBytes(spark, selectedPartitions)
+    val splitFiles = selectedPartitions.flatMap { partition =>
+      partition.files.flatMap { file =>
+        val filePath = file.getPath
+        PartitionedFileUtil.splitFiles(
+          sparkSession = spark,
+          file = file,
+          filePath = filePath,
+          isSplitable = true,
+          maxSplitBytes = maxSplitBytes,
+          partitionValues = partition.values
+        )
+      }.toArray.sortBy(_.length)(implicitly[Ordering[Long]].reverse)
+    }
+    FilePartition.getFilePartitions(spark, splitFiles, maxSplitBytes).toArray
+
+  }
+
+  override def createReaderFactory(): PartitionReaderFactory = {
+    hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, 
classOf[ParquetReadSupport].getName)
+    hadoopConf.set(
+      ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA,
+      requiredSchema.json)
+    hadoopConf.set(
+      ParquetWriteSupport.SPARK_ROW_SCHEMA,
+      requiredSchema.json)
+    hadoopConf.set(
+      SQLConf.SESSION_LOCAL_TIMEZONE.key,
+      spark.sessionState.conf.sessionLocalTimeZone)
+    hadoopConf.setBoolean(
+      SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.key,
+      spark.sessionState.conf.nestedSchemaPruningEnabled)
+    hadoopConf.setBoolean(
+      SQLConf.CASE_SENSITIVE.key,
+      spark.sessionState.conf.caseSensitiveAnalysis)
+
+    ParquetWriteSupport.setSchema(requiredSchema, hadoopConf)
+
+    // Sets flags for `ParquetToSparkSchemaConverter`
+    hadoopConf.setBoolean(
+      SQLConf.PARQUET_BINARY_AS_STRING.key,
+      spark.sessionState.conf.isParquetBinaryAsString)
+    hadoopConf.setBoolean(
+      SQLConf.PARQUET_INT96_AS_TIMESTAMP.key,
+      spark.sessionState.conf.isParquetINT96AsTimestamp)
+    val broadcastedConf = spark.sparkContext.broadcast(
+      new SerializableConfiguration(hadoopConf))
+    val sqlConf = spark.sessionState.conf
+    ReaderFactory(sqlConf, broadcastedConf)
+  }
+
+  case class ReaderFactory(sqlConf: SQLConf,
+                           broadcastedConf: 
Broadcast[SerializableConfiguration]) extends FilePartitionReaderFactory {
+    // TODO: if you move this into the closure it reverts to the default 
values.
+    // If true, enable using the custom RecordReader for parquet. This only 
works for
+    // a subset of the types (no complex types).
+    val resultSchema: StructType = StructType(partitionSchema.fields ++ 
requiredSchema.fields)

Review Comment:
   Do we really need to clone whole Hudi's custom ParquetFileFormat impl here?



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