You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by "the-other-tim-brown (via GitHub)" <gi...@apache.org> on 2023/02/24 01:53:39 UTC

[GitHub] [hudi] the-other-tim-brown commented on a diff in pull request #8010: [HUDI-4442] [HUDI-5001] Sanitize JsonConversion and RowSource

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


##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/SanitizationUtils.java:
##########
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities.sources.helpers;
+
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.utilities.sources.InputBatch;
+
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.avro.Schema;
+import org.apache.avro.SchemaParseException;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.types.ArrayType;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.MapType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+public class SanitizationUtils {
+
+  public static class Config {
+    // sanitizes names of invalid schema fields both in the data read from source and also in the schema.
+    // invalid definition here goes by avro naming convention (https://avro.apache.org/docs/current/spec.html#names).
+    public static final String SANITIZE_SCHEMA_FIELD_NAMES = "hoodie.deltastreamer.source.sanitize.invalid.schema.field.names";
+
+    public static final String SCHEMA_FIELD_NAME_INVALID_CHAR_MASK = "hoodie.deltastreamer.source.sanitize.invalid.char.mask";
+  }
+
+  private static final String AVRO_FIELD_NAME_KEY = "name";
+
+  private static DataType sanitizeDataTypeForAvro(DataType dataType, String invalidCharMask) {
+    if (dataType instanceof ArrayType) {
+      ArrayType arrayType = (ArrayType) dataType;
+      DataType sanitizedDataType = sanitizeDataTypeForAvro(arrayType.elementType(), invalidCharMask);
+      return new ArrayType(sanitizedDataType, arrayType.containsNull());
+    } else if (dataType instanceof MapType) {
+      MapType mapType = (MapType) dataType;
+      DataType sanitizedKeyDataType = sanitizeDataTypeForAvro(mapType.keyType(), invalidCharMask);
+      DataType sanitizedValueDataType = sanitizeDataTypeForAvro(mapType.valueType(), invalidCharMask);
+      return new MapType(sanitizedKeyDataType, sanitizedValueDataType, mapType.valueContainsNull());
+    } else if (dataType instanceof StructType) {
+      return sanitizeStructTypeForAvro((StructType) dataType, invalidCharMask);
+    }
+    return dataType;
+  }
+
+  // TODO(HUDI-5256): Refactor this to use InternalSchema when it is ready.
+  private static StructType sanitizeStructTypeForAvro(StructType structType, String invalidCharMask) {
+    StructType sanitizedStructType = new StructType();
+    StructField[] structFields = structType.fields();
+    for (StructField s : structFields) {
+      DataType currFieldDataTypeSanitized = sanitizeDataTypeForAvro(s.dataType(), invalidCharMask);
+      StructField structFieldCopy = new StructField(HoodieAvroUtils.sanitizeName(s.name(), invalidCharMask),
+          currFieldDataTypeSanitized, s.nullable(), s.metadata());
+      sanitizedStructType = sanitizedStructType.add(structFieldCopy);
+    }
+    return sanitizedStructType;
+  }
+
+  private static Dataset<Row> sanitizeColumnNamesForAvro(Dataset<Row> inputDataset, String invalidCharMask) {
+    StructField[] inputFields = inputDataset.schema().fields();
+    Dataset<Row> targetDataset = inputDataset;
+    for (StructField sf : inputFields) {
+      DataType sanitizedFieldDataType = sanitizeDataTypeForAvro(sf.dataType(), invalidCharMask);
+      if (!sanitizedFieldDataType.equals(sf.dataType())) {
+        // Sanitizing column names for nested types can be thought of as going from one schema to another
+        // which are structurally similar except for actual column names itself. So casting is safe and sufficient.
+        targetDataset = targetDataset.withColumn(sf.name(), targetDataset.col(sf.name()).cast(sanitizedFieldDataType));
+      }
+      String possibleRename = HoodieAvroUtils.sanitizeName(sf.name(), invalidCharMask);
+      if (!sf.name().equals(possibleRename)) {
+        targetDataset = targetDataset.withColumnRenamed(sf.name(), possibleRename);
+      }
+    }
+    return targetDataset;
+  }
+
+  /**
+   * Sanitize all columns including nested ones as per Avro conventions.
+   * @param srcBatch
+   * @param shouldSanitize
+   * @param invalidCharMask
+   * @return sanitized batch.
+   */
+  public static InputBatch<Dataset<Row>> maybeSanitizeFieldNames(InputBatch<Dataset<Row>> srcBatch,
+                                                                 boolean shouldSanitize,
+                                                                 String invalidCharMask) {
+    if (!shouldSanitize || !srcBatch.getBatch().isPresent()) {
+      return srcBatch;
+    }
+    Dataset<Row> srcDs = srcBatch.getBatch().get();
+    Dataset<Row> targetDs = sanitizeColumnNamesForAvro(srcDs, invalidCharMask);
+    return new InputBatch<>(Option.ofNullable(targetDs), srcBatch.getCheckpointForNextBatch(), srcBatch.getSchemaProvider());
+  }
+
+  /*
+   * We first rely on Avro to parse and then try to rename only for those failed.
+   * This way we can improve our parsing capabilities without breaking existing functionality.
+   * For example, we don't yet support multiple named schemas defined in a file.
+   */
+  public static Schema parseAvroSchema(String schemaStr, boolean shouldSanitize, String invalidCharMask) {
+    try {
+      return new Schema.Parser().parse(schemaStr);
+    } catch (SchemaParseException spe) {
+      // if sanitizing is not enabled rethrow the exception.
+      if (!shouldSanitize) {
+        throw spe;
+      }
+      // Rename avro fields and try parsing once again.
+      Option<Schema> parseResult = parseSanitizedAvroSchemaNoThrow(schemaStr, invalidCharMask);
+      if (!parseResult.isPresent()) {
+        // throw original exception.
+        throw spe;
+      }
+      return parseResult.get();
+    }
+  }
+
+  /**
+   * Parse list for sanitizing
+   * @param src - deserialized schema
+   * @param invalidCharMask - mask to replace invalid characters with
+   */
+  private static List<Object> transformList(List<Object> src, String invalidCharMask) {
+    return src.stream().map(obj -> {
+      if (obj instanceof List) {
+        return transformList((List<Object>) obj, invalidCharMask);
+      } else if (obj instanceof Map) {
+        return transformMap((Map<String, Object>) obj, invalidCharMask);
+      } else {
+        return obj;
+      }
+    }).collect(Collectors.toList());
+  }
+
+  /**
+   * Parse map for sanitizing. If we have a string in the map, and it is an avro field name key, then we sanitize the name.
+   * Otherwise, we keep recursively going through the schema.
+   * @param src - deserialized schema
+   * @param invalidCharMask - mask to replace invalid characters with
+   */
+  private static Map<String, Object> transformMap(Map<String, Object> src, String invalidCharMask) {
+    return src.entrySet().stream()
+        .map(kv -> {
+          if (kv.getValue() instanceof List) {
+            return Pair.of(kv.getKey(), transformList((List<Object>) kv.getValue(), invalidCharMask));
+          } else if (kv.getValue() instanceof Map) {
+            return Pair.of(kv.getKey(), transformMap((Map<String, Object>) kv.getValue(), invalidCharMask));
+          } else if (kv.getValue() instanceof String) {
+            String currentStrValue = (String) kv.getValue();
+            if (kv.getKey().equals(AVRO_FIELD_NAME_KEY)) {
+              return Pair.of(kv.getKey(), HoodieAvroUtils.sanitizeName(currentStrValue, invalidCharMask));
+            }
+            return Pair.of(kv.getKey(), currentStrValue);
+          } else {
+            return Pair.of(kv.getKey(), kv.getValue());
+          }
+        }).collect(Collectors.toMap(Pair::getLeft, Pair::getRight));
+  }
+
+  /**
+   * Sanitizes illegal field names in the schema using recursive calls to transformMap and transformList
+   */
+  private static Option<Schema> parseSanitizedAvroSchemaNoThrow(String schemaStr, String invalidCharMask) {
+    try {
+      ObjectMapper objectMapper = new ObjectMapper();

Review Comment:
   ObjectMapper instances are thread safe and it's recommended to reuse the object since it's heavy to construct. Can we make this a static instance?



##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/SanitizationUtils.java:
##########
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.utilities.sources.helpers;
+
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.utilities.sources.InputBatch;
+
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.avro.Schema;
+import org.apache.avro.SchemaParseException;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.types.ArrayType;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.MapType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+public class SanitizationUtils {
+
+  public static class Config {
+    // sanitizes names of invalid schema fields both in the data read from source and also in the schema.
+    // invalid definition here goes by avro naming convention (https://avro.apache.org/docs/current/spec.html#names).
+    public static final String SANITIZE_SCHEMA_FIELD_NAMES = "hoodie.deltastreamer.source.sanitize.invalid.schema.field.names";
+
+    public static final String SCHEMA_FIELD_NAME_INVALID_CHAR_MASK = "hoodie.deltastreamer.source.sanitize.invalid.char.mask";
+  }
+
+  private static final String AVRO_FIELD_NAME_KEY = "name";
+
+  private static DataType sanitizeDataTypeForAvro(DataType dataType, String invalidCharMask) {
+    if (dataType instanceof ArrayType) {
+      ArrayType arrayType = (ArrayType) dataType;
+      DataType sanitizedDataType = sanitizeDataTypeForAvro(arrayType.elementType(), invalidCharMask);
+      return new ArrayType(sanitizedDataType, arrayType.containsNull());
+    } else if (dataType instanceof MapType) {
+      MapType mapType = (MapType) dataType;
+      DataType sanitizedKeyDataType = sanitizeDataTypeForAvro(mapType.keyType(), invalidCharMask);
+      DataType sanitizedValueDataType = sanitizeDataTypeForAvro(mapType.valueType(), invalidCharMask);
+      return new MapType(sanitizedKeyDataType, sanitizedValueDataType, mapType.valueContainsNull());
+    } else if (dataType instanceof StructType) {
+      return sanitizeStructTypeForAvro((StructType) dataType, invalidCharMask);
+    }
+    return dataType;
+  }
+
+  // TODO(HUDI-5256): Refactor this to use InternalSchema when it is ready.
+  private static StructType sanitizeStructTypeForAvro(StructType structType, String invalidCharMask) {
+    StructType sanitizedStructType = new StructType();
+    StructField[] structFields = structType.fields();
+    for (StructField s : structFields) {
+      DataType currFieldDataTypeSanitized = sanitizeDataTypeForAvro(s.dataType(), invalidCharMask);
+      StructField structFieldCopy = new StructField(HoodieAvroUtils.sanitizeName(s.name(), invalidCharMask),
+          currFieldDataTypeSanitized, s.nullable(), s.metadata());
+      sanitizedStructType = sanitizedStructType.add(structFieldCopy);
+    }
+    return sanitizedStructType;
+  }
+
+  private static Dataset<Row> sanitizeColumnNamesForAvro(Dataset<Row> inputDataset, String invalidCharMask) {
+    StructField[] inputFields = inputDataset.schema().fields();
+    Dataset<Row> targetDataset = inputDataset;
+    for (StructField sf : inputFields) {
+      DataType sanitizedFieldDataType = sanitizeDataTypeForAvro(sf.dataType(), invalidCharMask);
+      if (!sanitizedFieldDataType.equals(sf.dataType())) {
+        // Sanitizing column names for nested types can be thought of as going from one schema to another
+        // which are structurally similar except for actual column names itself. So casting is safe and sufficient.
+        targetDataset = targetDataset.withColumn(sf.name(), targetDataset.col(sf.name()).cast(sanitizedFieldDataType));
+      }
+      String possibleRename = HoodieAvroUtils.sanitizeName(sf.name(), invalidCharMask);
+      if (!sf.name().equals(possibleRename)) {
+        targetDataset = targetDataset.withColumnRenamed(sf.name(), possibleRename);
+      }
+    }
+    return targetDataset;
+  }
+
+  /**
+   * Sanitize all columns including nested ones as per Avro conventions.
+   * @param srcBatch
+   * @param shouldSanitize
+   * @param invalidCharMask
+   * @return sanitized batch.
+   */
+  public static InputBatch<Dataset<Row>> maybeSanitizeFieldNames(InputBatch<Dataset<Row>> srcBatch,
+                                                                 boolean shouldSanitize,
+                                                                 String invalidCharMask) {
+    if (!shouldSanitize || !srcBatch.getBatch().isPresent()) {
+      return srcBatch;
+    }
+    Dataset<Row> srcDs = srcBatch.getBatch().get();
+    Dataset<Row> targetDs = sanitizeColumnNamesForAvro(srcDs, invalidCharMask);
+    return new InputBatch<>(Option.ofNullable(targetDs), srcBatch.getCheckpointForNextBatch(), srcBatch.getSchemaProvider());
+  }
+
+  /*
+   * We first rely on Avro to parse and then try to rename only for those failed.
+   * This way we can improve our parsing capabilities without breaking existing functionality.
+   * For example, we don't yet support multiple named schemas defined in a file.
+   */
+  public static Schema parseAvroSchema(String schemaStr, boolean shouldSanitize, String invalidCharMask) {
+    try {
+      return new Schema.Parser().parse(schemaStr);
+    } catch (SchemaParseException spe) {
+      // if sanitizing is not enabled rethrow the exception.
+      if (!shouldSanitize) {
+        throw spe;
+      }
+      // Rename avro fields and try parsing once again.
+      Option<Schema> parseResult = parseSanitizedAvroSchemaNoThrow(schemaStr, invalidCharMask);
+      if (!parseResult.isPresent()) {
+        // throw original exception.
+        throw spe;
+      }
+      return parseResult.get();
+    }
+  }
+
+  /**
+   * Parse list for sanitizing
+   * @param src - deserialized schema
+   * @param invalidCharMask - mask to replace invalid characters with
+   */
+  private static List<Object> transformList(List<Object> src, String invalidCharMask) {
+    return src.stream().map(obj -> {
+      if (obj instanceof List) {
+        return transformList((List<Object>) obj, invalidCharMask);
+      } else if (obj instanceof Map) {
+        return transformMap((Map<String, Object>) obj, invalidCharMask);
+      } else {
+        return obj;
+      }
+    }).collect(Collectors.toList());
+  }
+
+  /**
+   * Parse map for sanitizing. If we have a string in the map, and it is an avro field name key, then we sanitize the name.
+   * Otherwise, we keep recursively going through the schema.
+   * @param src - deserialized schema
+   * @param invalidCharMask - mask to replace invalid characters with
+   */
+  private static Map<String, Object> transformMap(Map<String, Object> src, String invalidCharMask) {
+    return src.entrySet().stream()
+        .map(kv -> {
+          if (kv.getValue() instanceof List) {
+            return Pair.of(kv.getKey(), transformList((List<Object>) kv.getValue(), invalidCharMask));
+          } else if (kv.getValue() instanceof Map) {
+            return Pair.of(kv.getKey(), transformMap((Map<String, Object>) kv.getValue(), invalidCharMask));
+          } else if (kv.getValue() instanceof String) {
+            String currentStrValue = (String) kv.getValue();
+            if (kv.getKey().equals(AVRO_FIELD_NAME_KEY)) {
+              return Pair.of(kv.getKey(), HoodieAvroUtils.sanitizeName(currentStrValue, invalidCharMask));
+            }
+            return Pair.of(kv.getKey(), currentStrValue);
+          } else {
+            return Pair.of(kv.getKey(), kv.getValue());
+          }
+        }).collect(Collectors.toMap(Pair::getLeft, Pair::getRight));
+  }
+
+  /**
+   * Sanitizes illegal field names in the schema using recursive calls to transformMap and transformList
+   */
+  private static Option<Schema> parseSanitizedAvroSchemaNoThrow(String schemaStr, String invalidCharMask) {
+    try {
+      ObjectMapper objectMapper = new ObjectMapper();

Review Comment:
   ObjectMapper instances are thread safe and it's recommended to reuse the object since it's heavy to construct. Can we make this a static instance?



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org