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


##########
hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieBootstrapRecordIterator.java:
##########
@@ -64,15 +63,16 @@ public boolean hasNext() {
   public HoodieRecord<T> next() {
     HoodieRecord<T> dataRecord = dataFileIterator.next();
     HoodieRecord<T> skeletonRecord = skeletonIterator.next();
-    HoodieRecord<T> ret = dataRecord.prependMetaFields(schema, schema,
-        new MetadataValues().setCommitTime(skeletonRecord.getRecordKey(schema, 
HoodieRecord.COMMIT_TIME_METADATA_FIELD))
-            .setCommitSeqno(skeletonRecord.getRecordKey(schema, 
HoodieRecord.COMMIT_SEQNO_METADATA_FIELD))
-            .setRecordKey(skeletonRecord.getRecordKey(schema, 
HoodieRecord.RECORD_KEY_METADATA_FIELD))
-            .setPartitionPath(skeletonRecord.getRecordKey(schema, 
HoodieRecord.PARTITION_PATH_METADATA_FIELD))
-            .setFileName(skeletonRecord.getRecordKey(schema, 
HoodieRecord.FILENAME_METADATA_FIELD)), null);
+    //TODO boundary to revisit in later pr to use HoodieSchema
+    HoodieRecord<T> ret = dataRecord.prependMetaFields(schema.getAvroSchema(), 
schema.getAvroSchema(),

Review Comment:
   Let's assign `schema.getAvroSchema()` to a variable that can be used instead 
of repeatedly calling the method in all of the following lines.



##########
hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchema.java:
##########
@@ -106,6 +106,27 @@ public static HoodieSchema fromAvroSchema(Schema 
avroSchema) {
     return new HoodieSchema(avroSchema);
   }
 
+  /**
+   * Converts an Option<Schema> to Option<HoodieSchema>.
+   *
+   * @param schemaOption the optional Avro schema
+   * @return Option containing HoodieSchema, or empty if input was empty
+   */
+  public static Option<HoodieSchema> fromAvroSchemaOption(Option<Schema> 
schemaOption) {
+    return schemaOption.map(HoodieSchema::fromAvroSchema);
+  }
+
+  /**
+   * Converts an Option<HoodieSchema> to Option<Schema>.
+   *
+   * @param hoodieSchemaOption the optional HoodieSchema
+   * @return Option containing Avro Schema, or empty if input was empty
+   */
+  public static Option<Schema> toAvroSchemaOption(Option<HoodieSchema>
+                                                          hoodieSchemaOption) {
+    return hoodieSchemaOption.map(HoodieSchema::toAvroSchema);
+  }
+

Review Comment:
   These look unused currently, let's remove them?



##########
hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieBootstrapFileReader.java:
##########
@@ -61,9 +61,12 @@ public Set<Pair<String, Long>> filterRowKeys(Set<String> 
candidateRowKeys) {
   }
 
   @Override
-  public ClosableIterator<HoodieRecord<T>> getRecordIterator(Schema 
readerSchema, Schema requestedSchema) throws IOException {
+  public ClosableIterator<HoodieRecord<T>> getRecordIterator(HoodieSchema 
readerSchema, HoodieSchema requestedSchema) throws IOException {
     ClosableIterator<HoodieRecord<T>> skeletonIterator = 
skeletonFileReader.getRecordIterator(readerSchema, requestedSchema);
-    ClosableIterator<HoodieRecord<T>> dataFileIterator = 
dataFileReader.getRecordIterator(HoodieAvroUtils.removeMetadataFields(readerSchema),
 requestedSchema);
+    // TODO boundary for now to revisit HoodieAvroUtils in later pr to use 
HoodieSchema
+    Schema avroReaderSchema = readerSchema.getAvroSchema();
+    Schema dataReaderFields = 
HoodieAvroUtils.removeMetadataFields(avroReaderSchema);

Review Comment:
   There is a HoodieSchemaUtils.removeMetadataFields you can use here



##########
hudi-hadoop-common/src/test/java/org/apache/hudi/io/hadoop/TestHoodieHFileReaderWriter.java:
##########


Review Comment:
   Update this to directly be a `HoodieSchema` and update the other tests to 
follow the same pattern.



##########
hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieBootstrapFileReader.java:
##########
@@ -100,18 +105,21 @@ public boolean hasNext() {
 
       @Override
       public String next() {
+        // TODO boundary for now
         HoodieRecord<T> skeletonRecord = skeletonIterator.next();
-        return skeletonRecord.getRecordKey(schema, 
HoodieRecord.RECORD_KEY_METADATA_FIELD);
+        return skeletonRecord.getRecordKey(schema.getAvroSchema(), 
HoodieRecord.RECORD_KEY_METADATA_FIELD);
       }
     };
   }
 
   protected abstract void setPartitionField(int position, Object fieldValue, T 
row);
 
   @Override
-  public Schema getSchema() {
+  public HoodieSchema getSchema() {
     // return merged schema (meta fields + data file schema)
-    return HoodieAvroUtils.addMetadataFields(dataFileReader.getSchema());
+    // TODO boundary for now to revisit HoodieAvroUtils in later pr to use 
HoodieSchema
+    Schema avroReaderSchema = dataFileReader.getSchema().getAvroSchema();
+    return 
HoodieSchema.fromAvroSchema(HoodieAvroUtils.addMetadataFields(avroReaderSchema));

Review Comment:
   Just directly use the HoodieSchemaUtils here



##########
hudi-hadoop-common/src/test/java/org/apache/hudi/io/hadoop/TestHoodieHFileReaderWriter.java:
##########
@@ -276,7 +277,7 @@ public void testReadHFileFormatRecords(boolean 
useBloomFilter) throws Exception
       Schema avroSchema =
           getSchemaFromResource(TestHoodieReaderWriterBase.class, 
"/exampleSchema.avsc");

Review Comment:
   Let's add a helper called `getHoodieSchemaFromResource` that will return the 
schema as a HoodieSchema that way we can leverage the same helper in future 
test cases.



##########
hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/TestHoodieAvroFileWriterFactory.java:
##########
@@ -55,23 +56,23 @@ public void testGetFileWriter() throws IOException {
     HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
     SparkTaskContextSupplier supplier = new SparkTaskContextSupplier();
     HoodieFileWriter parquetWriter = 
HoodieFileWriterFactory.getFileWriter(instantTime,
-        parquetPath, table.getStorage(), cfg.getStorageConfig(), 
HoodieTestDataGenerator.AVRO_SCHEMA, supplier, HoodieRecordType.AVRO);
+        parquetPath, table.getStorage(), cfg.getStorageConfig(), 
HoodieSchema.fromAvroSchema(HoodieTestDataGenerator.AVRO_SCHEMA), supplier, 
HoodieRecordType.AVRO);

Review Comment:
   Let's push this `fromAvroSchema` into the `HoodieTestDataGenerator` so we 
can reference a constant schema other test cases more easily.



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