1
0

Restore 0.8.0 config keys with deprecated annotation (#3506)

Co-authored-by: Sagar Sumit <sagarsumit09@gmail.com>
Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
Udit Mehrotra
2021-08-19 13:36:40 -07:00
committed by GitHub
parent 37c29e75dc
commit c350d05dd3
137 changed files with 3460 additions and 1527 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.HOODIE_ARCHIVELOG_FOLDER_PROP;
import static org.apache.hudi.common.table.HoodieTableConfig.ARCHIVELOG_FOLDER;
/**
* 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(HOODIE_ARCHIVELOG_FOLDER_PROP.defaultValue())
.setArchiveLogFolder(ARCHIVELOG_FOLDER.defaultValue())
.initTable(jsc.hadoopConfiguration(), cfg.targetBasePath);
if (cfg.cleanInput) {

View File

@@ -34,13 +34,13 @@ import org.apache.hudi.common.table.view.TableFileSystemView;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.config.HoodieMemoryConfig;
import org.apache.hudi.io.storage.HoodieFileReader;
import org.apache.hudi.io.storage.HoodieFileReaderFactory;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.io.storage.HoodieFileReader;
import org.apache.hudi.io.storage.HoodieFileReaderFactory;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
@@ -88,7 +88,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.METADATA_VALIDATE_PROP.defaultValue(), false);
HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, HoodieMetadataConfig.VALIDATE_ENABLE.defaultValue(), false);
// Sort partition so we can pick last N partitions by default
Collections.sort(partitionPaths);
if (!partitionPaths.isEmpty()) {

View File

@@ -59,7 +59,7 @@ class SparkBulkInsertNode(config1: Config) extends DagNode[RDD[WriteStatus]] {
.option(DataSourceWriteOptions.ENABLE_ROW_WRITER.key(), String.valueOf(config.enableRowWriting()))
.option(DataSourceWriteOptions.COMMIT_METADATA_KEYPREFIX.key(), "deltastreamer.checkpoint.key")
.option("deltastreamer.checkpoint.key", context.getWriterContext.getHoodieTestSuiteWriter.getLastCheckpoint.orElse(""))
.option(HoodieWriteConfig.TABLE_NAME.key(), context.getHoodieTestSuiteWriter.getCfg.targetTableName)
.option(HoodieWriteConfig.TBL_NAME.key(), context.getHoodieTestSuiteWriter.getCfg.targetTableName)
.mode(saveMode)
.save(context.getHoodieTestSuiteWriter.getWriteConfig.getBasePath)
}

View File

@@ -58,7 +58,7 @@ class SparkInsertNode(config1: Config) extends DagNode[RDD[WriteStatus]] {
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
.option(DataSourceWriteOptions.COMMIT_METADATA_KEYPREFIX.key, "deltastreamer.checkpoint.key")
.option("deltastreamer.checkpoint.key", context.getWriterContext.getHoodieTestSuiteWriter.getLastCheckpoint.orElse(""))
.option(HoodieWriteConfig.TABLE_NAME.key, context.getHoodieTestSuiteWriter.getCfg.targetTableName)
.option(HoodieWriteConfig.TBL_NAME.key, context.getHoodieTestSuiteWriter.getCfg.targetTableName)
.mode(SaveMode.Overwrite)
.save(context.getHoodieTestSuiteWriter.getWriteConfig.getBasePath)
}

View File

@@ -58,7 +58,7 @@ class SparkUpsertNode(config1: Config) extends DagNode[RDD[WriteStatus]] {
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
.option(DataSourceWriteOptions.COMMIT_METADATA_KEYPREFIX.key, "deltastreamer.checkpoint.key")
.option("deltastreamer.checkpoint.key", context.getWriterContext.getHoodieTestSuiteWriter.getLastCheckpoint.orElse(""))
.option(HoodieWriteConfig.TABLE_NAME.key, context.getHoodieTestSuiteWriter.getCfg.targetTableName)
.option(HoodieWriteConfig.TBL_NAME.key, context.getHoodieTestSuiteWriter.getCfg.targetTableName)
.mode(SaveMode.Append)
.save(context.getHoodieTestSuiteWriter.getWriteConfig.getBasePath)
}

View File

@@ -169,7 +169,7 @@ public class TestHoodieTestSuiteJob extends UtilitiesTestBase {
props.setProperty(DataSourceWriteOptions.HIVE_DATABASE().key(), "testdb1");
props.setProperty(DataSourceWriteOptions.HIVE_TABLE().key(), "table1");
props.setProperty(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "datestr");
props.setProperty(DataSourceWriteOptions.KEYGENERATOR_CLASS().key(), TimestampBasedKeyGenerator.class.getName());
props.setProperty(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key(), TimestampBasedKeyGenerator.class.getName());
props.setProperty("hoodie.write.lock.provider", "org.apache.hudi.client.transaction.lock.ZookeeperBasedLockProvider");
props.setProperty("hoodie.write.lock.hivemetastore.database", "testdb1");