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

@@ -16,7 +16,6 @@
* limitations under the License.
*/
import org.apache.hadoop.fs.Path;
import org.apache.hudi.DataSourceReadOptions;
import org.apache.hudi.DataSourceWriteOptions;
import org.apache.hudi.HoodieDataSourceHelpers;
@@ -33,6 +32,7 @@ import org.apache.hudi.keygen.SimpleKeyGenerator;
import com.beust.jcommander.JCommander;
import com.beust.jcommander.Parameter;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaSparkContext;
@@ -150,9 +150,9 @@ public class HoodieJavaApp {
// use to combine duplicate records in input/with disk val
.option(DataSourceWriteOptions.PRECOMBINE_FIELD().key(), "timestamp")
// Used by hive sync and queries
.option(HoodieWriteConfig.TABLE_NAME.key(), tableName)
.option(HoodieWriteConfig.TBL_NAME.key(), tableName)
// Add Key Extractor
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS().key(),
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key(),
nonPartitionedTable ? NonpartitionedKeyGenerator.class.getCanonicalName()
: SimpleKeyGenerator.class.getCanonicalName())
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE().key(), "false")
@@ -179,13 +179,13 @@ public class HoodieJavaApp {
.option(DataSourceWriteOptions.RECORDKEY_FIELD().key(), "_row_key")
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), "partition")
.option(DataSourceWriteOptions.PRECOMBINE_FIELD().key(), "timestamp")
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS().key(),
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key(),
nonPartitionedTable ? NonpartitionedKeyGenerator.class.getCanonicalName()
: SimpleKeyGenerator.class.getCanonicalName()) // Add Key Extractor
.option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS.key(), "1")
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE().key(), "false")
.option(DataSourceWriteOptions.ASYNC_CLUSTERING_ENABLE().key(), "true")
.option(HoodieWriteConfig.TABLE_NAME.key(), tableName).mode(SaveMode.Append);
.option(HoodieWriteConfig.TBL_NAME.key(), tableName).mode(SaveMode.Append);
updateHiveSyncConfig(writer);
writer.save(tablePath);
@@ -207,13 +207,13 @@ public class HoodieJavaApp {
.option(DataSourceWriteOptions.RECORDKEY_FIELD().key(), "_row_key")
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), "partition")
.option(DataSourceWriteOptions.PRECOMBINE_FIELD().key(), "_row_key")
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS().key(),
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key(),
nonPartitionedTable ? NonpartitionedKeyGenerator.class.getCanonicalName()
: SimpleKeyGenerator.class.getCanonicalName()) // Add Key Extractor
.option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS.key(), "1")
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE().key(), "false")
.option(DataSourceWriteOptions.ASYNC_CLUSTERING_ENABLE().key(), "true")
.option(HoodieWriteConfig.TABLE_NAME.key(), tableName).mode(SaveMode.Append);
.option(HoodieWriteConfig.TBL_NAME.key(), tableName).mode(SaveMode.Append);
updateHiveSyncConfig(writer);
writer.save(tablePath);

View File

@@ -175,9 +175,9 @@ public class HoodieJavaGenerateApp {
// use to combine duplicate records in input/with disk val
.option(DataSourceWriteOptions.PRECOMBINE_FIELD().key(), "timestamp")
// Used by hive sync and queries
.option(HoodieWriteConfig.TABLE_NAME.key(), tableName)
.option(HoodieWriteConfig.TBL_NAME.key(), tableName)
// Add Key Extractor
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS().key(),
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key(),
nonPartitionedTable ? NonpartitionedKeyGenerator.class.getCanonicalName()
: SimpleKeyGenerator.class.getCanonicalName())
.mode(commitType);

View File

@@ -16,7 +16,6 @@
* limitations under the License.
*/
import java.util.stream.Collectors;
import org.apache.hudi.DataSourceReadOptions;
import org.apache.hudi.DataSourceWriteOptions;
import org.apache.hudi.HoodieDataSourceHelpers;
@@ -43,13 +42,14 @@ import org.apache.spark.sql.SaveMode;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.sql.streaming.DataStreamWriter;
import org.apache.spark.sql.streaming.OutputMode;
import org.apache.spark.sql.streaming.StreamingQuery;
import org.apache.spark.sql.streaming.Trigger;
import java.util.List;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import org.apache.spark.sql.streaming.StreamingQuery;
import java.util.stream.Collectors;
import static org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings;
@@ -363,7 +363,7 @@ public class HoodieJavaStreamingApp {
.option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS.key(), "1")
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE().key(), "true")
.option(DataSourceWriteOptions.ASYNC_CLUSTERING_ENABLE().key(), "true")
.option(HoodieWriteConfig.TABLE_NAME.key(), tableName).option("checkpointLocation", checkpointLocation)
.option(HoodieWriteConfig.TBL_NAME.key(), tableName).option("checkpointLocation", checkpointLocation)
.outputMode(OutputMode.Append());
updateHiveSyncConfig(writer);

View File

@@ -18,12 +18,6 @@
package org.apache.hudi.functional;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hudi.DataSourceWriteOptions;
import org.apache.hudi.avro.model.HoodieFileStatus;
import org.apache.hudi.client.SparkRDDWriteClient;
@@ -49,6 +43,7 @@ import org.apache.hudi.common.testutils.HoodieTestUtils;
import org.apache.hudi.common.testutils.RawTripTestPayload;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ParquetReaderIterator;
import org.apache.hudi.common.util.PartitionPathEncodeUtils;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieBootstrapConfig;
import org.apache.hudi.config.HoodieCompactionConfig;
@@ -58,11 +53,17 @@ import org.apache.hudi.hadoop.HoodieParquetInputFormat;
import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat;
import org.apache.hudi.index.HoodieIndex.IndexType;
import org.apache.hudi.keygen.NonpartitionedKeyGenerator;
import org.apache.hudi.common.util.PartitionPathEncodeUtils;
import org.apache.hudi.keygen.SimpleKeyGenerator;
import org.apache.hudi.table.action.bootstrap.BootstrapUtils;
import org.apache.hudi.testutils.HoodieClientTestBase;
import org.apache.hudi.testutils.HoodieMergeOnReadTestUtils;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.mapred.JobConf;
import org.apache.parquet.avro.AvroParquetReader;
import org.apache.parquet.avro.AvroReadSupport;
import org.apache.parquet.avro.AvroSchemaConverter;
@@ -374,7 +375,7 @@ public class TestBootstrap extends HoodieClientTestBase {
List<GenericRecord> records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
jsc.hadoopConfiguration(),
FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS,
HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream()
HoodieMetadataConfig.VALIDATE_ENABLE.defaultValue(), false).stream()
.map(f -> basePath + "/" + f).collect(Collectors.toList()),
basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES, false, new ArrayList<>());
assertEquals(totalRecords, records.size());
@@ -393,7 +394,7 @@ public class TestBootstrap extends HoodieClientTestBase {
records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
jsc.hadoopConfiguration(),
FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS,
HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream()
HoodieMetadataConfig.VALIDATE_ENABLE.defaultValue(), false).stream()
.map(f -> basePath + "/" + f).collect(Collectors.toList()),
basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, false, new ArrayList<>());
assertEquals(totalRecords, records.size());
@@ -410,7 +411,7 @@ public class TestBootstrap extends HoodieClientTestBase {
records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
jsc.hadoopConfiguration(),
FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS,
HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream()
HoodieMetadataConfig.VALIDATE_ENABLE.defaultValue(), false).stream()
.map(f -> basePath + "/" + f).collect(Collectors.toList()),
basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES,
true, HoodieRecord.HOODIE_META_COLUMNS);
@@ -428,7 +429,7 @@ public class TestBootstrap extends HoodieClientTestBase {
records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
jsc.hadoopConfiguration(),
FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS,
HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream()
HoodieMetadataConfig.VALIDATE_ENABLE.defaultValue(), false).stream()
.map(f -> basePath + "/" + f).collect(Collectors.toList()),
basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, true,
HoodieRecord.HOODIE_META_COLUMNS);
@@ -444,7 +445,7 @@ public class TestBootstrap extends HoodieClientTestBase {
records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
jsc.hadoopConfiguration(),
FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS,
HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream()
HoodieMetadataConfig.VALIDATE_ENABLE.defaultValue(), false).stream()
.map(f -> basePath + "/" + f).collect(Collectors.toList()),
basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES, true,
Arrays.asList("_row_key"));
@@ -462,7 +463,7 @@ public class TestBootstrap extends HoodieClientTestBase {
records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
jsc.hadoopConfiguration(),
FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS,
HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream()
HoodieMetadataConfig.VALIDATE_ENABLE.defaultValue(), false).stream()
.map(f -> basePath + "/" + f).collect(Collectors.toList()),
basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, true,
Arrays.asList("_row_key"));

View File

@@ -214,12 +214,12 @@ public class TestHoodieDatasetBulkInsertHelper extends HoodieClientTestBase {
private Map<String, String> getProps(boolean setAll, boolean setKeyGen, boolean setRecordKey, boolean setPartitionPath) {
Map<String, String> props = new HashMap<>();
if (setAll) {
props.put(DataSourceWriteOptions.KEYGENERATOR_CLASS().key(), "org.apache.hudi.keygen.SimpleKeyGenerator");
props.put(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key(), "org.apache.hudi.keygen.SimpleKeyGenerator");
props.put(DataSourceWriteOptions.RECORDKEY_FIELD().key(), "_row_key");
props.put(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), "partition");
} else {
if (setKeyGen) {
props.put(DataSourceWriteOptions.KEYGENERATOR_CLASS().key(), "org.apache.hudi.keygen.SimpleKeyGenerator");
props.put(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key(), "org.apache.hudi.keygen.SimpleKeyGenerator");
}
if (setRecordKey) {
props.put(DataSourceWriteOptions.RECORDKEY_FIELD().key(), "_row_key");