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


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala:
##########
@@ -141,12 +145,37 @@ class 
HoodieFileGroupReaderBasedParquetFileFormat(tableState: HoodieTableState,
               case _ => baseFileReader(file)
             }
           }
+        // CDC queries.
+        case hoodiePartitionCDCFileGroupSliceMapping: 
HoodiePartitionCDCFileGroupMapping =>
+          val filePath: Path = 
sparkAdapter.getSparkPartitionedFileUtils.getPathFromPartitionedFile(file)
+          val fileGroupId: HoodieFileGroupId = new 
HoodieFileGroupId(filePath.getParent.toString, filePath.getName)
+          val fileSplits = 
hoodiePartitionCDCFileGroupSliceMapping.getFileSplitsFor(fileGroupId).get.toArray
+          val fileGroupSplit: HoodieCDCFileGroupSplit = 
HoodieCDCFileGroupSplit(fileSplits)
+          buildCDCRecordIterator(fileGroupSplit, preMergeBaseFileReader, 
hadoopConf, requiredSchema, props)
         // TODO: Use FileGroupReader here: HUDI-6942.
         case _ => baseFileReader(file)
       }
     }
   }
 
+  protected def buildCDCRecordIterator(cdcFileGroupSplit: 
HoodieCDCFileGroupSplit,
+                                       preMergeBaseFileReader: PartitionedFile 
=> Iterator[InternalRow],
+                                       hadoopConf: Configuration,
+                                       requiredSchema: StructType,
+                                       props: TypedProperties): 
Iterator[InternalRow] = {
+    val metaClient = 
HoodieTableMetaClient.initTableAndGetMetaClient(hadoopConf, 
tableState.tablePath, props)
+    val cdcSchema = CDCRelation.FULL_CDC_SPARK_SCHEMA
+    new CDCFileGroupIterator(

Review Comment:
   This does not seem to leverage `HoodieFileGroupReader`.



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala:
##########
@@ -119,6 +120,35 @@ case class MergeOnReadIncrementalRelation(override val 
sqlContext: SQLContext,
     }
   }
 
+  def listFileSplits(partitionFilters: Seq[Expression], dataFilters: 
Seq[Expression]): Map[InternalRow, Seq[FileSlice]] = {

Review Comment:
   Could this be extracted out as a util method instead of sitting inside the 
MOR incremental relation, which will not be used by the new Hudi parquet file 
format class?



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/CDCFileGroupIterator.scala:
##########
@@ -0,0 +1,558 @@
+/*
+ * 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.cdc
+
+import org.apache.avro.Schema
+import org.apache.avro.generic.{GenericData, GenericRecord, IndexedRecord}
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hudi.HoodieBaseRelation.BaseFileReader
+import org.apache.hudi.HoodieConversionUtils.toScalaOption
+import org.apache.hudi.HoodieDataSourceHelper.AvroDeserializerSupport
+import org.apache.hudi.avro.HoodieAvroUtils
+import org.apache.hudi.{AvroConversionUtils, AvroProjection, HoodieFileIndex, 
HoodieMergeOnReadFileSplit, HoodieTableSchema, HoodieTableState, 
LogFileIterator, RecordMergingFileIterator, SparkAdapterSupport}
+import org.apache.hudi.common.config.{HoodieMetadataConfig, TypedProperties}
+import org.apache.hudi.common.model.{FileSlice, HoodieAvroRecordMerger, 
HoodieLogFile, HoodieRecord, HoodieRecordMerger, HoodieRecordPayload}
+import org.apache.hudi.common.table.HoodieTableMetaClient
+import org.apache.hudi.common.table.cdc.{HoodieCDCFileSplit, HoodieCDCUtils}
+import org.apache.hudi.common.table.cdc.HoodieCDCInferenceCase._
+import org.apache.hudi.common.table.log.HoodieCDCLogRecordIterator
+import org.apache.hudi.common.table.cdc.HoodieCDCOperation._
+import org.apache.hudi.common.table.cdc.HoodieCDCSupplementalLoggingMode._
+import org.apache.hudi.common.util.ValidationUtils.checkState
+import org.apache.hudi.config.HoodiePayloadConfig
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory
+import 
org.apache.spark.sql.HoodieCatalystExpressionUtils.generateUnsafeProjection
+import org.apache.spark.sql.avro.HoodieAvroDeserializer
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Projection
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.unsafe.types.UTF8String
+
+import java.io.Closeable
+import java.util.Properties
+import java.util.stream.Collectors
+import scala.annotation.tailrec
+import scala.collection.mutable
+import scala.jdk.CollectionConverters.asScalaBufferConverter
+
+class CDCFileGroupIterator(split: HoodieCDCFileGroupSplit,

Review Comment:
   This InternalRow iterator for CDC still contains a lot of specific logic 
pertaining to Hudi storage layout, which should be common across multiple 
engines.  Right now the logic cannot be reused by other engines like Flink and 
Hive.  Could such logic be pushed into `HoodieFileGroupReader` so that it can 
be shared across engines?



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodiePartitionCDCFileGroupMapping.scala:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.common.model.HoodieFileGroupId
+import org.apache.hudi.common.table.cdc.HoodieCDCFileSplit
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.util.{ArrayData, MapData}
+import org.apache.spark.sql.types.{DataType, Decimal}
+import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
+
+import java.util
+
+case class HoodiePartitionCDCFileGroupMapping(partitionValues: InternalRow,

Review Comment:
   This class shares same amount of code with `PartitionFileSliceMapping`.  
Could you create a common parent class?



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