yihua commented on code in PR #17536:
URL: https://github.com/apache/hudi/pull/17536#discussion_r2612378542


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/SecondaryIndexRecordGenerationUtils.java:
##########
@@ -342,15 +342,15 @@ public Pair<String, String> next() {
     };
   }
 
-  private static Schema 
getRequestedSchemaForSecondaryIndex(HoodieTableMetaClient metaClient, 
HoodieSchema tableSchema, String secondaryKeyField) {
+  private static HoodieSchema 
getRequestedSchemaForSecondaryIndex(HoodieTableMetaClient metaClient, 
HoodieSchema tableSchema, String secondaryKeyField) {
     String[] recordKeyFields;
-    if (tableSchema.getField(RECORD_KEY_METADATA_FIELD) != null) {
+    if (tableSchema.getField(RECORD_KEY_METADATA_FIELD).isPresent()) {

Review Comment:
   Have we checked all `HoodieSchema#getField` calls to make sure they should 
use `isPresent()` instead of null check?  Looks like this is a bug from a 
previous PR but the test did not catch it.



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java:
##########
@@ -872,9 +871,9 @@ private static <T> HoodieData<HoodieRecord> 
readRecordKeysFromFileSliceSnapshot(
       final FileSlice fileSlice = partitionAndFileSlice.getValue();
       final String fileId = fileSlice.getFileId();
       HoodieReaderContext<T> readerContext = readerContextFactory.getContext();
-      Schema dataSchema = 
AvroSchemaCache.intern(HoodieAvroUtils.addMetadataFields(new 
Schema.Parser().parse(dataWriteConfig.getWriteSchema()), 
dataWriteConfig.allowOperationMetadataField()));
-      Schema requestedSchema = 
metaClient.getTableConfig().populateMetaFields() ? getRecordKeySchema()
-          : HoodieAvroUtils.projectSchema(dataSchema, 
Arrays.asList(metaClient.getTableConfig().getRecordKeyFields().orElse(new 
String[0])));
+      HoodieSchema dataSchema = 
HoodieSchemaCache.intern(HoodieSchemaUtils.addMetadataFields(HoodieSchema.parse(dataWriteConfig.getWriteSchema()),
 dataWriteConfig.allowOperationMetadataField()));
+      HoodieSchema requestedSchema = 
metaClient.getTableConfig().populateMetaFields() ? getRecordKeySchema()
+          : HoodieSchemaUtils.projectSchema(dataSchema, 
Arrays.asList(metaClient.getTableConfig().getRecordKeyFields().orElse(new 
String[0])));

Review Comment:
   To follow up in a separate PR: We should start adding more methods in 
`HoodieSchema` so we can avoid nested method calls (i.e., 
`AUtil.method1(BUtil.method2(C.method3(x), y), z)` which can be improved by 
`x.method(y, z)`) that reduce readability.



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