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

@@ -204,10 +204,10 @@ public class HoodieLogFileCommand implements CommandMarker {
.getCommitTimeline().lastInstant().get().getTimestamp())
.withReadBlocksLazily(
Boolean.parseBoolean(
HoodieCompactionConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED.defaultValue()))
HoodieCompactionConfig.COMPACTION_LAZY_BLOCK_READ_ENABLE.defaultValue()))
.withReverseReader(
Boolean.parseBoolean(
HoodieCompactionConfig.COMPACTION_REVERSE_LOG_READ_ENABLED.defaultValue()))
HoodieCompactionConfig.COMPACTION_REVERSE_LOG_READ_ENABLE.defaultValue()))
.withBufferSize(HoodieMemoryConfig.MAX_DFS_STREAM_BUFFER_SIZE.defaultValue())
.withMaxMemorySizeInBytes(
HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES)

View File

@@ -18,8 +18,8 @@
package org.apache.hudi.cli.commands;
import org.apache.hudi.cli.DeDupeType;
import org.apache.hudi.DataSourceWriteOptions;
import org.apache.hudi.cli.DeDupeType;
import org.apache.hudi.cli.DedupeSparkJob;
import org.apache.hudi.cli.utils.SparkUtil;
import org.apache.hudi.client.SparkRDDWriteClient;
@@ -361,17 +361,17 @@ public class SparkMain {
TypedProperties properties = propsFilePath == null ? UtilHelpers.buildProperties(configs)
: UtilHelpers.readConfig(FSUtils.getFs(propsFilePath, jsc.hadoopConfiguration()), new Path(propsFilePath), configs).getConfig();
properties.setProperty(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH.key(), sourcePath);
properties.setProperty(HoodieBootstrapConfig.BASE_PATH.key(), sourcePath);
if (!StringUtils.isNullOrEmpty(keyGenerator) && KeyGeneratorType.getNames().contains(keyGenerator.toUpperCase(Locale.ROOT))) {
properties.setProperty(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_TYPE.key(), keyGenerator.toUpperCase(Locale.ROOT));
if (!StringUtils.isNullOrEmpty(keyGenerator) && KeyGeneratorType.getNames().contains(keyGenerator.toUpperCase(Locale.ROOT))) {
properties.setProperty(HoodieBootstrapConfig.KEYGEN_TYPE.key(), keyGenerator.toUpperCase(Locale.ROOT));
} else {
properties.setProperty(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS.key(), keyGenerator);
properties.setProperty(HoodieBootstrapConfig.KEYGEN_CLASS_NAME.key(), keyGenerator);
}
properties.setProperty(HoodieBootstrapConfig.FULL_BOOTSTRAP_INPUT_PROVIDER.key(), fullBootstrapInputProvider);
properties.setProperty(HoodieBootstrapConfig.BOOTSTRAP_PARALLELISM.key(), parallelism);
properties.setProperty(HoodieBootstrapConfig.BOOTSTRAP_MODE_SELECTOR.key(), selectorClass);
properties.setProperty(HoodieBootstrapConfig.FULL_BOOTSTRAP_INPUT_PROVIDER_CLASS_NAME.key(), fullBootstrapInputProvider);
properties.setProperty(HoodieBootstrapConfig.PARALLELISM_VALUE.key(), parallelism);
properties.setProperty(HoodieBootstrapConfig.MODE_SELECTOR_CLASS_NAME.key(), selectorClass);
properties.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD().key(), recordKeyCols);
properties.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), partitionFields);

View File

@@ -25,8 +25,7 @@ import org.apache.hudi.avro.HoodieAvroWriteSupport
import org.apache.hudi.client.SparkTaskContextSupplier
import org.apache.hudi.common.HoodieJsonPayload
import org.apache.hudi.common.bloom.{BloomFilter, BloomFilterFactory}
import org.apache.hudi.common.model.HoodieFileFormat
import org.apache.hudi.common.model.HoodieRecord
import org.apache.hudi.common.model.{HoodieFileFormat, HoodieRecord}
import org.apache.hudi.common.util.BaseFileUtils
import org.apache.hudi.config.{HoodieIndexConfig, HoodieStorageConfig}
import org.apache.hudi.io.storage.{HoodieAvroParquetConfig, HoodieParquetWriter}
@@ -43,10 +42,10 @@ object SparkHelpers {
def skipKeysAndWriteNewFile(instantTime: String, fs: FileSystem, sourceFile: Path, destinationFile: Path, keysToSkip: Set[String]) {
val sourceRecords = BaseFileUtils.getInstance(HoodieFileFormat.PARQUET).readAvroRecords(fs.getConf, sourceFile)
val schema: Schema = sourceRecords.get(0).getSchema
val filter: BloomFilter = BloomFilterFactory.createBloomFilter(HoodieIndexConfig.BLOOM_FILTER_NUM_ENTRIES.defaultValue.toInt, HoodieIndexConfig.BLOOM_FILTER_FPP.defaultValue.toDouble,
HoodieIndexConfig.HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES.defaultValue.toInt, HoodieIndexConfig.BLOOM_INDEX_FILTER_TYPE.defaultValue);
val filter: BloomFilter = BloomFilterFactory.createBloomFilter(HoodieIndexConfig.BLOOM_FILTER_NUM_ENTRIES_VALUE.defaultValue.toInt, HoodieIndexConfig.BLOOM_FILTER_FPP_VALUE.defaultValue.toDouble,
HoodieIndexConfig.BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES.defaultValue.toInt, HoodieIndexConfig.BLOOM_FILTER_TYPE.defaultValue);
val writeSupport: HoodieAvroWriteSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter(fs.getConf).convert(schema), schema, org.apache.hudi.common.util.Option.of(filter))
val parquetConfig: HoodieAvroParquetConfig = new HoodieAvroParquetConfig(writeSupport, CompressionCodecName.GZIP, HoodieStorageConfig.PARQUET_BLOCK_SIZE_BYTES.defaultValue.toInt, HoodieStorageConfig.PARQUET_PAGE_SIZE_BYTES.defaultValue.toInt, HoodieStorageConfig.PARQUET_FILE_MAX_BYTES.defaultValue.toInt, fs.getConf, HoodieStorageConfig.PARQUET_COMPRESSION_RATIO.defaultValue.toDouble)
val parquetConfig: HoodieAvroParquetConfig = new HoodieAvroParquetConfig(writeSupport, CompressionCodecName.GZIP, HoodieStorageConfig.PARQUET_BLOCK_SIZE.defaultValue.toInt, HoodieStorageConfig.PARQUET_PAGE_SIZE.defaultValue.toInt, HoodieStorageConfig.PARQUET_MAX_FILE_SIZE.defaultValue.toInt, fs.getConf, HoodieStorageConfig.PARQUET_COMPRESSION_RATIO_FRACTION.defaultValue.toDouble)
// Add current classLoad for config, if not will throw classNotFound of 'HoodieWrapperFileSystem'.
parquetConfig.getHadoopConf().setClassLoader(Thread.currentThread.getContextClassLoader)

View File

@@ -208,10 +208,10 @@ public class TestHoodieLogFileCommand extends AbstractShellIntegrationTest {
HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES)
.withReadBlocksLazily(
Boolean.parseBoolean(
HoodieCompactionConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED.defaultValue()))
HoodieCompactionConfig.COMPACTION_LAZY_BLOCK_READ_ENABLE.defaultValue()))
.withReverseReader(
Boolean.parseBoolean(
HoodieCompactionConfig.COMPACTION_REVERSE_LOG_READ_ENABLED.defaultValue()))
HoodieCompactionConfig.COMPACTION_REVERSE_LOG_READ_ENABLE.defaultValue()))
.withBufferSize(HoodieMemoryConfig.MAX_DFS_STREAM_BUFFER_SIZE.defaultValue())
.withSpillableMapBasePath(HoodieMemoryConfig.SPILLABLE_MAP_BASE_PATH.defaultValue())
.withDiskMapType(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.defaultValue())

View File

@@ -67,7 +67,7 @@ public class TestRepairsCommand extends AbstractShellIntegrationTest {
// Create table and connect
new TableCommand().createTable(
tablePath, tableName, HoodieTableType.COPY_ON_WRITE.name(),
HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP.defaultValue(), TimelineLayoutVersion.VERSION_1, "org.apache.hudi.common.model.HoodieAvroPayload");
HoodieTableConfig.ARCHIVELOG_FOLDER.defaultValue(), TimelineLayoutVersion.VERSION_1, "org.apache.hudi.common.model.HoodieAvroPayload");
}
/**

View File

@@ -112,6 +112,6 @@ public class TestUpgradeDowngradeCommand extends AbstractShellIntegrationTest {
HoodieConfig hoodieConfig = HoodieConfig.create(fsDataInputStream);
fsDataInputStream.close();
assertEquals(Integer.toString(HoodieTableVersion.ZERO.versionCode()), hoodieConfig
.getString(HoodieTableConfig.HOODIE_TABLE_VERSION_PROP));
.getString(HoodieTableConfig.VERSION));
}
}