You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2022/09/19 06:19:44 UTC

[GitHub] [hudi] xushiyan commented on a diff in pull request #6697: [HUDI-3478] Spark CDC Write

xushiyan commented on code in PR #6697:
URL: https://github.com/apache/hudi/pull/6697#discussion_r973881052


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java:
##########
@@ -102,6 +106,10 @@
   protected Map<String, HoodieRecord<T>> keyToNewRecords;
   protected Set<String> writtenRecordKeys;
   protected HoodieFileWriter<IndexedRecord> fileWriter;
+  // a flag that indicate whether allow the change data to write out a cdc log file.
+  protected boolean cdcEnabled = false;
+  // used to write cdc data
+  protected HoodieCDCLogger cdcLogger;

Review Comment:
   i think these 2 comments are not needed, the var names are quite self-explanatory. 



##########
hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java:
##########
@@ -229,6 +230,10 @@ protected synchronized void scanInternal(Option<KeySpec> keySpecOpt) {
         HoodieLogBlock logBlock = logFormatReaderWrapper.next();
         final String instantTime = logBlock.getLogBlockHeader().get(INSTANT_TIME);
         totalLogBlocks.incrementAndGet();
+        if (logBlock.getBlockType() == CDC_DATA_BLOCK) {
+          // hit a cdc block, just skip.
+          continue;

Review Comment:
   don't you want to skip based on the file suffix, when init logFormatReaderWrapper?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCDCLogger.java:
##########
@@ -0,0 +1,245 @@
+/*
+ * 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.io;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.generic.IndexedRecord;
+
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieAvroPayload;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.table.HoodieTableConfig;
+import org.apache.hudi.common.table.cdc.HoodieCDCOperation;
+import org.apache.hudi.common.table.cdc.HoodieCDCSupplementalLoggingMode;
+import org.apache.hudi.common.table.cdc.HoodieCDCUtils;
+import org.apache.hudi.common.table.log.AppendResult;
+import org.apache.hudi.common.table.log.HoodieLogFormat;
+import org.apache.hudi.common.table.log.block.HoodieCDCDataBlock;
+import org.apache.hudi.common.table.log.block.HoodieLogBlock;
+import org.apache.hudi.common.util.DefaultSizeEstimator;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.ExternalSpillableMap;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.exception.HoodieUpsertException;
+import org.apache.hudi.keygen.KeyGenUtils;
+import org.apache.hudi.keygen.KeyGenerator;
+import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.stream.Collectors;
+
+public class HoodieCDCLogger implements Closeable {
+
+  private final String commitTime;
+
+  private final String keyField;
+
+  private final Schema dataSchema;
+
+  private final boolean populateMetaFields;
+
+  private final KeyGenerator keyGenerator;
+
+  // writer for cdc data
+  private final HoodieLogFormat.Writer cdcWriter;
+
+  private final boolean cdcEnabled;
+
+  private final HoodieCDCSupplementalLoggingMode cdcSupplementalLoggingMode;
+
+  private final Schema cdcSchema;
+
+  // the cdc data
+  private final Map<String, HoodieAvroPayload> cdcData;
+
+  // the count of records currently being written, used to generate the same seqno for the cdc data
+  private final AtomicLong writtenRecordCount = new AtomicLong(-1);
+
+  public HoodieCDCLogger(
+      String commitTime,
+      HoodieWriteConfig config,
+      HoodieTableConfig tableConfig,
+      Schema schema,
+      HoodieLogFormat.Writer cdcWriter,
+      long maxInMemorySizeInBytes) {
+    try {
+      this.commitTime = commitTime;
+      this.dataSchema = HoodieAvroUtils.removeMetadataFields(schema);
+      this.populateMetaFields = config.populateMetaFields();
+      this.keyField = populateMetaFields ? HoodieRecord.RECORD_KEY_METADATA_FIELD
+          : tableConfig.getRecordKeyFieldProp();
+
+      TypedProperties props = new TypedProperties();
+      props.put(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key(), tableConfig.getKeyGeneratorClassName());
+      props.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), tableConfig.getRecordKeyFieldProp());
+      props.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), tableConfig.getPartitionFieldProp());
+      this.keyGenerator = KeyGenUtils.createKeyGeneratorByClassName(new TypedProperties(props));
+      this.cdcWriter = cdcWriter;
+
+      this.cdcEnabled = config.getBooleanOrDefault(HoodieTableConfig.CDC_ENABLED);
+      this.cdcSupplementalLoggingMode = HoodieCDCSupplementalLoggingMode.parse(
+          config.getStringOrDefault(HoodieTableConfig.CDC_SUPPLEMENTAL_LOGGING_MODE));
+
+      if (cdcSupplementalLoggingMode.equals(HoodieCDCSupplementalLoggingMode.WITH_BEFORE_AFTER)) {
+        this.cdcSchema = HoodieCDCUtils.CDC_SCHEMA;
+      } else if (cdcSupplementalLoggingMode.equals(HoodieCDCSupplementalLoggingMode.WITH_BEFORE)) {
+        this.cdcSchema = HoodieCDCUtils.CDC_SCHEMA_OP_RECORDKEY_BEFORE;
+      } else {
+        this.cdcSchema = HoodieCDCUtils.CDC_SCHEMA_OP_AND_RECORDKEY;
+      }
+
+      this.cdcData = new ExternalSpillableMap<>(
+          maxInMemorySizeInBytes,
+          config.getSpillableMapBasePath(),
+          new DefaultSizeEstimator<>(),
+          new DefaultSizeEstimator<>(),
+          config.getCommonConfig().getSpillableDiskMapType(),
+          config.getCommonConfig().isBitCaskDiskMapCompressionEnabled()
+      );
+    } catch (IOException e) {
+      throw new HoodieUpsertException("Failed to initialize HoodieCDCLogger", e);
+    }
+  }
+
+  public void put(HoodieRecord hoodieRecord, GenericRecord oldRecord, Option<IndexedRecord> indexedRecord) {

Review Comment:
   for readability
   ```suggestion
     public void put(HoodieRecord hoodieRecord, GenericRecord oldRecord, Option<IndexedRecord> newRecord) {
   ```



##########
hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCUtils.java:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.cdc;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecord;
+
+import org.apache.hudi.exception.HoodieException;
+
+public class HoodieCDCUtils {
+
+  public static final String CDC_LOGFILE_SUFFIX = "-cdc";
+
+  /* the `op` column represents how a record is changed. */
+  public static final String CDC_OPERATION_TYPE = "op";
+
+  /* the `ts_ms` column represents when a record is changed. */
+  public static final String CDC_COMMIT_TIMESTAMP = "ts_ms";

Review Comment:
   i think `ts` would be sufficient. this timestamp is based on timeline instant time, right? which may involve beyond ms granularity so we may want to be flexible here with the key



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java:
##########
@@ -416,9 +463,19 @@ public List<WriteStatus> close() {
         fileWriter = null;
       }
 
-      long fileSizeInBytes = FSUtils.getFileSize(fs, newFilePath);
-      HoodieWriteStat stat = writeStatus.getStat();
+      Option<AppendResult> cdcResult;
+      if (cdcLogger == null || recordsWritten == 0L || (recordsWritten == insertRecordsWritten)) {
+        // the following cases where we do not need to write out the cdc file:
+        // case 1: all the data from the previous file slice are deleted. and no new data is inserted;
+        // case 2: all the data are new-coming,
+        cdcResult = Option.empty();
+      } else {
+        cdcResult = cdcLogger.writeCDCData();
+      }

Review Comment:
   better to encapsulate these into CDCUtils or CDCLogger as a static helper? 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCDCLogger.java:
##########
@@ -0,0 +1,245 @@
+/*
+ * 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.io;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.generic.IndexedRecord;
+
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieAvroPayload;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.table.HoodieTableConfig;
+import org.apache.hudi.common.table.cdc.HoodieCDCOperation;
+import org.apache.hudi.common.table.cdc.HoodieCDCSupplementalLoggingMode;
+import org.apache.hudi.common.table.cdc.HoodieCDCUtils;
+import org.apache.hudi.common.table.log.AppendResult;
+import org.apache.hudi.common.table.log.HoodieLogFormat;
+import org.apache.hudi.common.table.log.block.HoodieCDCDataBlock;
+import org.apache.hudi.common.table.log.block.HoodieLogBlock;
+import org.apache.hudi.common.util.DefaultSizeEstimator;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.ExternalSpillableMap;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.exception.HoodieUpsertException;
+import org.apache.hudi.keygen.KeyGenUtils;
+import org.apache.hudi.keygen.KeyGenerator;
+import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.stream.Collectors;
+
+public class HoodieCDCLogger implements Closeable {
+
+  private final String commitTime;
+
+  private final String keyField;
+
+  private final Schema dataSchema;
+
+  private final boolean populateMetaFields;
+
+  private final KeyGenerator keyGenerator;
+
+  // writer for cdc data
+  private final HoodieLogFormat.Writer cdcWriter;
+
+  private final boolean cdcEnabled;
+
+  private final HoodieCDCSupplementalLoggingMode cdcSupplementalLoggingMode;
+
+  private final Schema cdcSchema;
+
+  // the cdc data
+  private final Map<String, HoodieAvroPayload> cdcData;
+
+  // the count of records currently being written, used to generate the same seqno for the cdc data
+  private final AtomicLong writtenRecordCount = new AtomicLong(-1);
+
+  public HoodieCDCLogger(
+      String commitTime,
+      HoodieWriteConfig config,
+      HoodieTableConfig tableConfig,
+      Schema schema,
+      HoodieLogFormat.Writer cdcWriter,
+      long maxInMemorySizeInBytes) {
+    try {
+      this.commitTime = commitTime;
+      this.dataSchema = HoodieAvroUtils.removeMetadataFields(schema);
+      this.populateMetaFields = config.populateMetaFields();
+      this.keyField = populateMetaFields ? HoodieRecord.RECORD_KEY_METADATA_FIELD
+          : tableConfig.getRecordKeyFieldProp();
+
+      TypedProperties props = new TypedProperties();
+      props.put(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key(), tableConfig.getKeyGeneratorClassName());
+      props.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), tableConfig.getRecordKeyFieldProp());
+      props.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), tableConfig.getPartitionFieldProp());
+      this.keyGenerator = KeyGenUtils.createKeyGeneratorByClassName(new TypedProperties(props));
+      this.cdcWriter = cdcWriter;
+
+      this.cdcEnabled = config.getBooleanOrDefault(HoodieTableConfig.CDC_ENABLED);
+      this.cdcSupplementalLoggingMode = HoodieCDCSupplementalLoggingMode.parse(
+          config.getStringOrDefault(HoodieTableConfig.CDC_SUPPLEMENTAL_LOGGING_MODE));
+
+      if (cdcSupplementalLoggingMode.equals(HoodieCDCSupplementalLoggingMode.WITH_BEFORE_AFTER)) {
+        this.cdcSchema = HoodieCDCUtils.CDC_SCHEMA;
+      } else if (cdcSupplementalLoggingMode.equals(HoodieCDCSupplementalLoggingMode.WITH_BEFORE)) {
+        this.cdcSchema = HoodieCDCUtils.CDC_SCHEMA_OP_RECORDKEY_BEFORE;
+      } else {
+        this.cdcSchema = HoodieCDCUtils.CDC_SCHEMA_OP_AND_RECORDKEY;
+      }
+
+      this.cdcData = new ExternalSpillableMap<>(
+          maxInMemorySizeInBytes,
+          config.getSpillableMapBasePath(),
+          new DefaultSizeEstimator<>(),
+          new DefaultSizeEstimator<>(),
+          config.getCommonConfig().getSpillableDiskMapType(),
+          config.getCommonConfig().isBitCaskDiskMapCompressionEnabled()
+      );
+    } catch (IOException e) {
+      throw new HoodieUpsertException("Failed to initialize HoodieCDCLogger", e);
+    }
+  }
+
+  public void put(HoodieRecord hoodieRecord, GenericRecord oldRecord, Option<IndexedRecord> indexedRecord) {
+    if (cdcEnabled) {
+      String recordKey;
+      if (oldRecord == null) {
+        recordKey = hoodieRecord.getRecordKey();
+      } else {
+        recordKey = this.keyGenerator.getKey(oldRecord).getRecordKey();
+      }

Review Comment:
   can we always rely on hoodieRecord to provide the key? maybe just need to validate old/new record has the same key. we should have a consistent source of truth for determining the key.



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCDCLogger.java:
##########
@@ -0,0 +1,245 @@
+/*
+ * 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.io;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.generic.IndexedRecord;
+
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieAvroPayload;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.table.HoodieTableConfig;
+import org.apache.hudi.common.table.cdc.HoodieCDCOperation;
+import org.apache.hudi.common.table.cdc.HoodieCDCSupplementalLoggingMode;
+import org.apache.hudi.common.table.cdc.HoodieCDCUtils;
+import org.apache.hudi.common.table.log.AppendResult;
+import org.apache.hudi.common.table.log.HoodieLogFormat;
+import org.apache.hudi.common.table.log.block.HoodieCDCDataBlock;
+import org.apache.hudi.common.table.log.block.HoodieLogBlock;
+import org.apache.hudi.common.util.DefaultSizeEstimator;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.ExternalSpillableMap;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.exception.HoodieUpsertException;
+import org.apache.hudi.keygen.KeyGenUtils;
+import org.apache.hudi.keygen.KeyGenerator;
+import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.stream.Collectors;
+
+public class HoodieCDCLogger implements Closeable {
+
+  private final String commitTime;
+
+  private final String keyField;
+
+  private final Schema dataSchema;
+
+  private final boolean populateMetaFields;
+
+  private final KeyGenerator keyGenerator;
+
+  // writer for cdc data
+  private final HoodieLogFormat.Writer cdcWriter;
+
+  private final boolean cdcEnabled;
+
+  private final HoodieCDCSupplementalLoggingMode cdcSupplementalLoggingMode;
+
+  private final Schema cdcSchema;
+
+  // the cdc data
+  private final Map<String, HoodieAvroPayload> cdcData;
+
+  // the count of records currently being written, used to generate the same seqno for the cdc data
+  private final AtomicLong writtenRecordCount = new AtomicLong(-1);
+
+  public HoodieCDCLogger(
+      String commitTime,
+      HoodieWriteConfig config,
+      HoodieTableConfig tableConfig,
+      Schema schema,
+      HoodieLogFormat.Writer cdcWriter,
+      long maxInMemorySizeInBytes) {
+    try {
+      this.commitTime = commitTime;
+      this.dataSchema = HoodieAvroUtils.removeMetadataFields(schema);
+      this.populateMetaFields = config.populateMetaFields();
+      this.keyField = populateMetaFields ? HoodieRecord.RECORD_KEY_METADATA_FIELD
+          : tableConfig.getRecordKeyFieldProp();
+
+      TypedProperties props = new TypedProperties();
+      props.put(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key(), tableConfig.getKeyGeneratorClassName());
+      props.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), tableConfig.getRecordKeyFieldProp());
+      props.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), tableConfig.getPartitionFieldProp());
+      this.keyGenerator = KeyGenUtils.createKeyGeneratorByClassName(new TypedProperties(props));
+      this.cdcWriter = cdcWriter;
+
+      this.cdcEnabled = config.getBooleanOrDefault(HoodieTableConfig.CDC_ENABLED);
+      this.cdcSupplementalLoggingMode = HoodieCDCSupplementalLoggingMode.parse(
+          config.getStringOrDefault(HoodieTableConfig.CDC_SUPPLEMENTAL_LOGGING_MODE));
+
+      if (cdcSupplementalLoggingMode.equals(HoodieCDCSupplementalLoggingMode.WITH_BEFORE_AFTER)) {
+        this.cdcSchema = HoodieCDCUtils.CDC_SCHEMA;
+      } else if (cdcSupplementalLoggingMode.equals(HoodieCDCSupplementalLoggingMode.WITH_BEFORE)) {
+        this.cdcSchema = HoodieCDCUtils.CDC_SCHEMA_OP_RECORDKEY_BEFORE;
+      } else {
+        this.cdcSchema = HoodieCDCUtils.CDC_SCHEMA_OP_AND_RECORDKEY;
+      }
+
+      this.cdcData = new ExternalSpillableMap<>(
+          maxInMemorySizeInBytes,
+          config.getSpillableMapBasePath(),
+          new DefaultSizeEstimator<>(),
+          new DefaultSizeEstimator<>(),
+          config.getCommonConfig().getSpillableDiskMapType(),
+          config.getCommonConfig().isBitCaskDiskMapCompressionEnabled()
+      );
+    } catch (IOException e) {
+      throw new HoodieUpsertException("Failed to initialize HoodieCDCLogger", e);
+    }
+  }
+
+  public void put(HoodieRecord hoodieRecord, GenericRecord oldRecord, Option<IndexedRecord> indexedRecord) {
+    if (cdcEnabled) {
+      String recordKey;
+      if (oldRecord == null) {
+        recordKey = hoodieRecord.getRecordKey();
+      } else {
+        recordKey = this.keyGenerator.getKey(oldRecord).getRecordKey();
+      }
+      GenericData.Record cdcRecord;
+      if (indexedRecord.isPresent()) {
+        GenericRecord record = (GenericRecord) indexedRecord.get();
+        if (oldRecord == null) {
+          // inserted cdc record
+          cdcRecord = createCDCRecord(HoodieCDCOperation.INSERT, recordKey,
+              null, record);
+        } else {
+          // updated cdc record
+          cdcRecord = createCDCRecord(HoodieCDCOperation.UPDATE, recordKey,
+              oldRecord, record);
+        }
+      } else {
+        // deleted cdc record
+        cdcRecord = createCDCRecord(HoodieCDCOperation.DELETE, recordKey,
+            oldRecord, null);
+      }
+      cdcData.put(recordKey, new HoodieAvroPayload(Option.of(cdcRecord)));
+    }
+  }
+
+  private GenericData.Record createCDCRecord(HoodieCDCOperation operation,
+                                             String recordKey,
+                                             GenericRecord oldRecord,
+                                             GenericRecord newRecord) {
+    GenericData.Record record;
+    if (cdcSupplementalLoggingMode.equals(HoodieCDCSupplementalLoggingMode.WITH_BEFORE_AFTER)) {
+      record = HoodieCDCUtils.cdcRecord(operation.getValue(), commitTime,
+          removeCommitMetadata(oldRecord), newRecord);
+    } else if (cdcSupplementalLoggingMode.equals(HoodieCDCSupplementalLoggingMode.WITH_BEFORE)) {
+      record = HoodieCDCUtils.cdcRecord(operation.getValue(), recordKey, removeCommitMetadata(oldRecord));
+    } else {
+      record = HoodieCDCUtils.cdcRecord(operation.getValue(), recordKey);
+    }
+    return record;
+  }
+
+  private GenericRecord removeCommitMetadata(GenericRecord record) {
+    if (record != null && populateMetaFields) {
+      GenericData.Record newRecord = new GenericData.Record(dataSchema);
+      for (Schema.Field field : dataSchema.getFields()) {
+        newRecord.put(field.name(), record.get(field.name()));
+      }
+      return newRecord;
+    }
+    return record;
+  }
+
+  public long getAndIncrement() {
+    return writtenRecordCount.getAndIncrement();
+  }
+
+  public boolean isEmpty() {
+    return !this.cdcEnabled || this.cdcData.isEmpty();
+  }
+
+  public Option<AppendResult> writeCDCData() {
+    if (isEmpty()) {
+      return Option.empty();
+    }
+    try {
+      Map<HoodieLogBlock.HeaderMetadataType, String> header = buildCDCBlockHeader();
+      List<IndexedRecord> records = cdcData.values().stream()
+          .map(record -> {
+            try {
+              return record.getInsertValue(cdcSchema).get();
+            } catch (IOException e) {
+              throw new HoodieIOException("Failed to get cdc record", e);
+            }
+          }).collect(Collectors.toList());
+      HoodieLogBlock block = new HoodieCDCDataBlock(records, header, keyField);
+      AppendResult result = cdcWriter.appendBlocks(Collections.singletonList(block));
+
+      // call close to trigger the data flush.
+      this.close();
+
+      return Option.of(result);
+    } catch (Exception e) {
+      throw new HoodieException("Failed to write the cdc data to " + cdcWriter.getLogFile().getPath(), e);
+    }
+  }
+
+  private Map<HoodieLogBlock.HeaderMetadataType, String> buildCDCBlockHeader() {
+    Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
+    header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, commitTime);
+    if (cdcSupplementalLoggingMode.equals(HoodieCDCSupplementalLoggingMode.WITH_BEFORE_AFTER)) {
+      header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, HoodieCDCUtils.CDC_SCHEMA_STRING);
+    } else if (cdcSupplementalLoggingMode.equals(HoodieCDCSupplementalLoggingMode.WITH_BEFORE)) {
+      header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, HoodieCDCUtils.CDC_SCHEMA_OP_RECORDKEY_BEFORE_STRING);
+    } else {
+      header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, HoodieCDCUtils.CDC_SCHEMA_OP_AND_RECORDKEY_STRING);
+    }
+    return header;
+  }
+
+  @Override
+  public void close() {
+    if (cdcWriter != null) {
+      try {
+        cdcWriter.close();
+      } catch (IOException e) {
+        throw new HoodieIOException("Failed to close HoodieCDCLogger", e);
+      }
+    }
+    cdcData.clear();

Review Comment:
   can we flush this before close cdcWriter? there is a chance this clear() not invoked



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org