the-other-tim-brown commented on code in PR #13136:
URL: https://github.com/apache/hudi/pull/13136#discussion_r2041259134


##########
hudi-hadoop-common/src/main/java/org/apache/hudi/common/util/HFileUtils.java:
##########
@@ -199,26 +199,26 @@ public byte[] serializeRecordsToLogBlock(HoodieStorage 
storage,
       }
 
       final byte[] recordBytes = serializeRecord(record, writerSchema, 
keyFieldName);
-      // If key exists in the map, append to its list. If not, create a new 
list.
-      // Get the existing list of recordBytes for the recordKey, or an empty 
list if it doesn't exist
-      List<byte[]> recordBytesList = sortedRecordsMap.getOrDefault(recordKey, 
new ArrayList<>());
-      recordBytesList.add(recordBytes);
-      // Put the updated list back into the map
-      sortedRecordsMap.put(recordKey, recordBytesList);
+      if (sortedRecordsMap.containsKey(recordKey)) {
+        LOG.error("Found duplicate record with recordKey: " + recordKey);
+        printRecord("Previous record", sortedRecordsMap.get(recordKey), 
writerSchema);
+        printRecord("Current record", recordBytes, writerSchema);
+        throw new HoodieException(String.format("Writing multiple records with 
same key %s not supported for %s",
+            recordKey, this.getClass().getName()));

Review Comment:
   `this.getClass().getName()` will be `HFileUtils`, should this say something 
else like "for record index"?



##########
hudi-hadoop-common/src/main/java/org/apache/hudi/common/util/HFileUtils.java:
##########
@@ -232,6 +232,15 @@ public byte[] serializeRecordsToLogBlock(HoodieStorage 
storage,
     return baos.toByteArray();
   }
 
+  /**
+   * Print the record in json format
+   */
+  private void printRecord(String msg, byte[] bs, Schema schema) throws 
IOException {
+    GenericRecord record = HoodieAvroUtils.bytesToAvro(bs, schema);
+    byte[] json = HoodieAvroUtils.avroToJson(record, true);
+    LOG.error(String.format("%s: %s", msg, new String(json)));

Review Comment:
   Just want to make sure this is just metadata and not actual data since that 
can leak sensitive information into the logs.
   
   If so, also update this to use `{}`



##########
hudi-hadoop-common/src/main/java/org/apache/hudi/common/util/HFileUtils.java:
##########
@@ -199,26 +199,26 @@ public byte[] serializeRecordsToLogBlock(HoodieStorage 
storage,
       }
 
       final byte[] recordBytes = serializeRecord(record, writerSchema, 
keyFieldName);
-      // If key exists in the map, append to its list. If not, create a new 
list.
-      // Get the existing list of recordBytes for the recordKey, or an empty 
list if it doesn't exist
-      List<byte[]> recordBytesList = sortedRecordsMap.getOrDefault(recordKey, 
new ArrayList<>());
-      recordBytesList.add(recordBytes);
-      // Put the updated list back into the map
-      sortedRecordsMap.put(recordKey, recordBytesList);
+      if (sortedRecordsMap.containsKey(recordKey)) {
+        LOG.error("Found duplicate record with recordKey: " + recordKey);

Review Comment:
   Use `{}` instead of string concatenation



##########
hudi-hadoop-common/src/main/java/org/apache/hudi/io/hadoop/HoodieAvroHFileWriter.java:
##########
@@ -73,14 +72,15 @@ public class HoodieAvroHFileWriter
   private final String instantTime;
   private final TaskContextSupplier taskContextSupplier;
   private final boolean populateMetaFields;
+  private final Schema schema;
   private final Option<Schema.Field> keyFieldSchema;
   private HFile.Writer writer;
   private String minRecordKey;
   private String maxRecordKey;
   private String prevRecordKey;
 
   // This is private in CacheConfig so have been copied here.
-  private static final String DROP_BEHIND_CACHE_COMPACTION_KEY = 
"hbase.hfile.drop.behind.compaction";
+  private static String DROP_BEHIND_CACHE_COMPACTION_KEY = 
"hbase.hfile.drop.behind.compaction";

Review Comment:
   keep this as `final`?



##########
hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java:
##########
@@ -1980,6 +1980,9 @@ public void testColStatsPrefixLookup() throws IOException 
{
 
       // there are 3 partitions in total and 2 commits. total entries should 
be 6.
       assertEquals(result.size(), 6);
+      result.forEach(entry -> {
+        //LOG.warn("Prefix search entries just for record key col : " + 
entry.getRecordKey().toString() + " :: " + 
entry.getData().getColumnStatMetadata().get().toString());

Review Comment:
   Remove this commented out code and similarly do the same below?



##########
hudi-hadoop-common/src/main/java/org/apache/hudi/io/hadoop/HoodieAvroHFileWriter.java:
##########
@@ -73,14 +72,15 @@ public class HoodieAvroHFileWriter
   private final String instantTime;
   private final TaskContextSupplier taskContextSupplier;
   private final boolean populateMetaFields;
+  private final Schema schema;

Review Comment:
   If this is unused, let's keep it removed



##########
hudi-hadoop-common/src/main/java/org/apache/hudi/io/hadoop/HoodieHBaseAvroHFileReader.java:
##########
@@ -370,10 +370,7 @@ public IndexedRecord next() {
     return new KeyPrefixIterator();
   }
 
-  private static Iterator<IndexedRecord> 
getRecordByKeyIteratorInternal(HFileScanner scanner,
-                                                                        String 
key,
-                                                                        Schema 
writerSchema,
-                                                                        Schema 
readerSchema) throws IOException {
+  private static Option<IndexedRecord> fetchRecordByKeyInternal(HFileScanner 
scanner, String key, Schema writerSchema, Schema readerSchema) throws 
IOException {
     KeyValue kv = new KeyValue(getUTF8Bytes(key), null, null, null);

Review Comment:
   `getUTF8Bytes(key)` is evaluated twice, here and at line 397. Let's move 
this into a variable to ensure it's only computed once.



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