Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -38,11 +38,12 @@
import org.apache.hudi.utilities.checkpointing.InitialCheckPointProvider;
import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamerMetrics;
import org.apache.hudi.utilities.schema.DelegatingSchemaProvider;
import org.apache.hudi.utilities.schema.RowBasedSchemaProvider;
import org.apache.hudi.utilities.schema.SchemaPostProcessor;
import org.apache.hudi.utilities.schema.SchemaPostProcessor.Config;
import org.apache.hudi.utilities.schema.SchemaProvider;
import org.apache.hudi.utilities.schema.SchemaProviderWithPostProcessor;
import org.apache.hudi.utilities.schema.SparkAvroPostProcessor;
import org.apache.hudi.utilities.schema.RowBasedSchemaProvider;
import org.apache.hudi.utilities.sources.AvroKafkaSource;
import org.apache.hudi.utilities.sources.JsonKafkaSource;
import org.apache.hudi.utilities.sources.Source;
Expand Down Expand Up @@ -404,7 +405,7 @@ public static SchemaProvider getOriginalSchemaProvider(SchemaProvider schemaProv
}

public static SchemaProviderWithPostProcessor wrapSchemaProviderWithPostProcessor(SchemaProvider provider,
TypedProperties cfg, JavaSparkContext jssc) {
TypedProperties cfg, JavaSparkContext jssc, List<String> transformerClassNames) {

if (provider == null) {
return null;
Expand All @@ -413,14 +414,22 @@ public static SchemaProviderWithPostProcessor wrapSchemaProviderWithPostProcesso
if (provider instanceof SchemaProviderWithPostProcessor) {
return (SchemaProviderWithPostProcessor)provider;
}

String schemaPostProcessorClass = cfg.getString(Config.SCHEMA_POST_PROCESSOR_PROP, null);
boolean enableSparkAvroPostProcessor = Boolean.valueOf(cfg.getString(SparkAvroPostProcessor.Config.SPARK_AVRO_POST_PROCESSOR_PROP_ENABLE, "true"));

if (transformerClassNames != null && !transformerClassNames.isEmpty()
&& enableSparkAvroPostProcessor && StringUtils.isNullOrEmpty(schemaPostProcessorClass)) {
schemaPostProcessorClass = SparkAvroPostProcessor.class.getName();
}

return new SchemaProviderWithPostProcessor(provider,
Option.ofNullable(createSchemaPostProcessor(schemaPostProcessorClass, cfg, jssc)));
}

public static SchemaProvider createRowBasedSchemaProvider(StructType structType,
TypedProperties cfg, JavaSparkContext jssc) {
SchemaProvider rowSchemaProvider = new RowBasedSchemaProvider(structType);
return wrapSchemaProviderWithPostProcessor(rowSchemaProvider, cfg, jssc);
return wrapSchemaProviderWithPostProcessor(rowSchemaProvider, cfg, jssc, null);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -551,7 +551,7 @@ public DeltaSyncService(Config cfg, JavaSparkContext jssc, FileSystem fs, Config
this.props = properties.get();
LOG.info("Creating delta streamer with configs : " + props.toString());
this.schemaProvider = UtilHelpers.wrapSchemaProviderWithPostProcessor(
UtilHelpers.createSchemaProvider(cfg.schemaProviderClassName, props, jssc), props, jssc);
UtilHelpers.createSchemaProvider(cfg.schemaProviderClassName, props, jssc), props, jssc, cfg.transformerClassNames);

deltaSync = new DeltaSync(cfg, sparkSession, schemaProvider, props, jssc, fs, conf,
this::onInitializingWriteClient);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,47 @@
/*
* 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.schema;

import org.apache.hudi.AvroConversionUtils;
import org.apache.hudi.common.config.TypedProperties;

import org.apache.avro.Schema;
import org.apache.spark.api.java.JavaSparkContext;

/**
* HUDI-1343:Add standard schema postprocessor which would rewrite the schema using spark-avro conversion.
*/
public class SparkAvroPostProcessor extends SchemaPostProcessor {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can you reference the jira number as part of this class Javadoc so that developers would know why this class exist ?


public static class Config {
public static final String SPARK_AVRO_POST_PROCESSOR_PROP_ENABLE =
"hoodie.deltastreamer.schemaprovider.spark_avro_post_processor.enable";
}

public SparkAvroPostProcessor(TypedProperties props, JavaSparkContext jssc) {
super(props, jssc);
}

@Override
public Schema processSchema(Schema schema) {
return AvroConversionUtils.convertStructTypeToAvroSchema(
AvroConversionUtils.convertAvroSchemaToStructType(schema), RowBasedSchemaProvider.HOODIE_RECORD_STRUCT_NAME,
RowBasedSchemaProvider.HOODIE_RECORD_NAMESPACE);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,36 @@
/*
* 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.avro.Schema;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.utilities.schema.SchemaProvider;
import org.apache.spark.api.java.JavaSparkContext;

public class SparkAvroSchemaProvider extends SchemaProvider {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Does SparkAvroTestSchemaProvider look better?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Or define a static inner class in TestSchemaPostProcessor?


public SparkAvroSchemaProvider(TypedProperties props, JavaSparkContext jssc) {
super(props, jssc);
}

@Override
public Schema getSourceSchema() {
return new Schema.Parser().parse("{\"type\":\"record\",\"name\":\"test\",\"fields\":[{\"name\": \"day\", \"type\":\"string\"}]}");
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,15 +22,20 @@
import org.apache.hudi.utilities.schema.SchemaPostProcessor;
import org.apache.hudi.utilities.schema.SchemaPostProcessor.Config;
import org.apache.hudi.utilities.schema.SchemaProvider;
import org.apache.hudi.utilities.schema.SparkAvroPostProcessor;
import org.apache.hudi.utilities.testutils.UtilitiesTestBase;

import org.apache.avro.Schema;
import org.apache.avro.Schema.Type;
import org.apache.avro.SchemaBuilder;

import org.apache.hudi.utilities.transform.FlatteningTransformer;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

wrong place

import org.apache.spark.api.java.JavaSparkContext;
import org.junit.jupiter.api.Test;

import java.io.IOException;
import java.util.ArrayList;
import java.util.List;

import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
Expand All @@ -39,20 +44,51 @@ public class TestSchemaPostProcessor extends UtilitiesTestBase {

private TypedProperties properties = new TypedProperties();

private static String ORIGINAL_SCHEMA = "{\"name\":\"t3_biz_operation_t_driver\",\"type\":\"record\",\"fields\":[{\"name\":\"ums_id_\",\"type\":[\"null\",\"string\"],\"default\":null},"
+ "{\"name\":\"ums_ts_\",\"type\":[\"null\",\"string\"],\"default\":null}]}";

private static String RESULT_SCHEMA = "{\"type\":\"record\",\"name\":\"hoodie_source\",\"namespace\":\"hoodie.source\",\"fields\":[{\"name\":\"ums_id_\",\"type\":[\"string\",\"null\"]},"
+ "{\"name\":\"ums_ts_\",\"type\":[\"string\",\"null\"]}]}";

@Test
public void testPostProcessor() throws IOException {
properties.put(Config.SCHEMA_POST_PROCESSOR_PROP, DummySchemaPostProcessor.class.getName());
SchemaProvider provider =
UtilHelpers.wrapSchemaProviderWithPostProcessor(
UtilHelpers.createSchemaProvider(DummySchemaProvider.class.getName(), properties, jsc),
properties, jsc);
properties, jsc,null);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please add a backspace before ,null.


Schema schema = provider.getSourceSchema();
assertEquals(schema.getType(), Type.RECORD);
assertEquals(schema.getName(), "test");
assertNotNull(schema.getField("testString"));
}

@Test
public void testSparkAvro() throws IOException {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

IMO, this naming is not good.

properties.put(Config.SCHEMA_POST_PROCESSOR_PROP, SparkAvroPostProcessor.class.getName());
List<String> transformerClassNames = new ArrayList<>();
transformerClassNames.add(FlatteningTransformer.class.getName());

SchemaProvider provider =
UtilHelpers.wrapSchemaProviderWithPostProcessor(
UtilHelpers.createSchemaProvider(SparkAvroSchemaProvider.class.getName(), properties, jsc),
properties, jsc, transformerClassNames);

Schema schema = provider.getSourceSchema();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can you add a check that post processed schema is different from original schema.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@liujinhui1994 : Rest looks good. Can you add a assertion to ensure the 2 schemas are different in this case

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ok i will handle it

assertEquals(schema.getType(), Type.RECORD);
assertEquals(schema.getName(), "hoodie_source");
assertEquals(schema.getNamespace(), "hoodie.source");
assertNotNull(schema.getField("day"));
}

@Test
public void testSparkAvroSchema() throws IOException {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

WDYT about naming it to be testSparkAvroSchemaPostProcessor?

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) {
Expand Down