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


##########
hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchema.java:
##########
@@ -754,6 +775,172 @@ public HoodieSchema getNonNullType() {
     return types.get(0).getType() != HoodieSchemaType.NULL ? types.get(0) : 
types.get(1);
   }
 
+  /**
+   * Gets a nested field using dot notation, supporting Parquet-style 
array/map accessors.
+   *
+   * <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>
+   *   <li><b>ARRAY types:</b> Use {@code ".list.element"} to access array 
elements
+   *       <ul>
+   *         <li>Example: {@code "items.list.element"} accesses element schema 
of array</li>
+   *         <li>Example: {@code "items.list.element.id"} accesses nested 
field within array elements</li>
+   *       </ul>
+   *   </li>
+   *   <li><b>MAP types:</b> Use {@code ".key_value.key"} or {@code 
".key_value.value"} to access map components
+   *       <ul>
+   *         <li>Example: {@code "metadata.key_value.key"} accesses map keys 
(always STRING)</li>
+   *         <li>Example: {@code "metadata.key_value.value"} accesses map 
value schema</li>
+   *         <li>Example: {@code "nested_map.key_value.value.field"} accesses 
nested field within map values</li>
+   *       </ul>
+   *   </li>
+   * </ul>
+   *
+   * <p>Note: Spark Parquet files may use {@code ".array"} format instead of 
{@code ".list.element"}.
+   * This translation is handled at the Parquet reading level in ParquetUtils, 
not here.</p>
+   *
+   * @param fieldName Field path (e.g., "user.profile.name", 
"items.list.element", "metadata.key_value.value")
+   * @return Option containing a pair of canonical field name and the 
HoodieSchemaField, or Option.empty() if not found
+   */
+  public Option<Pair<String, HoodieSchemaField>> getNestedField(String 
fieldName) {
+    ValidationUtils.checkArgument(fieldName != null && !fieldName.isEmpty(), 
"Field name cannot be null or empty");
+    return getNestedFieldInternal(this, fieldName, 0, "");
+  }
+
+  /**
+   * Internal helper method for recursively retrieving nested fields using 
offset-based navigation.
+   *
+   * @param schema   the current schema to search in
+   * @param fullPath the full field path string
+   * @param offset   current position in fullPath
+   * @param prefix   the accumulated field path prefix
+   * @return Option containing a pair of canonical field name and the 
HoodieSchemaField, or Option.empty() if field not found
+   */
+  private static Option<Pair<String, HoodieSchemaField>> 
getNestedFieldInternal(
+      HoodieSchema schema, String fullPath, int offset, String prefix) {
+    HoodieSchema nonNullableSchema = schema.getNonNullType();
+    int nextDot = fullPath.indexOf('.', offset);
+    // Terminal case: no more dots in this segment
+    if (nextDot == -1) {
+      String fieldName = fullPath.substring(offset);
+
+      // Handle ARRAY terminal case: "element" (after "list.")
+      if (nonNullableSchema.getType() == HoodieSchemaType.ARRAY) {
+        if (ARRAY_ELEMENT.equals(fieldName)) {
+          HoodieSchema elSchema = nonNullableSchema.getElementType();
+          return Option.of(Pair.of(prefix + ARRAY_LIST_ELEMENT,
+              HoodieSchemaField.of(ARRAY_ELEMENT, elSchema, null, null)));
+        }
+        return Option.empty();
+      }
+
+      // Handle RECORD terminal case
+      if (nonNullableSchema.getType() != HoodieSchemaType.RECORD) {
+        return Option.empty();
+      }
+      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();
+  }
+
+  /**
+   * Handles navigation into ARRAY types using the ".list.element" pattern.
+   *
+   * @param arraySchema the ARRAY schema to navigate into
+   * @param fullPath    the full field path string
+   * @param offset      current position in fullPath (should point to 
"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 fullPath, int offset, String prefix) {
+    int nextPos = getNextOffset(fullPath, offset, ARRAY_ELEMENT);
+    if (nextPos == -1) {
+      return Option.empty();
+    }
+
+    HoodieSchema elSchema = arraySchema.getElementType();

Review Comment:
   nit: change `elSchema` to `elementSchema` throughout this file



##########
hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchema.java:
##########
@@ -754,6 +775,172 @@ public HoodieSchema getNonNullType() {
     return types.get(0).getType() != HoodieSchemaType.NULL ? types.get(0) : 
types.get(1);
   }
 
+  /**
+   * Gets a nested field using dot notation, supporting Parquet-style 
array/map accessors.
+   *
+   * <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>
+   *   <li><b>ARRAY types:</b> Use {@code ".list.element"} to access array 
elements
+   *       <ul>
+   *         <li>Example: {@code "items.list.element"} accesses element schema 
of array</li>
+   *         <li>Example: {@code "items.list.element.id"} accesses nested 
field within array elements</li>
+   *       </ul>
+   *   </li>
+   *   <li><b>MAP types:</b> Use {@code ".key_value.key"} or {@code 
".key_value.value"} to access map components
+   *       <ul>
+   *         <li>Example: {@code "metadata.key_value.key"} accesses map keys 
(always STRING)</li>
+   *         <li>Example: {@code "metadata.key_value.value"} accesses map 
value schema</li>
+   *         <li>Example: {@code "nested_map.key_value.value.field"} accesses 
nested field within map values</li>
+   *       </ul>
+   *   </li>
+   * </ul>
+   *
+   * <p>Note: Spark Parquet files may use {@code ".array"} format instead of 
{@code ".list.element"}.
+   * This translation is handled at the Parquet reading level in ParquetUtils, 
not here.</p>
+   *
+   * @param fieldName Field path (e.g., "user.profile.name", 
"items.list.element", "metadata.key_value.value")
+   * @return Option containing a pair of canonical field name and the 
HoodieSchemaField, or Option.empty() if not found
+   */
+  public Option<Pair<String, HoodieSchemaField>> getNestedField(String 
fieldName) {
+    ValidationUtils.checkArgument(fieldName != null && !fieldName.isEmpty(), 
"Field name cannot be null or empty");
+    return getNestedFieldInternal(this, fieldName, 0, "");
+  }
+
+  /**
+   * Internal helper method for recursively retrieving nested fields using 
offset-based navigation.
+   *
+   * @param schema   the current schema to search in
+   * @param fullPath the full field path string
+   * @param offset   current position in fullPath
+   * @param prefix   the accumulated field path prefix
+   * @return Option containing a pair of canonical field name and the 
HoodieSchemaField, or Option.empty() if field not found
+   */
+  private static Option<Pair<String, HoodieSchemaField>> 
getNestedFieldInternal(
+      HoodieSchema schema, String fullPath, int offset, String prefix) {
+    HoodieSchema nonNullableSchema = schema.getNonNullType();
+    int nextDot = fullPath.indexOf('.', offset);
+    // Terminal case: no more dots in this segment
+    if (nextDot == -1) {
+      String fieldName = fullPath.substring(offset);
+
+      // Handle ARRAY terminal case: "element" (after "list.")

Review Comment:
   Do you need special handling for the Map terminal case?



##########
hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchema.java:
##########
@@ -754,6 +775,172 @@ public HoodieSchema getNonNullType() {
     return types.get(0).getType() != HoodieSchemaType.NULL ? types.get(0) : 
types.get(1);
   }
 
+  /**
+   * Gets a nested field using dot notation, supporting Parquet-style 
array/map accessors.
+   *
+   * <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>
+   *   <li><b>ARRAY types:</b> Use {@code ".list.element"} to access array 
elements
+   *       <ul>
+   *         <li>Example: {@code "items.list.element"} accesses element schema 
of array</li>
+   *         <li>Example: {@code "items.list.element.id"} accesses nested 
field within array elements</li>
+   *       </ul>
+   *   </li>
+   *   <li><b>MAP types:</b> Use {@code ".key_value.key"} or {@code 
".key_value.value"} to access map components
+   *       <ul>
+   *         <li>Example: {@code "metadata.key_value.key"} accesses map keys 
(always STRING)</li>
+   *         <li>Example: {@code "metadata.key_value.value"} accesses map 
value schema</li>
+   *         <li>Example: {@code "nested_map.key_value.value.field"} accesses 
nested field within map values</li>
+   *       </ul>
+   *   </li>
+   * </ul>
+   *
+   * <p>Note: Spark Parquet files may use {@code ".array"} format instead of 
{@code ".list.element"}.
+   * This translation is handled at the Parquet reading level in ParquetUtils, 
not here.</p>
+   *
+   * @param fieldName Field path (e.g., "user.profile.name", 
"items.list.element", "metadata.key_value.value")
+   * @return Option containing a pair of canonical field name and the 
HoodieSchemaField, or Option.empty() if not found
+   */
+  public Option<Pair<String, HoodieSchemaField>> getNestedField(String 
fieldName) {
+    ValidationUtils.checkArgument(fieldName != null && !fieldName.isEmpty(), 
"Field name cannot be null or empty");
+    return getNestedFieldInternal(this, fieldName, 0, "");
+  }
+
+  /**
+   * Internal helper method for recursively retrieving nested fields using 
offset-based navigation.
+   *
+   * @param schema   the current schema to search in
+   * @param fullPath the full field path string
+   * @param offset   current position in fullPath
+   * @param prefix   the accumulated field path prefix
+   * @return Option containing a pair of canonical field name and the 
HoodieSchemaField, or Option.empty() if field not found
+   */
+  private static Option<Pair<String, HoodieSchemaField>> 
getNestedFieldInternal(
+      HoodieSchema schema, String fullPath, int offset, String prefix) {
+    HoodieSchema nonNullableSchema = schema.getNonNullType();
+    int nextDot = fullPath.indexOf('.', offset);
+    // Terminal case: no more dots in this segment
+    if (nextDot == -1) {
+      String fieldName = fullPath.substring(offset);
+
+      // Handle ARRAY terminal case: "element" (after "list.")
+      if (nonNullableSchema.getType() == HoodieSchemaType.ARRAY) {
+        if (ARRAY_ELEMENT.equals(fieldName)) {
+          HoodieSchema elSchema = nonNullableSchema.getElementType();
+          return Option.of(Pair.of(prefix + ARRAY_LIST_ELEMENT,
+              HoodieSchemaField.of(ARRAY_ELEMENT, elSchema, null, null)));
+        }
+        return Option.empty();
+      }
+
+      // Handle RECORD terminal case
+      if (nonNullableSchema.getType() != HoodieSchemaType.RECORD) {
+        return Option.empty();
+      }
+      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();
+  }
+
+  /**
+   * Handles navigation into ARRAY types using the ".list.element" pattern.
+   *
+   * @param arraySchema the ARRAY schema to navigate into
+   * @param fullPath    the full field path string
+   * @param offset      current position in fullPath (should point to 
"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 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 + ".");
+  }
+
+  /**
+   * Handles navigation into MAP types using the Parquet-style 
".key_value.key" or ".key_value.value" patterns.
+   *
+   * @param mapSchema the MAP schema to navigate into
+   * @param fullPath  the full field path string
+   * @param offset    current position in fullPath (should point to "key" or 
"value")
+   * @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>> 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, cannot navigate further
+      return Option.empty();
+    }
+
+    // Check for "value" path
+    int valPos = getNextOffset(fullPath, offset, MAP_VALUE);
+    if (valPos == -1) {
+      return Option.empty();
+    }
+
+    HoodieSchema vSchema = mapSchema.getValueType();

Review Comment:
   similarly here, let's use more meaningful names like `valueSchema`



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