nsivabalan commented on a change in pull request #3968:
URL: https://github.com/apache/hudi/pull/3968#discussion_r747386088



##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java
##########
@@ -89,6 +89,10 @@
 
   private static final Logger LOG = 
LogManager.getLogger(HoodieBackedTableMetadataWriter.class);
 
+  // Virtual keys support for metadata table
+  private static final String RECORD_KEY_FIELD = "key";
+  private static final boolean POPULATE_META_FIELDS = false;

Review comment:
       can we add a config to HoodieMetadataConfig for this and use that 
instead of hard coding.

##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java
##########
@@ -360,7 +360,8 @@ protected void 
appendDataAndDeleteBlocks(Map<HeaderMetadataType, String> header)
       header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, 
writeSchemaWithMetaFields.toString());
       List<HoodieLogBlock> blocks = new ArrayList<>(2);
       if (recordList.size() > 0) {
-        
blocks.add(HoodieDataBlock.getBlock(hoodieTable.getLogDataBlockFormat(), 
recordList, header));
+        final String keyField = 
hoodieTable.getMetaClient().getTableConfig().getRecordKeyFieldProp();
+        
blocks.add(HoodieDataBlock.getBlock(hoodieTable.getLogDataBlockFormat(), 
recordList, header, keyField));

Review comment:
       Can we add a overloaded method to HoodieDataBlock.getBlock(). One same 
as in master (non virtual key path) and another one for virtual key path. 
probably good to call the 2nd one only when populateMetaFields is set to false 
here. 

##########
File path: 
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java
##########
@@ -46,10 +49,10 @@ public SimpleKeyGenerator(TypedProperties props) {
 
   SimpleKeyGenerator(TypedProperties props, String recordKeyField, String 
partitionPathField) {
     super(props);
-    this.recordKeyFields = recordKeyField == null
-        ? Collections.emptyList()
-        : Collections.singletonList(recordKeyField);
-    this.partitionPathFields = Collections.singletonList(partitionPathField);
+    this.recordKeyFields = (recordKeyField == null

Review comment:
       can we avoid touching these if not for code changes. not sure why your 
code style is doing this, but I can show you how to fix these easily. 

##########
File path: 
hudi-common/src/main/java/org/apache/hudi/common/util/SpillableMapUtils.java
##########
@@ -115,22 +115,28 @@ public static long generateChecksum(byte[] data) {
    * Utility method to convert bytes to HoodieRecord using schema and payload 
class.
    */
   public static <R> R convertToHoodieRecordPayload(GenericRecord rec, String 
payloadClazz, String preCombineField, boolean withOperationField) {
-    return convertToHoodieRecordPayload(rec, payloadClazz, preCombineField, 
Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, 
HoodieRecord.PARTITION_PATH_METADATA_FIELD), withOperationField);
+    return convertToHoodieRecordPayload(rec, payloadClazz, preCombineField, 
Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, 
HoodieRecord.PARTITION_PATH_METADATA_FIELD),
+        withOperationField, Option.empty());
   }
 
   /**
    * Utility method to convert bytes to HoodieRecord using schema and payload 
class.
    */
-  public static <R> R convertToHoodieRecordPayload(GenericRecord rec, String 
payloadClazz,
-                                                   String preCombineField, 
Pair<String, String> recordKeyPartitionPathPair,
-                                                   boolean withOperationField) 
{
-    String recKey = rec.get(recordKeyPartitionPathPair.getLeft()).toString();
-    String partitionPath = 
rec.get(recordKeyPartitionPathPair.getRight()).toString();
-    Object preCombineVal = getPreCombineVal(rec, preCombineField);
+  public static <R> R convertToHoodieRecordPayload(GenericRecord record, 
String payloadClazz,
+                                                   String preCombineField,
+                                                   Pair<String, String> 
recordKeyPartitionPathPair,
+                                                   boolean withOperationField,
+                                                   Option<String> 
partitionName) {

Review comment:
       can we rename  recordKeyPartitionPathPair to 
recordKeyPartitionPath**Field**Pair. 
   also, can we introduce another explicit method to use hard coded partition 
path so that its very clear and we can add java docs that it is used for 
metadata table where partition path is fixed. 

##########
File path: 
hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java
##########
@@ -343,11 +362,13 @@ private void processDataBlock(HoodieDataBlock dataBlock, 
Option<List<String>> ke
     }
   }
 
-  protected HoodieRecord<?> createHoodieRecord(IndexedRecord rec) {
-    if (!simpleKeyGenFields.isPresent()) {
-      return SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) 
rec, this.payloadClassFQN, this.preCombineField, this.withOperationField);
+  private HoodieRecord<?> createHoodieRecord(IndexedRecord rec) {
+    if (!virtualKeysEnabled) {
+      return SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) 
rec, this.payloadClassFQN,
+          this.preCombineField, this.withOperationField);
     } else {
-      return SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) 
rec, this.payloadClassFQN, this.preCombineField, this.simpleKeyGenFields.get(), 
this.withOperationField);
+      return SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) 
rec, this.payloadClassFQN,

Review comment:
       I feel we should not touch the methods here in 
AbstractHoodieLogRecordReader. for a regular data table w/ virutal keys 
enabled, existing code should handle. Its only the metadata table which will 
have issues. So, may be we should override createHoodieRecord() within 
HoodieMetadataMergedLogRecordReader and handle the special case for 
partitionName. 
   Also, thinking if we should confine the the new argument for this class 
(partitionName) to HoodieMetadataMergedLogRecordReader. 
   

##########
File path: 
hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieAvroDataBlock.java
##########
@@ -58,22 +59,27 @@
   private ThreadLocal<BinaryDecoder> decoderCache = new ThreadLocal<>();
 
   public HoodieAvroDataBlock(@Nonnull Map<HeaderMetadataType, String> 
logBlockHeader,
-       @Nonnull Map<HeaderMetadataType, String> logBlockFooter,
-       @Nonnull Option<HoodieLogBlockContentLocation> blockContentLocation, 
@Nonnull Option<byte[]> content,
-       FSDataInputStream inputStream, boolean readBlockLazily) {
+                             @Nonnull Map<HeaderMetadataType, String> 
logBlockFooter,
+                             @Nonnull Option<HoodieLogBlockContentLocation> 
blockContentLocation, @Nonnull Option<byte[]> content,
+                             FSDataInputStream inputStream, boolean 
readBlockLazily) {
     super(logBlockHeader, logBlockFooter, blockContentLocation, content, 
inputStream, readBlockLazily);
   }
 
   public HoodieAvroDataBlock(HoodieLogFile logFile, FSDataInputStream 
inputStream, Option<byte[]> content,
-       boolean readBlockLazily, long position, long blockSize, long 
blockEndpos, Schema readerSchema,
-       Map<HeaderMetadataType, String> header, Map<HeaderMetadataType, String> 
footer) {
+                             boolean readBlockLazily, long position, long 
blockSize, long blockEndpos, Schema readerSchema,
+                             Map<HeaderMetadataType, String> header, 
Map<HeaderMetadataType, String> footer, String keyField) {

Review comment:
       looks like keyField is not used in AvroDataBlock and is used only in 
HFileDataBlock. is it possible to confine the usage to just HFile data blocks 
if it makes sense? 




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