Skip to content

Conversation

@vinishjail97
Copy link
Contributor

@vinishjail97 vinishjail97 commented Dec 23, 2025

Describe the issue this Pull Request addresses

Fixes column stats generation for MAP and ARRAY types in metadata table. The metadata table column stats indexing fails with AvroRuntimeException: Not a record when attempting to collect statistics for nested values within MAP and ARRAY fields using synthetic accessor patterns like mapField.key_value.key and arrayField.list.element.

Root Cause: HoodieSchemaUtils.getNestedFieldInternal() only supported RECORD type navigation and returned Option.empty() for MAP and ARRAY types, causing these columns to be filtered out before reaching the Parquet reader that could handle them.

Summary and Changelog

Users can now collect column statistics for primitive values nested within MAP and ARRAY types, enabling data skipping optimizations for these complex types.

Changes:

  • Enhanced HoodieSchemaUtils.getNestedFieldInternal() to handle synthetic field accessors:
    • ARRAY types: Navigate via .list.element pattern to access array element schema
    • MAP types: Navigate via .key_value.key for map keys (always STRING) and .key_value.value for map values
    • Supports deep nesting: record.map.key_value.value.array.list.element.field
  • Added comprehensive test coverage in TestHoodieSchemaUtils:
    • Simple MAP/ARRAY navigation tests
    • Nested navigation tests (MAP of RECORDS, ARRAY of RECORDS)
    • Complex combinations (MAP of ARRAY of RECORDS)
    • Invalid path validation tests

Impact

None. This is an internal enhancement to schema navigation.

Risk Level

Low.

Documentation Update

None - no new configs added.

  • The config description must be updated if new configs are added or the default value of the configs are changed.
  • Any new feature or user-facing change requires updating the Hudi website. Please follow the
    instruction to make changes to the website. -->

Contributor's checklist

  • Read through contributor's guide
  • Enough context is provided in the sections above
  • Adequate tests were added if applicable

@github-actions github-actions bot added the size:L PR with lines of changes in (300, 1000] label Dec 23, 2025
@hudi-bot
Copy link
Collaborator

CI report:

Bot commands @hudi-bot supports the following commands:
  • @hudi-bot run azure re-run the last Azure build

@voonhous
Copy link
Member

Thought i missed something during the migration. Phew. Just realised that the original code is only meant to be used on Avro RECORD types only as it uses Avro.Schema#getField. Only Avro$RecordSchema#getField overrides it.

* using Parquet-style accessor patterns:</p>
*
* <ul>
* <li><b>RECORD types:</b> Standard dot notation (e.g., {@code "user.profile.name"})</li>
Copy link
Member

@voonhous voonhous Dec 24, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We do not have official public docs supporting this feature now. If users were to select a nested (ARRAY/MAP) field as part of the column stats index list, they'll need to follow this format to define the hoodie.metadata.index.column.stats.column.list config.

If we follow through with this, this will need a doc.
Ref: https://hudi.apache.org/docs/metadata_indexing/#index-creation-using-datasource

Comment on lines +525 to +529
return getNestedFieldInternal(
elementSchema,
pathAfterElement,
prefix + ARRAY_LIST_ELEMENT + "."
);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think there's room for optimization here.

There's quite a bit of object allocation going on the entire stack here. Every time we call substring(), split(), or +, we are creating a new String object on the heap, which eventually triggers Garbage Collection (GC).

We can avoid substrings by using offsets. Purely using indexOf for string manipulation.

I tested this locally, and it's a 10 - 12% faster via offset manipulation, something like:

  1. private static Option<Pair<String, HoodieSchemaField>> getNestedFieldInternal( HoodieSchema schema, String fullPath, int offset, String prefix)
  2. private static Option<Pair<String, HoodieSchemaField>> handleArrayNavigation( HoodieSchema arraySchema, String fullPath, int offset, String prefix)
  3. private static Option<Pair<String, HoodieSchemaField>> handleMapNavigation( HoodieSchema mapSchema, String fullPath, int offset, String prefix)

So the only time we're creating new string objects is when we are invoking getNestedFieldInternal when passing a new "prefix".

I did a rough cut:

// Parquet-style accessor constants used for MDT Column Stats paths
public static final String ARRAY_LIST = "list";
public static final String ARRAY_ELEMENT = "element";
public static final String MAP_KEY_VALUE = "key_value";
public static final String MAP_KEY = "key";
public static final String MAP_VALUE = "value";

private static final String ARRAY_LIST_ELEMENT = ARRAY_LIST + "." + ARRAY_ELEMENT;
private static final String MAP_KEY_VALUE_KEY = MAP_KEY_VALUE + "." + MAP_KEY;
private static final String MAP_KEY_VALUE_VALUE = MAP_KEY_VALUE + "." + MAP_VALUE;

public static Option<Pair<String, HoodieSchemaField>> getNestedField(HoodieSchema schema, String fieldName) {
  ValidationUtils.checkArgument(schema != null, "Schema cannot be null");
  ValidationUtils.checkArgument(fieldName != null && !fieldName.isEmpty(), "Field name cannot be null or empty");
  return getNestedFieldInternal(schema, fieldName, 0, "");
}

private static Option<Pair<String, HoodieSchemaField>> getNestedFieldInternal(
    HoodieSchema schema, String fullPath, int offset, String prefix) {

  HoodieSchema nonNullableSchema = getNonNullTypeFromUnion(schema);
  int nextDot = fullPath.indexOf('.', offset);

  // Terminal Case: No more dots in this segment
  if (nextDot == -1) {
    if (nonNullableSchema.getType() != HoodieSchemaType.RECORD) {
      return Option.empty();
    }
    String fieldName = fullPath.substring(offset);
    return nonNullableSchema.getField(fieldName)
        .map(field -> Pair.of(prefix + fieldName, field));
  }

  // Recursive Case: More nesting to explore
  String rootFieldName = fullPath.substring(offset, nextDot);
  int nextOffset = nextDot + 1;

  // Handle RECORD: standard field navigation
  if (nonNullableSchema.getType() == HoodieSchemaType.RECORD) {
    return nonNullableSchema.getField(rootFieldName)
        .flatMap(f -> getNestedFieldInternal(f.schema(), fullPath, nextOffset, prefix + rootFieldName + "."));
  }

  // Handle ARRAY: expect ".list.element"
  if (nonNullableSchema.getType() == HoodieSchemaType.ARRAY && ARRAY_LIST.equals(rootFieldName)) {
    return handleArrayNavigation(nonNullableSchema, fullPath, nextOffset, prefix);
  }

  // Handle MAP: expect ".key_value.key" or ".key_value.value"
  if (nonNullableSchema.getType() == HoodieSchemaType.MAP && MAP_KEY_VALUE.equals(rootFieldName)) {
    return handleMapNavigation(nonNullableSchema, fullPath, nextOffset, prefix);
  }

  return Option.empty();
}

private static Option<Pair<String, HoodieSchemaField>> handleArrayNavigation(
    HoodieSchema arraySchema, String fullPath, int offset, String prefix) {
  int nextPos = getNextOffset(fullPath, offset, ARRAY_ELEMENT);
  if (nextPos == -1) return Option.empty();

  HoodieSchema elSchema = arraySchema.getElementType();
  if (nextPos == fullPath.length()) {
    return Option.of(Pair.of(prefix + ARRAY_LIST_ELEMENT, HoodieSchemaField.of(ARRAY_ELEMENT, elSchema, null, null)));
  }
  return getNestedFieldInternal(elSchema, fullPath, nextPos, prefix + ARRAY_LIST_ELEMENT + ".");
}

private static Option<Pair<String, HoodieSchemaField>> handleMapNavigation(
    HoodieSchema mapSchema, String fullPath, int offset, String prefix) {
  
  // Check for "key" path
  int keyPos = getNextOffset(fullPath, offset, MAP_KEY);
  if (keyPos != -1) {
    if (keyPos == fullPath.length()) {
      return Option.of(Pair.of(prefix + MAP_KEY_VALUE_KEY, HoodieSchemaField.of(MAP_KEY, mapSchema.getKeyType(), null, null)));
    }
   // Map keys are primitives (Only String is supported on Spark, but i think other engines support types outside of Strings)
    return Option.empty(); 
  }

  // Check for "value" path
  int valPos = getNextOffset(fullPath, offset, MAP_VALUE);
  if (valPos != -1) {
    HoodieSchema vSchema = mapSchema.getValueType();
    if (valPos == fullPath.length()) {
      return Option.of(Pair.of(prefix + MAP_KEY_VALUE_VALUE, HoodieSchemaField.of(MAP_VALUE, vSchema, null, null)));
    }
    return getNestedFieldInternal(vSchema, fullPath, valPos, prefix + MAP_KEY_VALUE_VALUE + ".");
  }
  return Option.empty();
}

private static int getNextOffset(String path, int offset, String component) {
  // Just a simple search, it can be documented better... 
  if (!path.regionMatches(offset, component, 0, component.length())) return -1;
  int next = offset + component.length();
  if (next == path.length()) return next;
  return (path.charAt(next) == '.') ? next + 1 : -1;
}

assertFalse(HoodieSchemaUtils.hasSmallPrecisionDecimalField(HoodieSchema.parse(EXAMPLE_SCHEMA)));
}

@Test
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we also have a E2E test to make sure data skipping is working for colstats index generated on nested ARRAY/MAP types?

We can use the same schema that triggered this error.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

size:L PR with lines of changes in (300, 1000]

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants