the-other-tim-brown commented on code in PR #8010:
URL: https://github.com/apache/hudi/pull/8010#discussion_r1119408743
##########
hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java:
##########
@@ -720,10 +721,22 @@ public static Schema getNullSchema() {
* @return sanitized name
*/
public static String sanitizeName(String name) {
- if (name.substring(0, 1).matches(INVALID_AVRO_FIRST_CHAR_IN_NAMES)) {
- name = name.replaceFirst(INVALID_AVRO_FIRST_CHAR_IN_NAMES,
MASK_FOR_INVALID_CHARS_IN_NAMES);
+ return sanitizeName(name, MASK_FOR_INVALID_CHARS_IN_NAMES);
Review Comment:
Do we want some basic unit tests for this to make sure that behavior is
consistent before and after this change?
##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/SanitizationUtils.java:
##########
@@ -0,0 +1,199 @@
+/*
+ * 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 {
+
+ private static ObjectMapper OM = new ObjectMapper();
Review Comment:
nitpick: make this final?
##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/SanitizationUtils.java:
##########
@@ -0,0 +1,199 @@
+/*
+ * 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 {
+
+ private static ObjectMapper OM = new ObjectMapper();
+
+ 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 {
+ OM.enable(JsonParser.Feature.ALLOW_COMMENTS);
Review Comment:
This will create a new instance of an objectmapper so you will want to just
add this to the initial declaration at the top of the file
##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/SanitizationUtils.java:
##########
@@ -0,0 +1,199 @@
+/*
+ * 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 {
Review Comment:
Should we add some unit tests for this?
##########
hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestSourceFormatAdapter.java:
##########
@@ -0,0 +1,339 @@
+/*
+ * 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.deltastreamer;
+
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.utilities.schema.SchemaProvider;
+import org.apache.hudi.utilities.sources.InputBatch;
+import org.apache.hudi.utilities.sources.Source;
+import org.apache.hudi.utilities.sources.helpers.SanitizationUtils;
+
+import org.apache.avro.Schema;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.avro.SchemaConverters;
+import org.apache.spark.sql.types.ArrayType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.MapType;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class TestSourceFormatAdapter {
+ private static final String DUMMY_CHECKPOINT = "dummy_checkpoint";
+
+ private static SparkSession spark;
+ private static JavaSparkContext jsc;
+ private TestRowDataSource testRowDataSource;
+ private TestJsonDataSource testJsonDataSource;
+
+ @BeforeAll
+ public static void start() {
+ spark = SparkSession
+ .builder()
+ .master("local[*]")
+ .appName(TestSourceFormatAdapter.class.getName())
+ .config("spark.serializer",
"org.apache.spark.serializer.KryoSerializer")
+ .getOrCreate();
+ jsc = JavaSparkContext.fromSparkContext(spark.sparkContext());
+ }
+
+ @AfterAll
+ public static void shutdown() {
+ jsc.close();
+ spark.close();
+ }
+
+ // Forces to initialize object before every test.
+ @AfterEach
+ public void teardown() {
+ testRowDataSource = null;
+ testJsonDataSource = null;
+ }
+
+ private String sanitizeIfNeeded(String src, boolean shouldSanitize) {
+ return shouldSanitize ? HoodieAvroUtils.sanitizeName(src, "__") : src;
+ }
+
+ private StructType getSchemaWithProperNaming() {
+ StructType addressStruct = new StructType(new StructField[] {
+ new StructField("state", DataTypes.StringType, true, Metadata.empty()),
+ new StructField("street", DataTypes.StringType, true,
Metadata.empty()),
+ new StructField("zip", DataTypes.LongType, true, Metadata.empty()),
+ });
+
+ StructType personStruct = new StructType(new StructField[] {
+ new StructField("address", addressStruct, true, Metadata.empty()),
+ new StructField("name", DataTypes.StringType, true, Metadata.empty()),
+ new StructField("occupation", DataTypes.StringType, true,
Metadata.empty()),
+ new StructField("place", DataTypes.StringType, true, Metadata.empty())
+ });
+ return personStruct;
+ }
+
+ private StructType getSchemaWithBadAvroNamingForStructType(boolean
shouldSanitize) {
+ StructType addressStruct = new StructType(new StructField[] {
+ new StructField(sanitizeIfNeeded("@state.", shouldSanitize),
+ DataTypes.StringType, true, Metadata.empty()),
+ new StructField(sanitizeIfNeeded("@@stree@t@", shouldSanitize),
+ DataTypes.StringType, true, Metadata.empty()),
+ new StructField(sanitizeIfNeeded("8@_zip", shouldSanitize),
+ DataTypes.LongType, true, Metadata.empty())
+ });
+
+ StructType personStruct = new StructType(new StructField[] {
+ new StructField(sanitizeIfNeeded("@_addr*$ess", shouldSanitize),
+ addressStruct, true, Metadata.empty()),
+ new StructField(sanitizeIfNeeded("9name", shouldSanitize),
+ DataTypes.StringType, true, Metadata.empty()),
+ new StructField(sanitizeIfNeeded("_occu9pation", shouldSanitize),
+ DataTypes.StringType, true, Metadata.empty()),
+ new StructField(sanitizeIfNeeded("@plac.e.", shouldSanitize),
+ DataTypes.StringType, true, Metadata.empty())
+ });
+ return personStruct;
+ }
+
+ private StructType getSchemaWithBadAvroNamingForArrayType(boolean
shouldSanitize) {
+ StructType addressStruct = new StructType(new StructField[] {
+ new StructField(sanitizeIfNeeded("@state.", shouldSanitize),
+ DataTypes.StringType, true, Metadata.empty()),
+ new StructField(sanitizeIfNeeded("@@stree@t@", shouldSanitize),
+ DataTypes.StringType, true, Metadata.empty()),
+ new StructField(sanitizeIfNeeded("8@_zip", shouldSanitize),
+ DataTypes.LongType, true, Metadata.empty())
+ });
+
+ StructType personStruct = new StructType(new StructField[] {
+ new StructField(sanitizeIfNeeded("@name", shouldSanitize),
+ DataTypes.StringType, true, Metadata.empty()),
+ new StructField(sanitizeIfNeeded("@arr@", shouldSanitize),
+ new ArrayType(addressStruct, true), true, Metadata.empty())
+ });
+ return personStruct;
+ }
+
+ private StructType getSchemaWithBadAvroNamingForMapType(boolean
shouldSanitize) {
+ StructType addressStruct = new StructType(new StructField[] {
+ new StructField(sanitizeIfNeeded("@state.", shouldSanitize),
+ DataTypes.StringType, true, Metadata.empty()),
+ new StructField(sanitizeIfNeeded("@@stree@t@", shouldSanitize),
+ DataTypes.StringType, true, Metadata.empty()),
+ new StructField(sanitizeIfNeeded("8@_zip", shouldSanitize),
+ DataTypes.LongType, true, Metadata.empty())
+ });
+
+ StructType personStruct = new StructType(new StructField[] {
+ new StructField(sanitizeIfNeeded("@name", shouldSanitize),
+ DataTypes.StringType, true, Metadata.empty()),
+ new StructField(sanitizeIfNeeded("@map9", shouldSanitize),
+ new MapType(DataTypes.StringType, addressStruct, true), true,
Metadata.empty()),
+ });
+ return personStruct;
+ }
+
+ private void setupRowSource(Dataset<Row> ds) {
+ SchemaProvider nullSchemaProvider = new InputBatch.NullSchemaProvider();
+ InputBatch<Dataset<Row>> batch = new InputBatch<>(Option.of(ds),
DUMMY_CHECKPOINT, nullSchemaProvider);
+ testRowDataSource = new TestRowDataSource(new TypedProperties(), jsc,
spark, nullSchemaProvider, batch);
+ }
+
+ private void setupJsonSource(JavaRDD<String> ds, Schema schema) {
+ SchemaProvider basicSchemaProvider = new BasicSchemaProvider(schema);
+ InputBatch<JavaRDD<String>> batch = new InputBatch<>(Option.of(ds),
DUMMY_CHECKPOINT, basicSchemaProvider);
+ testJsonDataSource = new TestJsonDataSource(new TypedProperties(), jsc,
spark, basicSchemaProvider, batch);
+ }
+
+ private InputBatch<Dataset<Row>> fetchRowData(JavaRDD<String> rdd,
StructType inputSchema) {
+ TypedProperties typedProperties = new TypedProperties();
+ typedProperties.put(SanitizationUtils.Config.SANITIZE_SCHEMA_FIELD_NAMES,
true);
+
typedProperties.put(SanitizationUtils.Config.SCHEMA_FIELD_NAME_INVALID_CHAR_MASK,
"__");
+ setupRowSource(spark.read().schema(inputSchema).json(rdd));
+ SourceFormatAdapter sourceFormatAdapter = new
SourceFormatAdapter(testRowDataSource, Option.of(typedProperties));
+ return
sourceFormatAdapter.fetchNewDataInRowFormat(Option.of(DUMMY_CHECKPOINT), 10L);
+ }
+
+ private InputBatch<Dataset<Row>> fetchJsonData(JavaRDD<String> rdd,
StructType inputSchema) {
+ TypedProperties typedProperties = new TypedProperties();
+ typedProperties.put(SanitizationUtils.Config.SANITIZE_SCHEMA_FIELD_NAMES,
true);
+
typedProperties.put(SanitizationUtils.Config.SCHEMA_FIELD_NAME_INVALID_CHAR_MASK,
"__");
+ setupJsonSource(rdd, SchemaConverters.toAvroType(inputSchema, false,
"record", ""));
+ SourceFormatAdapter sourceFormatAdapter = new
SourceFormatAdapter(testJsonDataSource, Option.of(typedProperties));
+ return
sourceFormatAdapter.fetchNewDataInRowFormat(Option.of(DUMMY_CHECKPOINT), 10L);
+ }
+
+ @ParameterizedTest
+ @ValueSource(strings = {"row", "json"})
+ public void nestedTypeWithProperNaming(String sourceType) {
+ JavaRDD<String> rdd =
jsc.textFile("src/test/resources/data/avro_sanitization.json");
+ StructType inputSchema = getSchemaWithProperNaming();
+ InputBatch<Dataset<Row>> inputBatch;
+ switch (sourceType) {
+ case "row":
+ inputBatch = fetchRowData(rdd, inputSchema);
+ break;
+ case "json":
+ inputBatch = fetchJsonData(rdd, inputSchema);
+ break;
+ default:
+ throw new HoodieException("Invalid test source");
+ }
+ assertTrue(inputBatch.getBatch().isPresent());
+ Dataset<Row> ds = inputBatch.getBatch().get();
+ assertTrue(ds.collectAsList().size() == 2);
+ assertTrue(inputSchema.equals(ds.schema()));
+ JavaRDD<String> expectedData =
jsc.textFile("src/test/resources/data/avro_sanitization.json");
+ assertEquals(expectedData.collect(), ds.toJSON().collectAsList());
+ }
+
+ @ParameterizedTest
+ @ValueSource(strings = {"row", "json"})
+ public void structTypeAndBadNaming(String sourceType) {
+ JavaRDD<String> rdd =
jsc.textFile("src/test/resources/data/avro_sanitization_bad_naming_in.json");
+ InputBatch<Dataset<Row>> inputBatch;
+ switch (sourceType) {
+ case "row":
+ inputBatch = fetchRowData(rdd,
getSchemaWithBadAvroNamingForStructType(false));
+ break;
+ case "json":
+ inputBatch = fetchJsonData(rdd,
getSchemaWithBadAvroNamingForStructType(true));
+ break;
+ default:
+ throw new HoodieException("Invalid test source");
+ }
+ assertTrue(inputBatch.getBatch().isPresent());
+ Dataset<Row> ds = inputBatch.getBatch().get();
+ assertTrue(ds.collectAsList().size() == 2);
+
assertTrue(getSchemaWithBadAvroNamingForStructType(true).equals(ds.schema()));
+ JavaRDD<String> expectedData =
jsc.textFile("src/test/resources/data/avro_sanitization_bad_naming_out.json");
+ assertEquals(expectedData.collect(), ds.toJSON().collectAsList());
+ }
+
+ @ParameterizedTest
+ @ValueSource(strings = {"row", "json"})
+ public void arrayTypeAndBadNaming(String sourceType) {
+ JavaRDD<String> rdd =
jsc.textFile("src/test/resources/data/avro_sanitization_bad_naming_nested_array_in.json");
+ InputBatch<Dataset<Row>> inputBatch;
+ switch (sourceType) {
+ case "row":
+ inputBatch = fetchRowData(rdd,
getSchemaWithBadAvroNamingForArrayType(false));
+ break;
+ case "json":
+ inputBatch = fetchJsonData(rdd,
getSchemaWithBadAvroNamingForArrayType(true));
+ break;
+ default:
+ throw new HoodieException("Invalid test source");
+ }
+ assertTrue(inputBatch.getBatch().isPresent());
+ Dataset<Row> ds = inputBatch.getBatch().get();
+ assertTrue(ds.collectAsList().size() == 2);
+ System.out.println(getSchemaWithBadAvroNamingForArrayType(true));
Review Comment:
Why print in tests?
--
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]