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

@@ -18,11 +18,11 @@
package org.apache.hudi.examples.spark
import org.apache.hudi.DataSourceReadOptions.{BEGIN_INSTANTTIME, END_INSTANTTIME, QUERY_TYPE_INCREMENTAL_OPT_VAL, QUERY_TYPE}
import org.apache.hudi.DataSourceReadOptions.{BEGIN_INSTANTTIME, END_INSTANTTIME, QUERY_TYPE, QUERY_TYPE_INCREMENTAL_OPT_VAL}
import org.apache.hudi.DataSourceWriteOptions.{PARTITIONPATH_FIELD, PRECOMBINE_FIELD, RECORDKEY_FIELD, PARTITIONS_TO_DELETE, OPERATION, DELETE_PARTITION_OPERATION_OPT_VAL, DELETE_OPERATION_OPT_VAL}
import org.apache.hudi.QuickstartUtils.getQuickstartWriteConfigs
import org.apache.hudi.common.model.HoodieAvroPayload
import org.apache.hudi.config.HoodieWriteConfig.TABLE_NAME
import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME
import org.apache.hudi.examples.common.{HoodieExampleDataGenerator, HoodieExampleSparkUtils}
import org.apache.spark.sql.SaveMode.{Append, Overwrite}
import org.apache.spark.sql.SparkSession
@@ -76,12 +76,12 @@ object HoodieDataSourceExample {
val inserts = dataGen.convertToStringList(dataGen.generateInserts(commitTime, 20))
val df = spark.read.json(spark.sparkContext.parallelize(inserts, 1))
df.write.format("org.apache.hudi").
options(getQuickstartWriteConfigs).
option(PRECOMBINE_FIELD.key, "ts").
option(RECORDKEY_FIELD.key, "uuid").
option(PARTITIONPATH_FIELD.key, "partitionpath").
option(TABLE_NAME.key, tableName).
mode(Overwrite).
options(getQuickstartWriteConfigs).
option(PRECOMBINE_FIELD.key, "ts").
option(RECORDKEY_FIELD.key, "uuid").
option(PARTITIONPATH_FIELD.key, "partitionpath").
option(TBL_NAME.key, tableName).
mode(Overwrite).
save(tablePath)
}
@@ -121,12 +121,12 @@ object HoodieDataSourceExample {
val updates = dataGen.convertToStringList(dataGen.generateUpdates(commitTime, 10))
val df = spark.read.json(spark.sparkContext.parallelize(updates, 1))
df.write.format("org.apache.hudi").
options(getQuickstartWriteConfigs).
option(PRECOMBINE_FIELD.key, "ts").
option(RECORDKEY_FIELD.key, "uuid").
option(PARTITIONPATH_FIELD.key, "partitionpath").
option(TABLE_NAME.key, tableName).
mode(Append).
options(getQuickstartWriteConfigs).
option(PRECOMBINE_FIELD.key, "ts").
option(RECORDKEY_FIELD.key, "uuid").
option(PARTITIONPATH_FIELD.key, "partitionpath").
option(TBL_NAME.key, tableName).
mode(Append).
save(tablePath)
}
@@ -144,7 +144,7 @@ object HoodieDataSourceExample {
option(PRECOMBINE_FIELD.key, "ts").
option(RECORDKEY_FIELD.key, "uuid").
option(PARTITIONPATH_FIELD.key, "partitionpath").
option(TABLE_NAME.key, tableName).
option(TBL_NAME.key, tableName).
option(OPERATION.key, DELETE_OPERATION_OPT_VAL).
mode(Append).
save(tablePath)
@@ -160,7 +160,7 @@ object HoodieDataSourceExample {
option(PRECOMBINE_FIELD.key, "ts").
option(RECORDKEY_FIELD.key, "uuid").
option(PARTITIONPATH_FIELD.key, "partitionpath").
option(TABLE_NAME.key, tableName).
option(TBL_NAME.key, tableName).
option(OPERATION.key, DELETE_PARTITION_OPERATION_OPT_VAL).
option(PARTITIONS_TO_DELETE.key(), HoodieExampleDataGenerator.DEFAULT_PARTITION_PATHS.mkString(",")).
mode(Append).

View File

@@ -25,7 +25,7 @@ import org.apache.hudi.client.SparkRDDWriteClient
import org.apache.hudi.client.common.HoodieSparkEngineContext
import org.apache.hudi.common.model.{HoodieAvroPayload, HoodieRecordPayload, HoodieTableType}
import org.apache.hudi.common.util.Option
import org.apache.hudi.config.HoodieWriteConfig.TABLE_NAME
import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME
import org.apache.hudi.config.{HoodieCompactionConfig, HoodieWriteConfig}
import org.apache.hudi.examples.common.{HoodieExampleDataGenerator, HoodieExampleSparkUtils}
import org.apache.spark.sql.SaveMode.{Append, Overwrite}
@@ -89,7 +89,7 @@ object HoodieMorCompactionJob {
option(PRECOMBINE_FIELD.key, "ts").
option(RECORDKEY_FIELD.key, "uuid").
option(PARTITIONPATH_FIELD.key, "partitionpath").
option(TABLE_NAME.key, tableName).
option(TBL_NAME.key, tableName).
option(TABLE_TYPE.key, tableType).
mode(Overwrite).
save(tablePath)
@@ -105,7 +105,7 @@ object HoodieMorCompactionJob {
option(PRECOMBINE_FIELD.key, "ts").
option(RECORDKEY_FIELD.key, "uuid").
option(PARTITIONPATH_FIELD.key, "partitionpath").
option(TABLE_NAME.key, tableName).
option(TBL_NAME.key, tableName).
option(TABLE_TYPE.key, tableType).
mode(Append).
save(tablePath)