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



##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java
##########
@@ -278,7 +287,7 @@ protected boolean writeRecord(HoodieRecord<T> hoodieRecord, 
Option<IndexedRecord
    * Go through an old record. Here if we detect a newer version shows up, we 
write the new one to the file.
    */
   public void write(GenericRecord oldRecord) {
-    String key = 
oldRecord.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
+    String key = populateMetaColumns ? 
oldRecord.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString() : 
keyGenerator.getKey(oldRecord).getRecordKey();

Review comment:
       not sure if we need to abstract this out and keep it outside of 
MergeHandle itself. there is only two options. Either use meta cols or use 
keyGen to compute record keys. So, have decided to manage it here itself. 

##########
File path: 
hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java
##########
@@ -225,6 +227,14 @@ protected void initMetaClient(HoodieTableType tableType) 
throws IOException {
     metaClient = HoodieTestUtils.init(hadoopConf, basePath, tableType);
   }
 
+  protected Properties getPropertiesForKeyGen() {
+    Properties properties = new Properties();
+    properties.put(HoodieTableConfig.HOODIE_POPULATE_META_COLUMNS.key(), 
"false");

Review comment:
       HoodieTestDataGenerator has these fields for commonly used schema and 
hence hardcoded it here. 

##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieParquetWriter.java
##########
@@ -69,14 +70,19 @@ public HoodieParquetWriter(String instantTime, Path file, 
HoodieAvroParquetConfi
     this.writeSupport = parquetConfig.getWriteSupport();
     this.instantTime = instantTime;
     this.taskContextSupplier = taskContextSupplier;
+    this.populateMetaCols = populateMetacols;
   }
 
   @Override
   public void writeAvroWithMetadata(R avroRecord, HoodieRecord record) throws 
IOException {
-    prepRecordWithMetadata(avroRecord, record, instantTime,
-        taskContextSupplier.getPartitionIdSupplier().get(), recordIndex, 
file.getName());
-    super.write(avroRecord);
-    writeSupport.add(record.getRecordKey());
+    if (populateMetaCols) {
+      prepRecordWithMetadata(avroRecord, record, instantTime,
+          taskContextSupplier.getPartitionIdSupplier().get(), recordIndex, 
file.getName());
+      super.write(avroRecord);
+      writeSupport.add(record.getRecordKey());

Review comment:
       as of this patch, I assume boom goes hand in hand w/ meta cols. If 
populateMetaCols is false, we are not adding bloom index. 

##########
File path: 
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/simple/FlinkHoodieSimpleIndex.java
##########
@@ -130,7 +130,7 @@ public boolean isImplicitWithStorage() {
                                                                     
List<Pair<String, HoodieBaseFile>> latestBaseFiles) {
 
     List<HoodieKeyLocationFetchHandle<T, List<HoodieRecord<T>>, 
List<HoodieKey>, List<WriteStatus>>> hoodieKeyLocationFetchHandles =
-        context.map(latestBaseFiles, partitionPathBaseFile -> new 
HoodieKeyLocationFetchHandle<>(config, hoodieTable, partitionPathBaseFile), 
parallelism);
+        context.map(latestBaseFiles, partitionPathBaseFile -> new 
HoodieKeyLocationFetchHandle<>(config, hoodieTable, partitionPathBaseFile, 
null), parallelism);

Review comment:
       as I have mentioned in the desc, I am not fixing flink and java in this 
patch. 

##########
File path: 
hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java
##########
@@ -121,7 +121,7 @@ private void 
assertSchemaEvolutionOnUpdateResult(WriteStatus insertResult, Hoodi
     jsc.parallelize(Arrays.asList(1)).map(x -> {
       Executable executable = () -> {
         HoodieMergeHandle mergeHandle = new 
HoodieMergeHandle(updateTable.getConfig(), "101", updateTable,
-            updateRecords.iterator(), updateRecords.get(0).getPartitionPath(), 
insertResult.getFileId(), supplier);
+            updateRecords.iterator(), updateRecords.get(0).getPartitionPath(), 
insertResult.getFileId(), supplier, null);

Review comment:
       Have not fixed this test for virtual keys

##########
File path: 
hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java
##########
@@ -142,6 +143,43 @@
     return hoodieKeys;
   }
 
+  /**
+   * Fetch {@link HoodieKey}s from the given parquet file.
+   *
+   * @param filePath      The parquet file path.
+   * @param configuration configuration to build fs object
+   * @return {@link List} of {@link HoodieKey}s fetched from the parquet file
+   */
+  @Override
+  public List<HoodieKey> fetchRecordKeyPartitionPath(Configuration 
configuration, Path filePath, BaseKeyGenerator keyGenerator) {

Review comment:
       not sure if we can add another argument to existing api and 
generate/fetch recordKeys and partition path based on that. Felt this is neat. 




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