1
0

[HUDI-89] Add configOption & refactor all configs based on that (#2833)

Co-authored-by: Wenning Ding <wenningd@amazon.com>
This commit is contained in:
wenningd
2021-06-30 14:26:30 -07:00
committed by GitHub
parent 07e93de8b4
commit d412fb2fe6
173 changed files with 4277 additions and 3309 deletions

View File

@@ -61,7 +61,7 @@ import java.io.IOException;
import java.util.List;
import java.util.Map;
import static org.apache.hudi.common.table.HoodieTableConfig.DEFAULT_ARCHIVELOG_FOLDER;
import static org.apache.hudi.common.table.HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP;
/**
* This is the entry point for running a Hudi Test Suite. Although this class has similarities with {@link HoodieDeltaStreamer} this class does not extend it since do not want to create a dependency
@@ -111,7 +111,7 @@ public class HoodieTestSuiteJob {
metaClient = HoodieTableMetaClient.withPropertyBuilder()
.setTableType(cfg.tableType)
.setTableName(cfg.targetTableName)
.setArchiveLogFolder(DEFAULT_ARCHIVELOG_FOLDER)
.setArchiveLogFolder(HOODIE_ARCHIVELOG_FOLDER_PROP.defaultValue())
.initTable(jsc.hadoopConfiguration(), cfg.targetBasePath);
if (cfg.cleanInput) {

View File

@@ -79,8 +79,8 @@ public class ValidateDatasetNode extends DagNode<Boolean> {
log.debug("Listing all Micro batches to be validated :: " + fileStatus.getPath().toString());
}
String recordKeyField = context.getWriterContext().getProps().getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY());
String partitionPathField = context.getWriterContext().getProps().getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY());
String recordKeyField = context.getWriterContext().getProps().getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY().key());
String partitionPathField = context.getWriterContext().getProps().getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY().key());
// todo: fix hard coded fields from configs.
// read input and resolve insert, updates, etc.
Dataset<Row> inputDf = session.read().format("avro").load(inputPath);
@@ -112,8 +112,8 @@ public class ValidateDatasetNode extends DagNode<Boolean> {
}
if (config.isValidateHive()) {
String database = context.getWriterContext().getProps().getString(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY());
String tableName = context.getWriterContext().getProps().getString(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY());
String database = context.getWriterContext().getProps().getString(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY().key());
String tableName = context.getWriterContext().getProps().getString(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY().key());
log.warn("Validating hive table with db : " + database + " and table : " + tableName);
Dataset<Row> cowDf = session.sql("SELECT * FROM " + database + "." + tableName);
Dataset<Row> trimmedCowDf = cowDf.drop(HoodieRecord.COMMIT_TIME_METADATA_FIELD).drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD).drop(HoodieRecord.RECORD_KEY_METADATA_FIELD)

View File

@@ -87,7 +87,7 @@ public class DFSHoodieDatasetInputReader extends DFSDeltaInputReader {
// calls in metrics as they are not part of normal HUDI operation.
HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc);
List<String> partitionPaths = FSUtils.getAllPartitionPaths(engineContext, metaClient.getBasePath(),
HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false);
HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false);
// Sort partition so we can pick last N partitions by default
Collections.sort(partitionPaths);
if (!partitionPaths.isEmpty()) {
@@ -276,8 +276,8 @@ public class DFSHoodieDatasetInputReader extends DFSDeltaInputReader {
HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES)
.withReadBlocksLazily(true)
.withReverseReader(false)
.withBufferSize(HoodieMemoryConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE)
.withSpillableMapBasePath(HoodieMemoryConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH)
.withBufferSize(HoodieMemoryConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP.defaultValue())
.withSpillableMapBasePath(HoodieMemoryConfig.SPILLABLE_MAP_BASE_PATH_PROP.defaultValue())
.build();
// readAvro log files
Iterable<HoodieRecord<? extends HoodieRecordPayload>> iterable = () -> scanner.iterator();

View File

@@ -53,12 +53,12 @@ class SparkInsertNode(config1: Config) extends DagNode[RDD[WriteStatus]] {
context.getWriterContext.getSparkSession)
inputDF.write.format("hudi")
.options(DataSourceWriteOptions.translateSqlOptions(context.getWriterContext.getProps.asScala.toMap))
.option(DataSourceWriteOptions.TABLE_NAME_OPT_KEY, context.getHoodieTestSuiteWriter.getCfg.targetTableName)
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, context.getHoodieTestSuiteWriter.getCfg.tableType)
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
.option(DataSourceWriteOptions.COMMIT_METADATA_KEYPREFIX_OPT_KEY, "deltastreamer.checkpoint.key")
.option(DataSourceWriteOptions.TABLE_NAME_OPT_KEY.key, context.getHoodieTestSuiteWriter.getCfg.targetTableName)
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key, context.getHoodieTestSuiteWriter.getCfg.tableType)
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
.option(DataSourceWriteOptions.COMMIT_METADATA_KEYPREFIX_OPT_KEY.key, "deltastreamer.checkpoint.key")
.option("deltastreamer.checkpoint.key", context.getWriterContext.getHoodieTestSuiteWriter.getLastCheckpoint.orElse(""))
.option(HoodieWriteConfig.TABLE_NAME, context.getHoodieTestSuiteWriter.getCfg.targetTableName)
.option(HoodieWriteConfig.TABLE_NAME.key, context.getHoodieTestSuiteWriter.getCfg.targetTableName)
.mode(SaveMode.Overwrite)
.save(context.getHoodieTestSuiteWriter.getWriteConfig.getBasePath)
}

View File

@@ -53,12 +53,12 @@ class SparkUpsertNode(config1: Config) extends DagNode[RDD[WriteStatus]] {
context.getWriterContext.getSparkSession)
inputDF.write.format("hudi")
.options(DataSourceWriteOptions.translateSqlOptions(context.getWriterContext.getProps.asScala.toMap))
.option(DataSourceWriteOptions.TABLE_NAME_OPT_KEY, context.getHoodieTestSuiteWriter.getCfg.targetTableName)
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, context.getHoodieTestSuiteWriter.getCfg.tableType)
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
.option(DataSourceWriteOptions.COMMIT_METADATA_KEYPREFIX_OPT_KEY, "deltastreamer.checkpoint.key")
.option(DataSourceWriteOptions.TABLE_NAME_OPT_KEY.key, context.getHoodieTestSuiteWriter.getCfg.targetTableName)
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key, context.getHoodieTestSuiteWriter.getCfg.tableType)
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
.option(DataSourceWriteOptions.COMMIT_METADATA_KEYPREFIX_OPT_KEY.key, "deltastreamer.checkpoint.key")
.option("deltastreamer.checkpoint.key", context.getWriterContext.getHoodieTestSuiteWriter.getLastCheckpoint.orElse(""))
.option(HoodieWriteConfig.TABLE_NAME, context.getHoodieTestSuiteWriter.getCfg.targetTableName)
.option(HoodieWriteConfig.TABLE_NAME.key, context.getHoodieTestSuiteWriter.getCfg.targetTableName)
.mode(SaveMode.Append)
.save(context.getHoodieTestSuiteWriter.getWriteConfig.getBasePath)
}