voonhous commented on code in PR #17694:
URL: https://github.com/apache/hudi/pull/17694#discussion_r2645453019
##########
hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchemaUtils.java:
##########
@@ -416,21 +453,138 @@ private static Option<Pair<String, HoodieSchemaField>>
getNestedFieldInternal(Ho
.map(field -> Pair.of(prefix + fieldName, field));
} else {
// Recursive case: nested field
- if (nonNullableSchema.getType() != HoodieSchemaType.RECORD) {
- return Option.empty();
- }
-
int dotIndex = fieldName.indexOf(".");
String rootFieldName = fieldName.substring(0, dotIndex);
String remainingPath = fieldName.substring(dotIndex + 1);
- return nonNullableSchema.getField(rootFieldName)
- .flatMap(rootField -> getNestedFieldInternal(
- rootField.schema(),
- remainingPath,
- prefix + rootFieldName + "."
- ));
+ // Handle RECORD types - standard field navigation
+ if (nonNullableSchema.getType() == HoodieSchemaType.RECORD) {
+ return nonNullableSchema.getField(rootFieldName)
+ .flatMap(rootField -> getNestedFieldInternal(
+ rootField.schema(),
+ remainingPath,
+ prefix + rootFieldName + "."
+ ));
+ }
+
+ // Handle ARRAY types - expect ".list.element" pattern
+ if (nonNullableSchema.getType() == HoodieSchemaType.ARRAY &&
ARRAY_LIST.equals(rootFieldName)) {
+ return handleArrayNavigation(nonNullableSchema, remainingPath, prefix);
+ }
+
+ // Handle MAP types - expect ".key_value.key" or ".key_value.value"
pattern
+ if (nonNullableSchema.getType() == HoodieSchemaType.MAP &&
MAP_KEY_VALUE.equals(rootFieldName)) {
+ return handleMapNavigation(nonNullableSchema, remainingPath, prefix);
+ }
+ // For all other types (primitives, etc.), cannot navigate
+ return Option.empty();
+ }
+ }
+
+ /**
+ * Extracts the remaining path after a component name, validating the format.
+ * Supports both "component" (end of path) and "component.rest" (continued
navigation).
+ *
+ * @param remainingPath the remaining path to parse
+ * @param component the component name to extract after (e.g.,
"element", "key", "value")
+ * @return Option containing the path after the component (empty string if
at end), or Option.empty() if invalid format
+ */
+ private static Option<String> extractPathAfterComponent(String
remainingPath, String component) {
+ if (remainingPath.equals(component)) {
+ return Option.of("");
+ } else if (remainingPath.startsWith(component + ".")) {
+ return Option.of(remainingPath.substring((component + ".").length()));
+ }
+ return Option.empty(); // Invalid format (e.g., "elementXYZ")
+ }
+
+ /**
+ * Handles navigation into ARRAY types using the Parquet-style
".list.element" pattern.
+ *
+ * @param arraySchema the ARRAY schema to navigate into
+ * @param remainingPath the remaining path after "list" (should start with
"element")
+ * @param prefix the accumulated field path prefix
+ * @return Option containing the nested field, or Option.empty() if invalid
path
+ */
+ private static Option<Pair<String, HoodieSchemaField>> handleArrayNavigation(
+ HoodieSchema arraySchema, String remainingPath, String prefix) {
+ return extractPathAfterComponent(remainingPath, "element")
+ .flatMap(pathAfterElement -> {
+ HoodieSchema elementSchema = arraySchema.getElementType();
+ if (pathAfterElement.isEmpty()) {
+ // We've reached the end - return nested component field for
element
+ HoodieSchemaField nestedComponentField = HoodieSchemaField.of(
+ "element",
+ elementSchema,
+ null, // doc
+ null // defaultVal
+ );
+ return Option.of(Pair.of(prefix + ARRAY_LIST_ELEMENT,
nestedComponentField));
+ } else {
+ // Continue navigating into element schema
+ return getNestedFieldInternal(
+ elementSchema,
+ pathAfterElement,
+ prefix + ARRAY_LIST_ELEMENT + "."
+ );
Review Comment:
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:
```java
// 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;
}
```
##########
hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchemaUtils.java:
##########
@@ -399,6 +410,32 @@ public static Option<Pair<String, HoodieSchemaField>>
getNestedField(HoodieSchem
/**
* Internal helper method for recursively retrieving nested fields.
*
+ * <p>Supports nested field access using dot notation including MAP and
ARRAY types
+ * using Parquet-style accessor patterns:</p>
+ *
+ * <ul>
+ * <li><b>RECORD types:</b> Standard dot notation (e.g., {@code
"user.profile.name"})</li>
Review Comment:
We do not have official public docs supporting this feature now. If users
will like 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.
##########
hudi-common/src/test/java/org/apache/hudi/common/schema/TestHoodieSchemaUtils.java:
##########
@@ -1588,4 +1588,204 @@ void testHasSmallPrecisionDecimalField() {
assertFalse(HoodieSchemaUtils.hasSmallPrecisionDecimalField(HoodieSchema.parse(SCHEMA_WITH_AVRO_TYPES_STR)));
assertFalse(HoodieSchemaUtils.hasSmallPrecisionDecimalField(HoodieSchema.parse(EXAMPLE_SCHEMA)));
}
+
+ @Test
Review Comment:
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.
--
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]