1
0

[HUDI-3568] Introduce ChainedSchemaPostProcessor to support setting multi processors at once (#4968)

This commit is contained in:
wangxianghu
2022-03-09 11:16:22 +04:00
committed by GitHub
parent 4324e874ae
commit 548000b0d6
5 changed files with 178 additions and 25 deletions

View File

@@ -0,0 +1,39 @@
/*
* 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;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.utilities.schema.SchemaPostProcessor;
import org.apache.avro.Schema;
import org.apache.avro.SchemaBuilder;
import org.apache.spark.api.java.JavaSparkContext;
public class DummySchemaPostProcessor extends SchemaPostProcessor {
public DummySchemaPostProcessor(TypedProperties props, JavaSparkContext jssc) {
super(props, jssc);
}
@Override
public Schema processSchema(Schema schema) {
return SchemaBuilder.record("test").fields().optionalString("testString").endRecord();
}
}

View File

@@ -29,8 +29,6 @@ import org.apache.hudi.utilities.transform.FlatteningTransformer;
import org.apache.avro.Schema;
import org.apache.avro.Schema.Type;
import org.apache.avro.SchemaBuilder;
import org.apache.spark.api.java.JavaSparkContext;
import org.junit.jupiter.api.Test;
import java.io.IOException;
@@ -39,6 +37,7 @@ import java.util.List;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
public class TestSchemaPostProcessor extends UtilitiesTestBase {
@@ -90,22 +89,37 @@ public class TestSchemaPostProcessor extends UtilitiesTestBase {
assertNotNull(targetSchema.getField("_hoodie_is_deleted"));
}
@Test
public void testChainedSchemaPostProcessor() {
// DeleteSupportSchemaPostProcessor first, DummySchemaPostProcessor second
properties.put(Config.SCHEMA_POST_PROCESSOR_PROP,
"org.apache.hudi.utilities.schema.DeleteSupportSchemaPostProcessor,org.apache.hudi.utilities.DummySchemaPostProcessor");
SchemaPostProcessor processor = UtilHelpers.createSchemaPostProcessor(properties.getString(Config.SCHEMA_POST_PROCESSOR_PROP), properties, jsc);
Schema schema = new Schema.Parser().parse(ORIGINAL_SCHEMA);
Schema targetSchema = processor.processSchema(schema);
assertNull(targetSchema.getField("ums_id_"));
assertNull(targetSchema.getField("_hoodie_is_deleted"));
assertNotNull(targetSchema.getField("testString"));
// DummySchemaPostProcessor first, DeleteSupportSchemaPostProcessor second
properties.put(Config.SCHEMA_POST_PROCESSOR_PROP,
"org.apache.hudi.utilities.DummySchemaPostProcessor,org.apache.hudi.utilities.schema.DeleteSupportSchemaPostProcessor");
processor = UtilHelpers.createSchemaPostProcessor(properties.getString(Config.SCHEMA_POST_PROCESSOR_PROP), properties, jsc);
schema = new Schema.Parser().parse(ORIGINAL_SCHEMA);
targetSchema = processor.processSchema(schema);
assertNull(targetSchema.getField("ums_id_"));
assertNotNull(targetSchema.getField("_hoodie_is_deleted"));
assertNotNull(targetSchema.getField("testString"));
}
@Test
public void testSparkAvroSchema() throws IOException {
SparkAvroPostProcessor processor = new SparkAvroPostProcessor(properties, null);
Schema schema = new Schema.Parser().parse(ORIGINAL_SCHEMA);
assertEquals(processor.processSchema(schema).toString(), RESULT_SCHEMA);
}
public static class DummySchemaPostProcessor extends SchemaPostProcessor {
public DummySchemaPostProcessor(TypedProperties props, JavaSparkContext jssc) {
super(props, jssc);
}
@Override
public Schema processSchema(Schema schema) {
return SchemaBuilder.record("test").fields().optionalString("testString").endRecord();
}
}
}