danny0405 commented on code in PR #6727:
URL: https://github.com/apache/hudi/pull/6727#discussion_r979159411


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/CDCRelation.scala:
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.hudi.AvroConversionUtils
+import org.apache.hudi.DataSourceReadOptions
+import org.apache.hudi.HoodieDataSourceHelper
+import org.apache.hudi.HoodieTableSchema
+import org.apache.hudi.common.table.cdc.HoodieCDCUtils._
+import org.apache.hudi.common.table.cdc.HoodieCDCOperation._
+import org.apache.hudi.common.table.HoodieTableMetaClient
+import org.apache.hudi.common.table.TableSchemaResolver
+import org.apache.hudi.common.table.cdc.HoodieCDCExtractor
+import org.apache.hudi.exception.HoodieException
+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.util.{Failure, Success, Try}
+
+class CDCRelation(
+    override val sqlContext: SQLContext,
+    metaClient: HoodieTableMetaClient,
+    startInstant: String,
+    endInstant: String,
+    options: Map[String, String]
+) extends BaseRelation with PrunedFilteredScan with Logging {
+
+  val spark: SparkSession = sqlContext.sparkSession
+
+  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 cdcExtractor: HoodieCDCExtractor = new HoodieCDCExtractor(metaClient, 
startInstant, endInstant)
+
+  override final def needConversion: Boolean = false
+
+  override def schema: StructType = CDCRelation.FULL_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 changes = cdcExtractor.extractCDCFileSplits().values().asScala.map { 
pairs =>
+      HoodieCDCFileGroupSplit(
+        pairs.asScala.map(pair => (pair.getLeft, 
pair.getRight)).sortBy(_._1).toArray
+      )
+    }
+    val cdcRdd = new HoodieCDCRDD(
+      spark,
+      metaClient,
+      parquetReader,
+      originTableSchema,
+      schema,
+      requiredSchema,
+      changes.toArray
+    )
+    cdcRdd.asInstanceOf[RDD[InternalRow]]
+  }
+}
+
+object CDCRelation {
+
+  val CDC_OPERATION_DELETE: UTF8String = UTF8String.fromString(DELETE.getValue)
+  val CDC_OPERATION_INSERT: UTF8String = UTF8String.fromString(INSERT.getValue)
+  val CDC_OPERATION_UPDATE: UTF8String = UTF8String.fromString(UPDATE.getValue)
+
+  /**
+   * CDC Schema For Spark.
+   * Also it's schema when `hoodie.table.cdc.supplemental.logging.mode` is 
`cdc_data_before_after`.
+   * Here we use the debezium format.
+   */
+  val FULL_CDC_SPARK_SCHEMA: StructType = {
+    StructType(

Review Comment:
   Do we need to expose the schema as debezium for reader internal ? Why not 
reuse the field `_hoodie_operaton` which is a hoodie format.
   
   Curious how downstream pipeline handle these records ? For SQL users, they 
declare the table schema, for example with fields (a, b, c, d), now you return 
RDD with schema of avro, how and when to deserialize them into (a, b, c ,d) 
then ?



##########
hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCUtils.java:
##########
@@ -103,7 +103,7 @@ private static Schema createCDCSchema(Schema tableSchema, 
boolean withAfterImage
       fields = Arrays.asList(opField, keyField, beforeField);
     }
 
-    Schema mergedSchema = Schema.createRecord("CDC", null, 
tableSchema.getNamespace(), false);
+    Schema mergedSchema = Schema.createRecord("CDC", null, "", false);
     mergedSchema.setFields(fields);

Review Comment:
   Why this change ?



##########
hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieCDCLogRecordIterator.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.common.table.log;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import org.apache.hudi.common.model.HoodieLogFile;
+import org.apache.hudi.common.table.log.block.HoodieDataBlock;
+import org.apache.hudi.common.table.log.block.HoodieLogBlock;
+import org.apache.hudi.common.util.ClosableIterator;
+import org.apache.hudi.exception.HoodieIOException;
+
+import java.io.IOException;
+
+public class HoodieCDCLogRecordIterator implements 
ClosableIterator<IndexedRecord> {
+
+  private final HoodieLogFile cdcLogFile;
+
+  private final HoodieLogFormat.Reader reader;
+
+  private ClosableIterator<IndexedRecord> itr;
+
+  public HoodieCDCLogRecordIterator(
+      FileSystem fs,
+      Path cdcLogPath,
+      Schema cdcSchema) throws IOException {
+    this.cdcLogFile = new HoodieLogFile(fs.getFileStatus(cdcLogPath));
+    this.reader = new HoodieLogFileReader(fs, cdcLogFile, cdcSchema,
+        HoodieLogFileReader.DEFAULT_BUFFER_SIZE, false);
+  }
+
+  @Override
+  public boolean hasNext() {
+    if (itr == null || !itr.hasNext()) {
+      if (reader.hasNext()) {
+        HoodieDataBlock dataBlock = (HoodieDataBlock) reader.next();
+        if (dataBlock.getBlockType() == 
HoodieLogBlock.HoodieLogBlockType.CDC_DATA_BLOCK) {
+          itr = dataBlock.getRecordIterator();

Review Comment:
   Are there other data blocks here ? If not, we can remove this check for 
efficiency.



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