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


##########
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java:
##########
@@ -1869,62 +1889,70 @@ public static BigDecimal tryUpcastDecimal(BigDecimal 
value, final LogicalTypes.D
     return value;
   }
 
-  public static Option<Schema> tryResolveSchemaForTable(HoodieTableMetaClient 
dataTableMetaClient) {
+  public static Option<HoodieSchema> 
tryResolveSchemaForTable(HoodieTableMetaClient dataTableMetaClient) {
     if 
(dataTableMetaClient.getCommitsTimeline().filterCompletedInstants().countInstants()
 == 0) {
       return Option.empty();
     }
     try {
       TableSchemaResolver schemaResolver = new 
TableSchemaResolver(dataTableMetaClient);
-      return Option.of(schemaResolver.getTableAvroSchema());
+      return 
Option.of(HoodieSchema.fromAvroSchema(schemaResolver.getTableAvroSchema()));
     } catch (Exception e) {
       throw new HoodieException("Failed to get latest columns for " + 
dataTableMetaClient.getBasePath(), e);
     }
   }
 
   /**
-   * Given a schema, coerces provided value to instance of {@link 
Comparable<?>} such that
-   * it could subsequently be used in column stats
+   * Given a HoodieSchema, coerces provided value to instance of {@link 
Comparable<?>} such that it could subsequently be used in column stats. This 
method uses HoodieSchema for in-memory processing
+   * while maintaining compatibility with existing Avro-based serialization.
+   * <p>
+   * NOTE: This method has to stay compatible with the semantic of {@link 
FileFormatUtils#readColumnStatsFromMetadata} as they are used in tandem
    *
-   * NOTE: This method has to stay compatible with the semantic of
-   *      {@link FileFormatUtils#readColumnStatsFromMetadata} as they are used 
in tandem
+   * @param schema       the HoodieSchema to use for type coercion
+   * @param val          the value to coerce
+   * @return the coerced value as a Comparable
+   * @since 1.2.0
    */
-  public static Comparable<?> coerceToComparable(Schema schema, Object val) {
+  public static Comparable<?> coerceToComparable(HoodieSchema schema, Object 
val) {
     if (val == null) {
       return null;
     }
 
-    switch (schema.getType()) {
+    HoodieSchemaType schemaType = schema.getType();
+
+    switch (schemaType) {
       case UNION:
         // TODO we need to handle unions in general case as well
-        return coerceToComparable(getNonNullTypeFromUnion(schema), val);
+        return coerceToComparable(schema.getNonNullType(), val);
 
       case FIXED:
       case BYTES:
-        if (schema.getLogicalType() instanceof LogicalTypes.Decimal) {
-          return (Comparable<?>) val;
-        }
         return (ByteBuffer) val;
-
-
+      case DECIMAL:
+        return (Comparable<?>) val;
       case INT:
-        if (schema.getLogicalType() == LogicalTypes.date()
-            || schema.getLogicalType() == LogicalTypes.timeMillis()) {
-          // NOTE: This type will be either {@code java.sql.Date} or 
{org.joda.LocalDate}
-          //       depending on the Avro version. Hence, we simply cast it to 
{@code Comparable<?>}
+        return castToInteger(val);
+      case DATE:
+        // NOTE: This type will be either {@code java.sql.Date} or 
{org.joda.LocalDate}
+        //       depending on the Avro version. Hence, we simply cast it to 
{@code Comparable<?>}
+        return (Comparable<?>) val;
+      case TIME:
+        HoodieSchema.Time timeSchema = (HoodieSchema.Time) schema;
+        TimePrecision precision = timeSchema.getPrecision();
+        if (precision.equals(TimePrecision.MILLIS) || 
precision.equals(TimePrecision.MICROS)) {
           return (Comparable<?>) val;
         }
         return castToInteger(val);

Review Comment:
   To match the old behavior, we can simply do the following if I'm 
understanding this correctly:
   ```suggestion
         case DATE:
          case TIME:
           // NOTE: This type will be either {@code java.sql.Date} or 
{org.joda.LocalDate}
           //       depending on the Avro version. Hence, we simply cast it to 
{@code Comparable<?>}
           return (Comparable<?>) val;
   ```



##########
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java:
##########
@@ -1869,62 +1889,70 @@ public static BigDecimal tryUpcastDecimal(BigDecimal 
value, final LogicalTypes.D
     return value;
   }
 
-  public static Option<Schema> tryResolveSchemaForTable(HoodieTableMetaClient 
dataTableMetaClient) {
+  public static Option<HoodieSchema> 
tryResolveSchemaForTable(HoodieTableMetaClient dataTableMetaClient) {
     if 
(dataTableMetaClient.getCommitsTimeline().filterCompletedInstants().countInstants()
 == 0) {
       return Option.empty();
     }
     try {
       TableSchemaResolver schemaResolver = new 
TableSchemaResolver(dataTableMetaClient);
-      return Option.of(schemaResolver.getTableAvroSchema());
+      return 
Option.of(HoodieSchema.fromAvroSchema(schemaResolver.getTableAvroSchema()));
     } catch (Exception e) {
       throw new HoodieException("Failed to get latest columns for " + 
dataTableMetaClient.getBasePath(), e);
     }
   }
 
   /**
-   * Given a schema, coerces provided value to instance of {@link 
Comparable<?>} such that
-   * it could subsequently be used in column stats
+   * Given a HoodieSchema, coerces provided value to instance of {@link 
Comparable<?>} such that it could subsequently be used in column stats. This 
method uses HoodieSchema for in-memory processing
+   * while maintaining compatibility with existing Avro-based serialization.
+   * <p>
+   * NOTE: This method has to stay compatible with the semantic of {@link 
FileFormatUtils#readColumnStatsFromMetadata} as they are used in tandem
    *
-   * NOTE: This method has to stay compatible with the semantic of
-   *      {@link FileFormatUtils#readColumnStatsFromMetadata} as they are used 
in tandem
+   * @param schema       the HoodieSchema to use for type coercion
+   * @param val          the value to coerce
+   * @return the coerced value as a Comparable
+   * @since 1.2.0
    */
-  public static Comparable<?> coerceToComparable(Schema schema, Object val) {
+  public static Comparable<?> coerceToComparable(HoodieSchema schema, Object 
val) {
     if (val == null) {
       return null;
     }
 
-    switch (schema.getType()) {
+    HoodieSchemaType schemaType = schema.getType();
+
+    switch (schemaType) {
       case UNION:
         // TODO we need to handle unions in general case as well
-        return coerceToComparable(getNonNullTypeFromUnion(schema), val);
+        return coerceToComparable(schema.getNonNullType(), val);
 
       case FIXED:
       case BYTES:
-        if (schema.getLogicalType() instanceof LogicalTypes.Decimal) {
-          return (Comparable<?>) val;
-        }
         return (ByteBuffer) val;
-
-
+      case DECIMAL:
+        return (Comparable<?>) val;
       case INT:
-        if (schema.getLogicalType() == LogicalTypes.date()
-            || schema.getLogicalType() == LogicalTypes.timeMillis()) {
-          // NOTE: This type will be either {@code java.sql.Date} or 
{org.joda.LocalDate}
-          //       depending on the Avro version. Hence, we simply cast it to 
{@code Comparable<?>}
+        return castToInteger(val);
+      case DATE:
+        // NOTE: This type will be either {@code java.sql.Date} or 
{org.joda.LocalDate}
+        //       depending on the Avro version. Hence, we simply cast it to 
{@code Comparable<?>}
+        return (Comparable<?>) val;
+      case TIME:
+        HoodieSchema.Time timeSchema = (HoodieSchema.Time) schema;
+        TimePrecision precision = timeSchema.getPrecision();
+        if (precision.equals(TimePrecision.MILLIS) || 
precision.equals(TimePrecision.MICROS)) {
           return (Comparable<?>) val;
         }
         return castToInteger(val);
-
       case LONG:
-        if (schema.getLogicalType() == LogicalTypes.timeMicros()
-            || schema.getLogicalType() == LogicalTypes.timestampMicros()
-            || schema.getLogicalType() == LogicalTypes.timestampMillis()) {
+        return castToLong(val);
+      case TIMESTAMP:
+        HoodieSchema.Timestamp timestampSchema = (HoodieSchema.Timestamp) 
schema;
+        TimePrecision tsPrecision = timestampSchema.getPrecision();
+        if (tsPrecision.equals(TimePrecision.MILLIS) || 
tsPrecision.equals(TimePrecision.MICROS)) {
           // NOTE: This type will be either {@code java.sql.Date} or 
{org.joda.LocalDate}
           //       depending on the Avro version. Hence, we simply cast it to 
{@code Comparable<?>}
           return (Comparable<?>) val;
         }
         return castToLong(val);

Review Comment:
   This can just cast to a Comparable, it does not need to cast to a long



##########
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java:
##########
@@ -2640,16 +2673,20 @@ private static Stream<HoodieRecord> 
collectAndProcessColumnMetadata(Iterable<Hoo
 
   private static Stream<HoodieRecord> collectAndProcessColumnMetadata(String 
partitionPath, boolean isTightBound, Option<String> indexPartitionOpt,
                                                                       
Stream<HoodieColumnRangeMetadata<Comparable>> fileColumnMetadata,
-                                                                      
Map<String, Schema> colsToIndexSchemaMap,
+                                                                      
Map<String, HoodieSchema> colsToIndexSchemaMap,
                                                                       
HoodieIndexVersion partitionStatsIndexVersion
   ) {
+    // Convert Avro Schema map to HoodieSchema map
+    Map<String, HoodieSchema> hoodieSchemaMap = 
colsToIndexSchemaMap.entrySet().stream()
+        .collect(Collectors.toMap(Map.Entry::getKey, entry -> 
entry.getValue()));

Review Comment:
   This is just an identity mapping, why can't we just use 
`colsToIndexSchemaMap`?



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/ColumnStatsIndexSupport.scala:
##########
@@ -57,7 +57,7 @@ import scala.collection.parallel.mutable.ParHashMap
 
 class ColumnStatsIndexSupport(spark: SparkSession,
                               tableSchema: StructType,
-                              avroSchema: Schema,
+                              hoodieSchema: HoodieSchema,

Review Comment:
   nitpick: let's just call this `schema`?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java:
##########
@@ -434,16 +436,20 @@ protected void processAppendResult(AppendResult result, 
Option<HoodieLogBlock> d
     updateWriteStatus(result, stat);
 
     if (config.isMetadataColumnStatsIndexEnabled()) {
+      HoodieSchema writerHoodieSchemaWithMetaFields = 
HoodieSchema.fromAvroSchema(writeSchemaWithMetaFields);
       HoodieIndexVersion indexVersion = 
HoodieTableMetadataUtil.existingIndexVersionOrDefault(PARTITION_NAME_COLUMN_STATS,
 hoodieTable.getMetaClient());
       Set<String> columnsToIndexSet = new HashSet<>(HoodieTableMetadataUtil
           .getColumnsToIndex(hoodieTable.getMetaClient().getTableConfig(),
-              config.getMetadataConfig(), 
Lazy.eagerly(Option.of(writeSchemaWithMetaFields)),
+              config.getMetadataConfig(), 
Lazy.eagerly(Option.of(writerHoodieSchemaWithMetaFields)),
               Option.of(this.recordMerger.getRecordType()), 
indexVersion).keySet());
-      final List<Pair<String, Schema.Field>> fieldsToIndex = 
columnsToIndexSet.stream()
-          .map(fieldName -> 
HoodieAvroUtils.getSchemaForField(writeSchemaWithMetaFields, 
fieldName)).collect(Collectors.toList());
+      final List<Pair<String, HoodieSchemaField>> fieldsToIndex = 
columnsToIndexSet.stream()
+          .map(fieldName -> {
+            Pair<String, Schema.Field> avroFieldPair = 
HoodieAvroUtils.getSchemaForField(writeSchemaWithMetaFields, fieldName);
+            return Pair.of(avroFieldPair.getKey(), new 
HoodieSchemaField(avroFieldPair.getValue()));

Review Comment:
   Let's simplify this by fetching the field schema directly from 
`writerHoodieSchemaWithMetaFields` instead of `writeSchemaWithMetaFields`



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkMetadataWriterUtils.java:
##########
@@ -393,16 +396,25 @@ public static HoodiePairData<String, 
List<HoodieColumnRangeMetadata<Comparable>>
                       ? Option.empty()
                       : Option.of(new Schema.Parser().parse(writerSchemaStr)));
       HoodieTableConfig tableConfig = dataMetaClient.getTableConfig();
-      Schema tableSchema = writerSchema.map(schema -> 
tableConfig.populateMetaFields() ? addMetadataFields(schema) : schema)
+      HoodieSchema tableSchema = writerSchema.map(schema -> 
tableConfig.populateMetaFields() ? addMetadataFields(schema) : 
schema).map(HoodieSchema::fromAvroSchema)
           .orElseThrow(() -> new IllegalStateException(String.format("Expected 
writer schema in commit metadata %s", commitMetadata)));
-      List<Pair<String, Schema>> columnsToIndexSchemaMap = 
columnsToIndex.stream()
-          .map(columnToIndex -> Pair.of(columnToIndex, 
HoodieAvroUtils.getSchemaForField(tableSchema, 
columnToIndex).getValue().schema()))
+      List<Pair<String, HoodieSchema>> columnsToIndexSchemaMap = 
columnsToIndex.stream()
+          .map(columnToIndex -> {
+            Pair<String, Schema.Field> avroFieldPair = 
HoodieAvroUtils.getSchemaForField(tableSchema.toAvroSchema(), columnToIndex);

Review Comment:
   Let's get the schema directly from the HoodieSchema here as well



##########
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java:
##########
@@ -699,25 +701,28 @@ public static List<String> 
getValidIndexedColumns(HoodieIndexDefinition indexDef
 
     return indexDefinition.getSourceFields().stream()
         .filter(indexCol -> {
-          Pair<String, Schema.Field> fieldSchemaPair = 
HoodieAvroUtils.getSchemaForField(tableSchema, indexCol);
-          Schema.Field fieldSchema = fieldSchemaPair.getRight();
-          return fieldSchema != null && 
!isTimestampMillisField(fieldSchema.schema());
+          Pair<String, Schema.Field> avroFieldPair = 
HoodieAvroUtils.getSchemaForField(tableSchema.toAvroSchema(), indexCol);

Review Comment:
   Similarly here, let's get the schema directly from the HoodieSchema



##########
hudi-common/src/test/java/org/apache/hudi/metadata/TestHoodieTableMetadataUtil.java:
##########
@@ -281,37 +283,27 @@ void testFilterNestedLogicalTimestampColumn() {
   @Test
   void testIsTimestampMillisField() {
     // Test timestamp-millis
-    Schema timestampMillisSchema = Schema.create(Schema.Type.LONG);
-    LogicalTypes.timestampMillis().addToSchema(timestampMillisSchema);
+    HoodieSchema timestampMillisSchema = HoodieSchema.createTimestampMillis();
     
assertTrue(HoodieTableMetadataUtil.isTimestampMillisField(timestampMillisSchema),
         "Should return true for timestamp-millis");
 
-    // Test local-timestamp-millis
-    Schema localTimestampMillisSchema = Schema.create(Schema.Type.LONG);
-    
LogicalTypes.localTimestampMillis().addToSchema(localTimestampMillisSchema);
-    
assertTrue(HoodieTableMetadataUtil.isTimestampMillisField(localTimestampMillisSchema),
-        "Should return true for local-timestamp-millis");

Review Comment:
   Should this test be preserved?



##########
hudi-hadoop-common/src/test/java/org/apache/hudi/metadata/TestHoodieTableMetadataUtil.java:
##########
@@ -432,7 +432,7 @@ public void testGetColumnsToIndex() {
     expected.add("col_11");
     expected.add("col_12.col12_1");
 
-    Schema schema = SchemaBuilder.record("TestRecord")
+    Schema avroSchema = SchemaBuilder.record("TestRecord")

Review Comment:
   Can this be updated to directly construct the HoodieSchema?



##########
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java:
##########
@@ -286,19 +286,20 @@ public static Map<String, 
HoodieColumnRangeMetadata<Comparable>> collectColumnRa
       // with the values from this record
       targetFields.forEach(fieldNameFieldPair -> {
         String fieldName = fieldNameFieldPair.getKey();
-        Schema fieldSchema = 
getNonNullTypeFromUnion(fieldNameFieldPair.getValue().schema());
-        ColumnStats colStats = allColumnStats.computeIfAbsent(fieldName, 
ignored -> new ColumnStats(getValueMetadata(fieldSchema, indexVersion)));
-        Object fieldValue = collectColumnRangeFieldValue(record, 
colStats.valueMetadata, fieldName, fieldSchema, recordSchema, properties);
+        HoodieSchemaField field = fieldNameFieldPair.getValue();
+        HoodieSchema hoodieFieldSchema = 
HoodieSchemaUtils.getNonNullTypeFromUnion(field.schema());

Review Comment:
   nitpick: can we keep the variable name `fieldSchema`?



##########
hudi-common/src/test/java/org/apache/hudi/metadata/TestHoodieTableMetadataUtil.java:
##########
@@ -188,7 +190,7 @@ void testFiltersOutTimestampMillisColumns() {
         .build();
     HoodieTableConfig tableConfig = mock(HoodieTableConfig.class);
     
when(tableConfig.getTableInitialVersion()).thenReturn(HoodieTableVersion.NINE);
-    List<String> result = 
HoodieTableMetadataUtil.getValidIndexedColumns(indexDefinition, tableSchema, 
tableConfig);
+    List<String> result = 
HoodieTableMetadataUtil.getValidIndexedColumns(indexDefinition, 
HoodieSchema.fromAvroSchema(tableSchema), tableConfig);

Review Comment:
   Can you convert `tableSchema` to use `HoodieSchema`?



##########
hudi-hadoop-common/src/test/java/org/apache/hudi/metadata/TestHoodieTableMetadataUtil.java:
##########
@@ -596,16 +598,16 @@ private void addNColumns(List<String> list, int n) {
     }
   }
 
-  private Schema getTableSchema(List<String> fieldNames) {
+  private HoodieSchema getTableSchema(List<String> fieldNames) {
     List<Schema.Field> fields = fieldNames.stream()
         .map(fieldName -> new Schema.Field(fieldName, 
createNullableSchema(Schema.Type.STRING), "", 
JsonProperties.NULL_VALUE)).collect(Collectors.toList());
-    return Schema.createRecord("Test_Hoodie_Record", "", "", false, fields);
+    return 
HoodieSchema.fromAvroSchema(Schema.createRecord("Test_Hoodie_Record", "", "", 
false, fields));

Review Comment:
   Let's update this to also directly construct the HoodieSchemaField and 
HoodieSchema



##########
hudi-hadoop-common/src/test/java/org/apache/hudi/metadata/TestHoodieTableMetadataUtil.java:
##########
@@ -624,47 +626,46 @@ public void testValidateDataTypeForPartitionStats() {
         .noDefault()
         .endRecord();
 
+    // Convert to HoodieSchema
+    HoodieSchema schema = HoodieSchema.fromAvroSchema(avroSchema);
+
     // Test for primitive fields
-    
assertTrue(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("stringField").schema(),
 Option.empty(), V1));
-    
assertTrue(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("intField").schema(),
 Option.empty(), V1));
-    
assertTrue(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("booleanField").schema(),
 Option.empty(), V1));
-    
assertTrue(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("floatField").schema(),
 Option.empty(), V1));
-    
assertTrue(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("doubleField").schema(),
 Option.empty(), V1));
-    
assertTrue(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("longField").schema(),
 Option.empty(), V1));
-    
assertTrue(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("unionIntField").schema(),
 Option.empty(), V1));
+    
assertTrue(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("stringField").get().schema(),
 Option.empty(), V1));
+    
assertTrue(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("intField").get().schema(),
 Option.empty(), V1));
+    
assertTrue(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("booleanField").get().schema(),
 Option.empty(), V1));
+    
assertTrue(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("floatField").get().schema(),
 Option.empty(), V1));
+    
assertTrue(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("doubleField").get().schema(),
 Option.empty(), V1));
+    
assertTrue(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("longField").get().schema(),
 Option.empty(), V1));
+    
assertTrue(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("unionIntField").get().schema(),
 Option.empty(), V1));
 
     // Test for unsupported fields
-    
assertFalse(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("arrayField").schema(),
 Option.empty(), V1));
-    
assertFalse(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("mapField").schema(),
 Option.empty(), V1));
-    
assertFalse(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("structField").schema(),
 Option.empty(), V1));
-    
assertFalse(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("bytesField").schema(),
 Option.of(HoodieRecord.HoodieRecordType.SPARK), V1));
+    
assertFalse(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("arrayField").get().schema(),
 Option.empty(), V1));
+    
assertFalse(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("mapField").get().schema(),
 Option.empty(), V1));
+    
assertFalse(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("structField").get().schema(),
 Option.empty(), V1));
+    
assertFalse(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("bytesField").get().schema(),
 Option.of(HoodieRecord.HoodieRecordType.SPARK), V1));
 
     // Test for logical types
     Schema dateFieldSchema = 
LogicalTypes.date().addToSchema(Schema.create(Schema.Type.INT));
-    schema = SchemaBuilder.record("TestRecord")
-        .fields()
-        .name("dateField").type(dateFieldSchema).noDefault()
-        .endRecord();
-    
assertTrue(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("dateField").schema(),
 Option.empty(), V1));
+    schema = HoodieSchema.fromAvroSchema(
+        SchemaBuilder.record("TestRecord")
+            .fields()
+            .name("dateField").type(dateFieldSchema).noDefault()
+            .endRecord());

Review Comment:
   Similarly here, let's construct the HoodieSchema directly



##########
hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/TestSecondaryIndexStreamingTrackerFileSlice.java:
##########
@@ -63,8 +63,9 @@ public class TestSecondaryIndexStreamingTrackerFileSlice {
   private HoodieTable hoodieTable;
   private HoodieWriteConfig config;
   private WriteStatus writeStatus;
-  private Schema schema;
+  private HoodieSchema schema;
   private HoodieTableMetaClient metaClient;
+  @SuppressWarnings("FieldCanBeLocal")

Review Comment:
   Let's just make this a local variable instead of adding this annotation?



##########
hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/functional/TestMetadataUtilRLIandSIRecordGeneration.java:
##########
@@ -605,22 +596,22 @@ public void testReducedByKeysForRLIRecords() throws 
IOException {
       fail("Should not have reached here");
     } catch (Exception e) {
       // expected. no-op
-      assertTrue(e.getCause() instanceof HoodieIOException);
+      assertInstanceOf(HoodieIOException.class, e.getCause());

Review Comment:
   If you're going to fix this syntax, it is better to migrate to 
`assertThrows` to simplify the setup



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