YannByron commented on code in PR #5885:
URL: https://github.com/apache/hudi/pull/5885#discussion_r923039487


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/CDCRelation.scala:
##########
@@ -0,0 +1,414 @@
+/*
+ * 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.hadoop.fs.{FileSystem, Path}
+
+import org.apache.hudi.{AvroConversionUtils, DataSourceReadOptions, 
HoodieDataSourceHelper, HoodieTableSchema, SparkAdapterSupport}
+import org.apache.hudi.HoodieConversionUtils._
+import org.apache.hudi.common.table.cdc.CDCFileTypeEnum._
+import org.apache.hudi.common.table.cdc.CDCUtils._
+import org.apache.hudi.common.table.cdc.CDCOperationEnum._
+import org.apache.hudi.common.fs.FSUtils
+import org.apache.hudi.common.model.{FileSlice, HoodieBaseFile, 
HoodieCommitMetadata, HoodieFileFormat, HoodieFileGroupId, HoodieLogFile, 
HoodieReplaceCommitMetadata, HoodieWriteStat, WriteOperationType}
+import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline}
+import org.apache.hudi.common.table.timeline.HoodieTimeline._
+import org.apache.hudi.common.table.HoodieTableMetaClient
+import org.apache.hudi.common.table.TableSchemaResolver
+import org.apache.hudi.common.table.view.HoodieTableFileSystemView
+import org.apache.hudi.common.util.StringUtils
+import org.apache.hudi.exception.{HoodieException, HoodieNotSupportedException}
+import org.apache.hudi.internal.schema.InternalSchema
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.{Row, SQLContext, SparkSession}
+import org.apache.spark.sql.sources.{BaseRelation, Filter, PrunedFilteredScan}
+import org.apache.spark.sql.types.{StringType, StructField, StructType}
+import org.apache.spark.unsafe.types.UTF8String
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.util.{Failure, Success, Try}
+
+class CDCRelation(
+    override val sqlContext: SQLContext,
+    metaClient: HoodieTableMetaClient,
+    cdcSupplementalLogging: Boolean,
+    startInstant: String,
+    endInstant: String,
+    options: Map[String, String]
+) extends BaseRelation with PrunedFilteredScan with Logging {
+
+  val spark: SparkSession = sqlContext.sparkSession
+
+  val fs: FileSystem = metaClient.getFs.getFileSystem
+
+  val basePath: Path = metaClient.getBasePathV2
+
+  val (tableAvroSchema, _) = {
+    val schemaUtil = new TableSchemaResolver(metaClient)
+    val avroSchema = Try(schemaUtil.getTableAvroSchema) match {
+      case Success(schema) => schema
+      case Failure(e) =>
+        throw new IllegalArgumentException("Failed to fetch schema from the 
table", e)
+    }
+    // try to find internalSchema
+    val internalSchemaFromMeta = try {
+      
schemaUtil.getTableInternalSchemaFromCommitMetadata.orElse(InternalSchema.getEmptyInternalSchema)
+    } catch {
+      case _: Exception => InternalSchema.getEmptyInternalSchema
+    }
+    (avroSchema, internalSchemaFromMeta)
+  }
+
+  val tableStructSchema: StructType = 
AvroConversionUtils.convertAvroSchemaToStructType(tableAvroSchema)
+
+  val commits: Map[HoodieInstant, HoodieCommitMetadata] =
+    CDCRelation.getCompletedCommitInstantInSpecifiedRange(metaClient, 
startInstant, endInstant)
+
+  /**
+   * Parse the commit metadata between (startInstant, endInstant], and extract 
the touched partitions
+   * and files to build the filesystem view.
+   */
+  lazy val fsView: HoodieTableFileSystemView = {
+    val touchedPartition = commits.flatMap { case (_, commitMetadata) =>
+      val partitionSet = commitMetadata.getPartitionToWriteStats.keySet()
+      val replacedPartitionSet = commitMetadata match {
+        case replaceCommitMetadata: HoodieReplaceCommitMetadata =>
+          replaceCommitMetadata.getPartitionToReplaceFileIds.keySet().asScala
+        case _ => Set.empty[String]
+      }
+      partitionSet.asScala ++ replacedPartitionSet
+    }.toSet
+    val touchedFiles = touchedPartition.flatMap { partition =>
+      val partitionPath = FSUtils.getPartitionPath(basePath, partition)
+      fs.listStatus(partitionPath)
+    }.toArray
+    new HoodieTableFileSystemView(metaClient, 
metaClient.getCommitsTimeline.filterCompletedInstants, touchedFiles)
+  }
+
+  /**
+   * At the granularity of a file group, trace the mapping between each 
commit/instant and changes to this file group.
+   */
+  val changeFilesForPerFileGroupAndCommit: Map[HoodieFileGroupId, 
HoodieCDCFileGroupSplit] = {
+    val fgToCommitChanges = mutable.Map.empty[HoodieFileGroupId,
+      mutable.Map[HoodieInstant, ChangeFileForSingleFileGroupAndCommit]]
+
+    commits.foreach {
+      case (instant, commitMetadata) =>
+        // parse `partitionToWriteStats` in the metadata of commit
+        commitMetadata.getPartitionToWriteStats.asScala.foreach {
+          case (partition, hoodieWriteStats) =>
+            hoodieWriteStats.asScala.foreach { writeStat =>
+              val fileGroupId = new HoodieFileGroupId(partition, 
writeStat.getFileId)
+              // Identify the CDC source involved in this commit and
+              // determine its type for subsequent loading using different 
methods.
+              val changeFile = parseWriteStat(fileGroupId, instant, writeStat,
+                commitMetadata.getOperationType == WriteOperationType.DELETE)
+              if (fgToCommitChanges.contains(fileGroupId)) {
+                fgToCommitChanges(fileGroupId)(instant) = changeFile
+              } else {
+                fgToCommitChanges.put(fileGroupId, mutable.Map(instant -> 
changeFile))
+              }
+            }
+        }
+
+        // parse `partitionToReplaceFileIds` in the metadata of commit
+        commitMetadata match {
+          case replaceCommitMetadata: HoodieReplaceCommitMetadata =>
+            replaceCommitMetadata.getPartitionToReplaceFileIds.asScala.foreach 
{
+              case (partition, fileIds) =>
+                fileIds.asScala.foreach { fileId =>
+                  toScalaOption(fsView.fetchLatestFileSlice(partition, 
fileId)).foreach {
+                    fileSlice =>
+                      val fileGroupId = new HoodieFileGroupId(partition, 
fileId)
+                      val changeFile =
+                        
ChangeFileForSingleFileGroupAndCommit(REPLACED_FILE_GROUP, null, 
Some(fileSlice))
+                      if (fgToCommitChanges.contains(fileGroupId)) {
+                        fgToCommitChanges(fileGroupId)(instant) = changeFile
+                      } else {
+                        fgToCommitChanges.put(fileGroupId, mutable.Map(instant 
-> changeFile))
+                      }
+                  }
+                }
+            }
+          case _ =>
+        }
+      case _ =>
+    }
+    fgToCommitChanges.map { case (fgId, instantToChanges) =>
+      (fgId, HoodieCDCFileGroupSplit(instantToChanges.toArray.sortBy(_._1)))
+    }.toMap
+  }
+
+  override final def needConversion: Boolean = false
+
+  override def schema: StructType = CDCRelation.CDC_SPARK_SCHEMA
+
+  override def buildScan(requiredColumns: Array[String], filters: 
Array[Filter]): RDD[Row] = {
+    val internalRows = buildScan0(requiredColumns, filters)
+    internalRows.asInstanceOf[RDD[Row]]
+  }
+
+  def buildScan0(requiredColumns: Array[String], filters: Array[Filter]): 
RDD[InternalRow] = {
+    val nameToField = schema.fields.map(f => f.name -> f).toMap
+    val requiredSchema = StructType(requiredColumns.map(nameToField))
+    val originTableSchema = HoodieTableSchema(tableStructSchema, 
tableAvroSchema.toString)
+    val parquetReader = HoodieDataSourceHelper.buildHoodieParquetReader(
+      sparkSession = spark,
+      dataSchema = tableStructSchema,
+      partitionSchema = StructType(Nil),
+      requiredSchema = tableStructSchema,
+      filters = Nil,
+      options = options,
+      hadoopConf = spark.sessionState.newHadoopConf()
+    )
+    val cdcRdd = new HoodieCDCRDD(
+      spark,
+      metaClient,
+      cdcSupplementalLogging,
+      parquetReader,
+      originTableSchema,
+      schema,
+      requiredSchema,
+      changeFilesForPerFileGroupAndCommit.values.toArray
+    )
+    cdcRdd.asInstanceOf[RDD[InternalRow]]
+  }
+
+  /**
+   * Parse HoodieWriteStat, judge which type the file is, and what strategy 
should be used to parse CDC data.
+   * Then build a [[ChangeFileForSingleFileGroupAndCommit]] object.
+   */
+  private def parseWriteStat(

Review Comment:
   yes. let me move this to the common place.



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