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


##########
hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroBinaryRecord.java:
##########
@@ -0,0 +1,333 @@
+/*
+ * 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.model;
+
+import org.apache.hudi.avro.HoodieAvroReaderContext;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.util.ConfigUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.OrderingValues;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.generic.IndexedRecord;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.Map;
+import java.util.Properties;
+
+import static 
org.apache.hudi.common.model.HoodieAvroIndexedRecord.updateMetadataValuesInternal;
+
+public class HoodieAvroBinaryRecord extends HoodieRecord<byte[]> {
+
+  public HoodieAvroBinaryRecord(HoodieKey key, byte[] data) {
+    super(key, data);
+  }
+
+  public HoodieAvroBinaryRecord(HoodieKey key, byte[] data, Comparable 
orderingValue) {
+    super(key, data);
+    this.orderingValue = orderingValue;
+  }
+
+  public HoodieAvroBinaryRecord(HoodieKey key, byte[] data, HoodieOperation 
operation) {
+    super(key, data, operation, Option.empty());
+  }
+
+  public HoodieAvroBinaryRecord(HoodieRecord<byte[]> record) {
+    super(record);
+  }
+
+  public HoodieAvroBinaryRecord(
+      HoodieKey key,
+      byte[] data,
+      HoodieOperation operation,
+      HoodieRecordLocation currentLocation,
+      HoodieRecordLocation newLocation) {
+    super(key, data, operation, currentLocation, newLocation);
+  }
+
+  public HoodieAvroBinaryRecord(
+      HoodieKey key,
+      byte[] data,
+      HoodieOperation operation,
+      Option<Map<String, String>> metaData) {
+    super(key, data, operation, metaData);
+  }
+
+  public HoodieAvroBinaryRecord() {
+
+  }
+
+  @Override
+  public HoodieRecord<byte[]> newInstance() {
+    return new HoodieAvroBinaryRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<byte[]> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieAvroBinaryRecord(key, data, op);
+  }
+
+  @Override
+  public HoodieRecord<byte[]> newInstance(HoodieKey key) {
+    return new HoodieAvroBinaryRecord(key, data);
+  }
+
+  @Override
+  public byte[] getData() {
+    if (data == null) {
+      throw new IllegalStateException("Payload already deflated for record.");
+    }
+    return data;
+  }
+
+  @Override
+  protected Comparable<?> doGetOrderingValue(Schema recordSchema, Properties 
props) {
+    if (orderingValue != null) {
+      return orderingValue;
+    }
+    String[] orderingFields = ConfigUtils.getOrderingFields(props);
+    if (orderingFields != null) {
+      boolean consistentLogicalTimestampEnabled = 
Boolean.parseBoolean(props.getProperty(
+          
KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(),
+          
KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue()));
+      orderingValue = OrderingValues.create(orderingFields, field -> {
+        GenericRecord avroRecord = null;
+        try {
+          avroRecord = HoodieAvroUtils.bytesToAvro(data, recordSchema);

Review Comment:
   we should convert the bytes to avro record in one shot instead of for each 
field.



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