lokeshj1703 commented on code in PR #8574:
URL: https://github.com/apache/hudi/pull/8574#discussion_r1204125671
##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/transform/ChainedTransformer.java:
##########
@@ -109,6 +131,28 @@ private void validateIdentifier(String id, Set<String>
identifiers, String confi
}
}
+ private Option<StructType> validateAndGetTransformedSchema(TransformerInfo
transformerInfo, Dataset<Row> dataset, Option<StructType> incomingStructOpt,
+ JavaSparkContext
jsc, SparkSession sparkSession, TypedProperties properties) {
+ if (!incomingStructOpt.isPresent()) {
Review Comment:
Removed Option from tranformedSchema API so removed this code
##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/transform/Transformer.java:
##########
@@ -45,4 +47,11 @@ public interface Transformer {
*/
@PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
Dataset<Row> apply(JavaSparkContext jsc, SparkSession sparkSession,
Dataset<Row> rowDataset, TypedProperties properties);
+
+ @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
+ default Option<StructType> transformedSchema(JavaSparkContext jsc,
SparkSession sparkSession, StructType incomingStruct, TypedProperties
properties) {
Review Comment:
Addressed
##########
hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestTransformer.java:
##########
@@ -92,4 +211,95 @@ public Dataset<Row> apply(JavaSparkContext jsc,
SparkSession sparkSession, Datas
return rowDataset.withColumn("timestamp",
functions.col("timestamp").multiply(multiplier).plus(increment));
}
}
+
+ /**
+ * Provides a transformedSchema implementation for FlatteningTransformer.
+ */
+ public static class FlatteningTransformerWithTransformedSchema extends
FlatteningTransformer {
+
+ @Override
+ public Option<StructType> transformedSchema(JavaSparkContext jsc,
SparkSession sparkSession, StructType incomingStruct, TypedProperties
properties) {
+ String flattenedSelect = flattenSchema(incomingStruct, null);
+ String[] cols = flattenedSelect.split(",");
+ List<Pair<String, String>> replacements = new LinkedList<>();
+ for (String col : cols) {
+ String[] names = col.split(" as ");
+ if (!names[0].equals(names[1])) {
+ replacements.add(Pair.of(names[0], names[1]));
+ }
+ }
+
+ List<StructField> incomingFields =
Arrays.asList(incomingStruct.fields());
+ List<StructField> transformedFields = new LinkedList<>(incomingFields);
+ Set<StructField> fieldsToRemove = new HashSet<>();
+ for (Pair<String, String> replacement : replacements) {
+ String fieldToRemoveName = replacement.getKey().replaceAll("\\..*",
"");
+ StructField fieldToAdd = StructUtils.getField(incomingStruct,
replacement.getKey()).get();
+ StructField fieldToRemove = transformedFields.stream().filter(f ->
f.name().equals(fieldToRemoveName)).findAny().get();
+ fieldsToRemove.add(fieldToRemove);
+ transformedFields.add(transformedFields.indexOf(fieldToRemove), new
StructField(replacement.getKey().replaceAll("\\.", "_"),
+ fieldToAdd.dataType(), fieldToAdd.nullable(),
fieldToAdd.metadata()));
+ }
+ transformedFields.removeAll(fieldsToRemove);
+
+ return Option.of(new StructType(transformedFields.toArray(new
StructField[0])));
+ }
+ }
+
+ /**
+ * Does not provide transformedSchema implementation for
FlatteningTransformer.
+ */
+ public static class FlatteningTransformerWithoutTransformedSchema extends
FlatteningTransformer {
+ @Override
+ public Option<StructType> transformedSchema(JavaSparkContext jsc,
SparkSession sparkSession, StructType incomingStruct, TypedProperties
properties) {
+ return Option.empty();
+ }
+ }
+
+ /**
+ * Adds a new column named random in the dataset.
+ */
+ public static class AddColumnTransformer implements Transformer {
Review Comment:
Addressed
--
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]