[HUDI-1130] hudi-test-suite support for schema evolution (can be triggered on any insert/upsert DAG node).
This commit is contained in:
committed by
n3nash
parent
53d1e55110
commit
fec7cd3c97
@@ -26,21 +26,19 @@ import org.apache.hadoop.fs.FileSystem;
|
|||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.hive.conf.HiveConf;
|
import org.apache.hadoop.hive.conf.HiveConf;
|
||||||
import org.apache.hudi.DataSourceUtils;
|
import org.apache.hudi.DataSourceUtils;
|
||||||
import org.apache.hudi.common.config.SerializableConfiguration;
|
|
||||||
import org.apache.hudi.common.config.TypedProperties;
|
import org.apache.hudi.common.config.TypedProperties;
|
||||||
import org.apache.hudi.common.fs.FSUtils;
|
import org.apache.hudi.common.fs.FSUtils;
|
||||||
import org.apache.hudi.common.model.HoodieTableType;
|
import org.apache.hudi.common.model.HoodieTableType;
|
||||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||||
import org.apache.hudi.common.util.ReflectionUtils;
|
import org.apache.hudi.common.util.ReflectionUtils;
|
||||||
import org.apache.hudi.exception.HoodieException;
|
import org.apache.hudi.exception.HoodieException;
|
||||||
import org.apache.hudi.integ.testsuite.configuration.DFSDeltaConfig;
|
|
||||||
import org.apache.hudi.integ.testsuite.dag.DagUtils;
|
import org.apache.hudi.integ.testsuite.dag.DagUtils;
|
||||||
import org.apache.hudi.integ.testsuite.dag.WorkflowDag;
|
import org.apache.hudi.integ.testsuite.dag.WorkflowDag;
|
||||||
import org.apache.hudi.integ.testsuite.dag.WorkflowDagGenerator;
|
import org.apache.hudi.integ.testsuite.dag.WorkflowDagGenerator;
|
||||||
import org.apache.hudi.integ.testsuite.dag.scheduler.DagScheduler;
|
import org.apache.hudi.integ.testsuite.dag.scheduler.DagScheduler;
|
||||||
import org.apache.hudi.integ.testsuite.generator.DeltaGenerator;
|
|
||||||
import org.apache.hudi.integ.testsuite.reader.DeltaInputType;
|
import org.apache.hudi.integ.testsuite.reader.DeltaInputType;
|
||||||
import org.apache.hudi.integ.testsuite.writer.DeltaOutputMode;
|
import org.apache.hudi.integ.testsuite.writer.DeltaOutputMode;
|
||||||
|
import org.apache.hudi.integ.testsuite.dag.WriterContext;
|
||||||
import org.apache.hudi.keygen.BuiltinKeyGenerator;
|
import org.apache.hudi.keygen.BuiltinKeyGenerator;
|
||||||
import org.apache.hudi.utilities.UtilHelpers;
|
import org.apache.hudi.utilities.UtilHelpers;
|
||||||
import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer;
|
import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer;
|
||||||
@@ -64,10 +62,6 @@ public class HoodieTestSuiteJob {
|
|||||||
* Bag of properties with source, hoodie client, key generator etc.
|
* Bag of properties with source, hoodie client, key generator etc.
|
||||||
*/
|
*/
|
||||||
TypedProperties props;
|
TypedProperties props;
|
||||||
/**
|
|
||||||
* Schema provider that supplies the command for writing out the generated payloads.
|
|
||||||
*/
|
|
||||||
private transient SchemaProvider schemaProvider;
|
|
||||||
/**
|
/**
|
||||||
* Filesystem used.
|
* Filesystem used.
|
||||||
*/
|
*/
|
||||||
@@ -95,7 +89,6 @@ public class HoodieTestSuiteJob {
|
|||||||
this.fs = FSUtils.getFs(cfg.inputBasePath, jsc.hadoopConfiguration());
|
this.fs = FSUtils.getFs(cfg.inputBasePath, jsc.hadoopConfiguration());
|
||||||
this.props = UtilHelpers.readConfig(fs, new Path(cfg.propsFilePath), cfg.configs).getConfig();
|
this.props = UtilHelpers.readConfig(fs, new Path(cfg.propsFilePath), cfg.configs).getConfig();
|
||||||
log.info("Creating workload generator with configs : {}", props.toString());
|
log.info("Creating workload generator with configs : {}", props.toString());
|
||||||
this.schemaProvider = UtilHelpers.createSchemaProvider(cfg.schemaProviderClassName, props, jsc);
|
|
||||||
this.hiveConf = getDefaultHiveConf(jsc.hadoopConfiguration());
|
this.hiveConf = getDefaultHiveConf(jsc.hadoopConfiguration());
|
||||||
this.keyGenerator = (BuiltinKeyGenerator) DataSourceUtils.createKeyGenerator(props);
|
this.keyGenerator = (BuiltinKeyGenerator) DataSourceUtils.createKeyGenerator(props);
|
||||||
|
|
||||||
@@ -138,13 +131,9 @@ public class HoodieTestSuiteJob {
|
|||||||
WorkflowDag workflowDag = createWorkflowDag();
|
WorkflowDag workflowDag = createWorkflowDag();
|
||||||
log.info("Workflow Dag => " + DagUtils.convertDagToYaml(workflowDag));
|
log.info("Workflow Dag => " + DagUtils.convertDagToYaml(workflowDag));
|
||||||
long startTime = System.currentTimeMillis();
|
long startTime = System.currentTimeMillis();
|
||||||
String schemaStr = schemaProvider.getSourceSchema().toString();
|
WriterContext writerContext = new WriterContext(jsc, props, cfg, keyGenerator, sparkSession);
|
||||||
final HoodieTestSuiteWriter writer = new HoodieTestSuiteWriter(jsc, props, cfg, schemaStr);
|
writerContext.initContext(jsc);
|
||||||
final DeltaGenerator deltaGenerator = new DeltaGenerator(
|
DagScheduler dagScheduler = new DagScheduler(workflowDag, writerContext);
|
||||||
new DFSDeltaConfig(DeltaOutputMode.valueOf(cfg.outputTypeName), DeltaInputType.valueOf(cfg.inputFormatName),
|
|
||||||
new SerializableConfiguration(jsc.hadoopConfiguration()), cfg.inputBasePath, cfg.targetBasePath,
|
|
||||||
schemaStr, cfg.limitFileSize), jsc, sparkSession, schemaStr, keyGenerator);
|
|
||||||
DagScheduler dagScheduler = new DagScheduler(workflowDag, writer, deltaGenerator);
|
|
||||||
dagScheduler.schedule();
|
dagScheduler.schedule();
|
||||||
log.info("Finished scheduling all tasks, Time taken {}", System.currentTimeMillis() - startTime);
|
log.info("Finished scheduling all tasks, Time taken {}", System.currentTimeMillis() - startTime);
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
|
|||||||
@@ -82,6 +82,7 @@ public class DeltaConfig implements Serializable {
|
|||||||
private static String DISABLE_GENERATE = "disable_generate";
|
private static String DISABLE_GENERATE = "disable_generate";
|
||||||
private static String DISABLE_INGEST = "disable_ingest";
|
private static String DISABLE_INGEST = "disable_ingest";
|
||||||
private static String HIVE_LOCAL = "hive_local";
|
private static String HIVE_LOCAL = "hive_local";
|
||||||
|
private static String REINIT_CONTEXT = "reinitialize_context";
|
||||||
|
|
||||||
private Map<String, Object> configsMap;
|
private Map<String, Object> configsMap;
|
||||||
|
|
||||||
@@ -133,6 +134,10 @@ public class DeltaConfig implements Serializable {
|
|||||||
return Boolean.valueOf(configsMap.getOrDefault(DISABLE_INGEST, false).toString());
|
return Boolean.valueOf(configsMap.getOrDefault(DISABLE_INGEST, false).toString());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public boolean getReinitContext() {
|
||||||
|
return Boolean.valueOf(configsMap.getOrDefault(REINIT_CONTEXT, false).toString());
|
||||||
|
}
|
||||||
|
|
||||||
public Map<String, Object> getOtherConfigs() {
|
public Map<String, Object> getOtherConfigs() {
|
||||||
if (configsMap == null) {
|
if (configsMap == null) {
|
||||||
return new HashMap<>();
|
return new HashMap<>();
|
||||||
@@ -222,6 +227,11 @@ public class DeltaConfig implements Serializable {
|
|||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Builder reinitializeContext(boolean reinitContext) {
|
||||||
|
this.configsMap.put(REINIT_CONTEXT, reinitContext);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
public Builder withConfig(String name, Object value) {
|
public Builder withConfig(String name, Object value) {
|
||||||
this.configsMap.put(name, value);
|
this.configsMap.put(name, value);
|
||||||
return this;
|
return this;
|
||||||
|
|||||||
@@ -30,25 +30,27 @@ import org.apache.spark.api.java.JavaSparkContext;
|
|||||||
*/
|
*/
|
||||||
public class ExecutionContext implements Serializable {
|
public class ExecutionContext implements Serializable {
|
||||||
|
|
||||||
private HoodieTestSuiteWriter hoodieTestSuiteWriter;
|
private WriterContext writerContext;
|
||||||
private DeltaGenerator deltaGenerator;
|
|
||||||
private transient JavaSparkContext jsc;
|
private transient JavaSparkContext jsc;
|
||||||
|
|
||||||
public ExecutionContext(JavaSparkContext jsc, HoodieTestSuiteWriter hoodieTestSuiteWriter, DeltaGenerator deltaGenerator) {
|
public ExecutionContext(JavaSparkContext jsc, WriterContext writerContext) {
|
||||||
this.hoodieTestSuiteWriter = hoodieTestSuiteWriter;
|
this.writerContext = writerContext;
|
||||||
this.deltaGenerator = deltaGenerator;
|
|
||||||
this.jsc = jsc;
|
this.jsc = jsc;
|
||||||
}
|
}
|
||||||
|
|
||||||
public HoodieTestSuiteWriter getHoodieTestSuiteWriter() {
|
public HoodieTestSuiteWriter getHoodieTestSuiteWriter() {
|
||||||
return hoodieTestSuiteWriter;
|
return writerContext.getHoodieTestSuiteWriter();
|
||||||
}
|
}
|
||||||
|
|
||||||
public DeltaGenerator getDeltaGenerator() {
|
public DeltaGenerator getDeltaGenerator() {
|
||||||
return deltaGenerator;
|
return writerContext.getDeltaGenerator();
|
||||||
}
|
}
|
||||||
|
|
||||||
public JavaSparkContext getJsc() {
|
public JavaSparkContext getJsc() {
|
||||||
return jsc;
|
return jsc;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public WriterContext getWriterContext() {
|
||||||
|
return writerContext;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -0,0 +1,102 @@
|
|||||||
|
/*
|
||||||
|
* 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.integ.testsuite.dag;
|
||||||
|
|
||||||
|
import org.apache.hudi.common.config.SerializableConfiguration;
|
||||||
|
import org.apache.hudi.common.config.TypedProperties;
|
||||||
|
import org.apache.hudi.exception.HoodieException;
|
||||||
|
import org.apache.hudi.integ.testsuite.HoodieTestSuiteWriter;
|
||||||
|
import org.apache.hudi.integ.testsuite.configuration.DFSDeltaConfig;
|
||||||
|
import org.apache.hudi.integ.testsuite.reader.DeltaInputType;
|
||||||
|
import org.apache.hudi.integ.testsuite.writer.DeltaOutputMode;
|
||||||
|
import org.apache.hudi.keygen.BuiltinKeyGenerator;
|
||||||
|
import org.apache.hudi.integ.testsuite.generator.DeltaGenerator;
|
||||||
|
import org.apache.hudi.integ.testsuite.HoodieTestSuiteJob.HoodieTestSuiteConfig;
|
||||||
|
import org.apache.hudi.utilities.UtilHelpers;
|
||||||
|
import org.apache.hudi.utilities.schema.SchemaProvider;
|
||||||
|
import org.apache.log4j.LogManager;
|
||||||
|
import org.apache.log4j.Logger;
|
||||||
|
import org.apache.spark.api.java.JavaSparkContext;
|
||||||
|
import org.apache.spark.sql.SparkSession;
|
||||||
|
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* WriterContext wraps the delta writer/data generator related configuration needed
|
||||||
|
* to init/reinit.
|
||||||
|
*/
|
||||||
|
public class WriterContext {
|
||||||
|
|
||||||
|
protected static Logger log = LogManager.getLogger(WriterContext.class);
|
||||||
|
|
||||||
|
private final HoodieTestSuiteConfig cfg;
|
||||||
|
private TypedProperties props;
|
||||||
|
private HoodieTestSuiteWriter hoodieTestSuiteWriter;
|
||||||
|
private DeltaGenerator deltaGenerator;
|
||||||
|
private transient SchemaProvider schemaProvider;
|
||||||
|
private BuiltinKeyGenerator keyGenerator;
|
||||||
|
private transient SparkSession sparkSession;
|
||||||
|
private transient JavaSparkContext jsc;
|
||||||
|
public WriterContext(JavaSparkContext jsc, TypedProperties props, HoodieTestSuiteConfig cfg,
|
||||||
|
BuiltinKeyGenerator keyGenerator, SparkSession sparkSession) {
|
||||||
|
this.cfg = cfg;
|
||||||
|
this.props = props;
|
||||||
|
this.keyGenerator = keyGenerator;
|
||||||
|
this.sparkSession = sparkSession;
|
||||||
|
this.jsc = jsc;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void initContext(JavaSparkContext jsc) throws HoodieException {
|
||||||
|
try {
|
||||||
|
this.schemaProvider = UtilHelpers.createSchemaProvider(cfg.schemaProviderClassName, props, jsc);
|
||||||
|
String schemaStr = schemaProvider.getSourceSchema().toString();
|
||||||
|
this.hoodieTestSuiteWriter = new HoodieTestSuiteWriter(jsc, props, cfg, schemaStr);
|
||||||
|
this.deltaGenerator = new DeltaGenerator(
|
||||||
|
new DFSDeltaConfig(DeltaOutputMode.valueOf(cfg.outputTypeName), DeltaInputType.valueOf(cfg.inputFormatName),
|
||||||
|
new SerializableConfiguration(jsc.hadoopConfiguration()), cfg.inputBasePath, cfg.targetBasePath,
|
||||||
|
schemaStr, cfg.limitFileSize),
|
||||||
|
jsc, sparkSession, schemaStr, keyGenerator);
|
||||||
|
log.info(String.format("Initialized writerContext with: %s", schemaStr));
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new HoodieException("Failed to reinitialize writerContext", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public void reinitContext(Map<String, Object> newConfig) throws HoodieException {
|
||||||
|
// update props with any config overrides.
|
||||||
|
for (Map.Entry<String, Object> e : newConfig.entrySet()) {
|
||||||
|
if (this.props.containsKey(e.getKey())) {
|
||||||
|
this.props.setProperty(e.getKey(), e.getValue().toString());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
initContext(jsc);
|
||||||
|
}
|
||||||
|
|
||||||
|
public HoodieTestSuiteWriter getHoodieTestSuiteWriter() {
|
||||||
|
return hoodieTestSuiteWriter;
|
||||||
|
}
|
||||||
|
|
||||||
|
public DeltaGenerator getDeltaGenerator() {
|
||||||
|
return deltaGenerator;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String toString() {
|
||||||
|
return this.hoodieTestSuiteWriter.toString() + "\n" + this.deltaGenerator.toString() + "\n";
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -37,6 +37,11 @@ public class InsertNode extends DagNode<JavaRDD<WriteStatus>> {
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void execute(ExecutionContext executionContext) throws Exception {
|
public void execute(ExecutionContext executionContext) throws Exception {
|
||||||
|
// if the insert node has schema override set, reinitialize the table with new schema.
|
||||||
|
if (this.config.getReinitContext()) {
|
||||||
|
log.info(String.format("Reinitializing table with %s", this.config.getOtherConfigs().toString()));
|
||||||
|
executionContext.getWriterContext().reinitContext(this.config.getOtherConfigs());
|
||||||
|
}
|
||||||
generate(executionContext.getDeltaGenerator());
|
generate(executionContext.getDeltaGenerator());
|
||||||
log.info("Configs : {}", this.config);
|
log.info("Configs : {}", this.config);
|
||||||
if (!config.isDisableIngest()) {
|
if (!config.isDisableIngest()) {
|
||||||
|
|||||||
@@ -30,10 +30,9 @@ import java.util.concurrent.Future;
|
|||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import org.apache.hudi.exception.HoodieException;
|
import org.apache.hudi.exception.HoodieException;
|
||||||
import org.apache.hudi.integ.testsuite.dag.nodes.DagNode;
|
import org.apache.hudi.integ.testsuite.dag.nodes.DagNode;
|
||||||
import org.apache.hudi.integ.testsuite.HoodieTestSuiteWriter;
|
|
||||||
import org.apache.hudi.integ.testsuite.dag.ExecutionContext;
|
import org.apache.hudi.integ.testsuite.dag.ExecutionContext;
|
||||||
import org.apache.hudi.integ.testsuite.dag.WorkflowDag;
|
import org.apache.hudi.integ.testsuite.dag.WorkflowDag;
|
||||||
import org.apache.hudi.integ.testsuite.generator.DeltaGenerator;
|
import org.apache.hudi.integ.testsuite.dag.WriterContext;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
@@ -47,9 +46,9 @@ public class DagScheduler {
|
|||||||
private WorkflowDag workflowDag;
|
private WorkflowDag workflowDag;
|
||||||
private ExecutionContext executionContext;
|
private ExecutionContext executionContext;
|
||||||
|
|
||||||
public DagScheduler(WorkflowDag workflowDag, HoodieTestSuiteWriter hoodieTestSuiteWriter, DeltaGenerator deltaGenerator) {
|
public DagScheduler(WorkflowDag workflowDag, WriterContext writerContext) {
|
||||||
this.workflowDag = workflowDag;
|
this.workflowDag = workflowDag;
|
||||||
this.executionContext = new ExecutionContext(null, hoodieTestSuiteWriter, deltaGenerator);
|
this.executionContext = new ExecutionContext(null, writerContext);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|||||||
Reference in New Issue
Block a user