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


##########
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:
   yes, here can use `hoodieRecord.getRecordKey` directly. `HoodieMergeHandle` 
guarantees that the old/new record has the same key.



-- 
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-unsubscr...@hudi.apache.org

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

Reply via email to