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:
@@ -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);
|
||||
|
||||
@@ -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);
|
||||
|
||||
@@ -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);
|
||||
|
||||
@@ -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"));
|
||||
|
||||
@@ -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");
|
||||
|
||||
@@ -22,9 +22,8 @@ import org.apache.hadoop.fs.Path
|
||||
import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.client.SparkRDDWriteClient
|
||||
import org.apache.hudi.common.config.HoodieConfig
|
||||
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.common.model.{HoodieFileFormat, HoodieRecord, HoodieRecordPayload, HoodieTableType, WriteOperationType}
|
||||
import org.apache.hudi.common.table.HoodieTableConfig
|
||||
import org.apache.hudi.common.model._
|
||||
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator
|
||||
import org.apache.hudi.config.{HoodieBootstrapConfig, HoodieWriteConfig}
|
||||
import org.apache.hudi.exception.HoodieException
|
||||
@@ -73,7 +72,7 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
val session = SparkSession.builder().appName("hoodie_test").master("local").getOrCreate()
|
||||
try {
|
||||
val sqlContext = session.sqlContext
|
||||
val options = Map("path" -> "hoodie/test/path", HoodieWriteConfig.TABLE_NAME.key -> "hoodie_test_tbl")
|
||||
val options = Map("path" -> "hoodie/test/path", HoodieWriteConfig.TBL_NAME.key -> "hoodie_test_tbl")
|
||||
val e = intercept[HoodieException](HoodieSparkSqlWriter.write(sqlContext, SaveMode.ErrorIfExists, options,
|
||||
session.emptyDataFrame))
|
||||
assert(e.getMessage.contains("spark.serializer"))
|
||||
@@ -89,7 +88,7 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
val hoodieFooTableName = "hoodie_foo_tbl"
|
||||
//create a new table
|
||||
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
|
||||
HoodieWriteConfig.TABLE_NAME.key -> hoodieFooTableName,
|
||||
HoodieWriteConfig.TBL_NAME.key -> hoodieFooTableName,
|
||||
"hoodie.insert.shuffle.parallelism" -> "4",
|
||||
"hoodie.upsert.shuffle.parallelism" -> "4")
|
||||
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
||||
@@ -98,7 +97,7 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
|
||||
//on same path try append with different("hoodie_bar_tbl") table name which should throw an exception
|
||||
val barTableModifier = Map("path" -> path.toAbsolutePath.toString,
|
||||
HoodieWriteConfig.TABLE_NAME.key -> "hoodie_bar_tbl",
|
||||
HoodieWriteConfig.TBL_NAME.key -> "hoodie_bar_tbl",
|
||||
"hoodie.insert.shuffle.parallelism" -> "4",
|
||||
"hoodie.upsert.shuffle.parallelism" -> "4")
|
||||
val barTableParams = HoodieWriterUtils.parametersWithWriteDefaults(barTableModifier)
|
||||
@@ -151,8 +150,8 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
.updated("hoodie.bulkinsert.shuffle.parallelism", "4")
|
||||
.updated(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL)
|
||||
.updated(DataSourceWriteOptions.ENABLE_ROW_WRITER.key, "true")
|
||||
.updated(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key(), String.valueOf(populateMetaFields))
|
||||
.updated(HoodieWriteConfig.BULKINSERT_SORT_MODE.key(), sortMode.name())
|
||||
.updated(HoodieTableConfig.POPULATE_META_FIELDS.key(), String.valueOf(populateMetaFields))
|
||||
.updated(HoodieWriteConfig.BULK_INSERT_SORT_MODE.key(), sortMode.name())
|
||||
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
||||
|
||||
// generate the inserts
|
||||
@@ -200,7 +199,7 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
.updated("hoodie.bulkinsert.shuffle.parallelism", "4")
|
||||
.updated(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL)
|
||||
.updated(DataSourceWriteOptions.ENABLE_ROW_WRITER.key, "true")
|
||||
.updated(HoodieWriteConfig.BULKINSERT_SORT_MODE.key(), BulkInsertSortMode.NONE.name())
|
||||
.updated(HoodieWriteConfig.BULK_INSERT_SORT_MODE.key(), BulkInsertSortMode.NONE.name())
|
||||
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
||||
|
||||
// generate the inserts
|
||||
@@ -352,15 +351,15 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
try {
|
||||
val hoodieFooTableName = "hoodie_foo_tbl"
|
||||
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
|
||||
HoodieWriteConfig.TABLE_NAME.key -> hoodieFooTableName,
|
||||
HoodieWriteConfig.TBL_NAME.key -> hoodieFooTableName,
|
||||
HoodieWriteConfig.BASE_FILE_FORMAT.key -> baseFileFormat,
|
||||
DataSourceWriteOptions.TABLE_TYPE.key -> tableType,
|
||||
HoodieWriteConfig.INSERT_PARALLELISM.key -> "4",
|
||||
HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key -> "4",
|
||||
DataSourceWriteOptions.OPERATION.key -> DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL,
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
|
||||
HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key() -> String.valueOf(populateMetaFields),
|
||||
DataSourceWriteOptions.KEYGENERATOR_CLASS.key -> classOf[SimpleKeyGenerator].getCanonicalName)
|
||||
HoodieTableConfig.POPULATE_META_FIELDS.key() -> String.valueOf(populateMetaFields),
|
||||
DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key -> classOf[SimpleKeyGenerator].getCanonicalName)
|
||||
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
||||
|
||||
// generate the inserts
|
||||
@@ -428,14 +427,14 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
.save(srcPath.toAbsolutePath.toString)
|
||||
|
||||
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
|
||||
HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH.key -> srcPath.toAbsolutePath.toString,
|
||||
HoodieWriteConfig.TABLE_NAME.key -> hoodieFooTableName,
|
||||
HoodieBootstrapConfig.BASE_PATH.key -> srcPath.toAbsolutePath.toString,
|
||||
HoodieWriteConfig.TBL_NAME.key -> hoodieFooTableName,
|
||||
DataSourceWriteOptions.TABLE_TYPE.key -> tableType,
|
||||
HoodieBootstrapConfig.BOOTSTRAP_PARALLELISM.key -> "4",
|
||||
HoodieBootstrapConfig.PARALLELISM_VALUE.key -> "4",
|
||||
DataSourceWriteOptions.OPERATION.key -> DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL,
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
|
||||
HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS.key -> classOf[NonpartitionedKeyGenerator].getCanonicalName)
|
||||
HoodieBootstrapConfig.KEYGEN_CLASS_NAME.key -> classOf[NonpartitionedKeyGenerator].getCanonicalName)
|
||||
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
||||
|
||||
val client = spy(DataSourceUtils.createHoodieClient(
|
||||
@@ -552,7 +551,7 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
"path" -> basePath,
|
||||
DataSourceWriteOptions.TABLE_NAME.key -> "test_hoodie",
|
||||
DataSourceWriteOptions.HIVE_PARTITION_FIELDS.key -> "partition",
|
||||
DataSourceWriteOptions.HIVE_SKIP_RO_SUFFIX.key -> "true",
|
||||
DataSourceWriteOptions.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.key -> "true",
|
||||
DataSourceWriteOptions.HIVE_CREATE_MANAGED_TABLE.key -> "true"
|
||||
)
|
||||
val parameters = HoodieWriterUtils.parametersWithWriteDefaults(params)
|
||||
@@ -621,8 +620,8 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "col3",
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD.key -> "keyid",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "",
|
||||
DataSourceWriteOptions.KEYGENERATOR_CLASS.key -> "org.apache.hudi.keygen.NonpartitionedKeyGenerator",
|
||||
HoodieWriteConfig.TABLE_NAME.key -> "hoodie_test")
|
||||
DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key -> "org.apache.hudi.keygen.NonpartitionedKeyGenerator",
|
||||
HoodieWriteConfig.TBL_NAME.key -> "hoodie_test")
|
||||
try {
|
||||
val df = spark.range(0, 1000).toDF("keyid")
|
||||
.withColumn("col3", expr("keyid"))
|
||||
@@ -652,10 +651,10 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
df.write.mode(SaveMode.Overwrite).save(baseBootStrapPath)
|
||||
spark.emptyDataFrame.write.format("hudi")
|
||||
.options(options)
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH.key, baseBootStrapPath)
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS.key, classOf[NonpartitionedKeyGenerator].getCanonicalName)
|
||||
.option(HoodieBootstrapConfig.BASE_PATH.key, baseBootStrapPath)
|
||||
.option(HoodieBootstrapConfig.KEYGEN_CLASS_NAME.key, classOf[NonpartitionedKeyGenerator].getCanonicalName)
|
||||
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL)
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_PARALLELISM.key, "4")
|
||||
.option(HoodieBootstrapConfig.PARALLELISM_VALUE.key, "4")
|
||||
.mode(SaveMode.Overwrite).save(basePath)
|
||||
|
||||
df.write.format("hudi")
|
||||
@@ -750,13 +749,13 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
|
||||
def getCommonParams(path: java.nio.file.Path, hoodieFooTableName: String, tableType: String) : Map[String, String] = {
|
||||
Map("path" -> path.toAbsolutePath.toString,
|
||||
HoodieWriteConfig.TABLE_NAME.key -> hoodieFooTableName,
|
||||
HoodieWriteConfig.TBL_NAME.key -> hoodieFooTableName,
|
||||
"hoodie.insert.shuffle.parallelism" -> "1",
|
||||
"hoodie.upsert.shuffle.parallelism" -> "1",
|
||||
DataSourceWriteOptions.TABLE_TYPE.key -> tableType,
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
|
||||
DataSourceWriteOptions.KEYGENERATOR_CLASS.key -> "org.apache.hudi.keygen.SimpleKeyGenerator")
|
||||
DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key -> "org.apache.hudi.keygen.SimpleKeyGenerator")
|
||||
}
|
||||
|
||||
test("test Non partition table with metatable support") {
|
||||
@@ -775,11 +774,11 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
.option(DataSourceWriteOptions.PRECOMBINE_FIELD.key, "col3")
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD.key, "keyid")
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "")
|
||||
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS.key, "org.apache.hudi.keygen.NonpartitionedKeyGenerator")
|
||||
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key, "org.apache.hudi.keygen.NonpartitionedKeyGenerator")
|
||||
.option(DataSourceWriteOptions.OPERATION.key, "insert")
|
||||
.option("hoodie.insert.shuffle.parallelism", "1")
|
||||
.option("hoodie.metadata.enable", "true")
|
||||
.option(HoodieWriteConfig.TABLE_NAME.key, "hoodie_test")
|
||||
.option(HoodieWriteConfig.TBL_NAME.key, "hoodie_test")
|
||||
.mode(SaveMode.Overwrite).save(basePath)
|
||||
// upsert same record again
|
||||
val df_update = spark.range(0, 10).toDF("keyid")
|
||||
@@ -790,11 +789,11 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
.option(DataSourceWriteOptions.PRECOMBINE_FIELD.key, "col3")
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD.key, "keyid")
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "")
|
||||
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS.key, "org.apache.hudi.keygen.NonpartitionedKeyGenerator")
|
||||
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key, "org.apache.hudi.keygen.NonpartitionedKeyGenerator")
|
||||
.option(DataSourceWriteOptions.OPERATION.key, "upsert")
|
||||
.option("hoodie.upsert.shuffle.parallelism", "1")
|
||||
.option("hoodie.metadata.enable", "true")
|
||||
.option(HoodieWriteConfig.TABLE_NAME.key, "hoodie_test")
|
||||
.option(HoodieWriteConfig.TBL_NAME.key, "hoodie_test")
|
||||
.mode(SaveMode.Append).save(basePath)
|
||||
assert(spark.read.format("hudi").load(basePath).count() == 10)
|
||||
assert(spark.read.format("hudi").load(basePath).where("age >= 2000").count() == 10)
|
||||
|
||||
@@ -29,10 +29,10 @@ import org.apache.hudi.keygen.ComplexKeyGenerator
|
||||
import org.apache.hudi.keygen.TimestampBasedAvroKeyGenerator.{Config, TimestampType}
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase
|
||||
import org.apache.spark.sql.catalyst.InternalRow
|
||||
import org.apache.spark.sql.{SaveMode, SparkSession}
|
||||
import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, EqualTo, GreaterThanOrEqual, LessThan, Literal}
|
||||
import org.apache.spark.sql.execution.datasources.PartitionDirectory
|
||||
import org.apache.spark.sql.types.StringType
|
||||
import org.apache.spark.sql.{SaveMode, SparkSession}
|
||||
import org.junit.jupiter.api.Assertions.assertEquals
|
||||
import org.junit.jupiter.api.BeforeEach
|
||||
import org.junit.jupiter.params.ParameterizedTest
|
||||
@@ -50,7 +50,7 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
|
||||
DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
|
||||
HoodieWriteConfig.TABLE_NAME.key -> "hoodie_test"
|
||||
HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
|
||||
)
|
||||
|
||||
var queryOpts = Map(
|
||||
@@ -95,7 +95,7 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
|
||||
inputDF1.write.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS.key, keyGenerator)
|
||||
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key, keyGenerator)
|
||||
.option(Config.TIMESTAMP_TYPE_FIELD_PROP, TimestampType.DATE_STRING.name())
|
||||
.option(Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP, "yyyy/MM/dd")
|
||||
.option(Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP, "yyyy-MM-dd")
|
||||
@@ -116,7 +116,7 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
|
||||
inputDF1.write.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS.key, keyGenerator)
|
||||
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key, keyGenerator)
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "partition:simple")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
@@ -176,14 +176,14 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
|
||||
.option(RECORDKEY_FIELD.key, "id")
|
||||
.option(PRECOMBINE_FIELD.key, "version")
|
||||
.option(PARTITIONPATH_FIELD.key, "dt,hh")
|
||||
.option(KEYGENERATOR_CLASS.key, classOf[ComplexKeyGenerator].getName)
|
||||
.option(KEYGENERATOR_CLASS_NAME.key, classOf[ComplexKeyGenerator].getName)
|
||||
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING.key, "false")
|
||||
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, useMetaFileList)
|
||||
.option(HoodieMetadataConfig.ENABLE.key, useMetaFileList)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient)
|
||||
val fileIndex = HoodieFileIndex(spark, metaClient, None,
|
||||
queryOpts ++ Map(HoodieMetadataConfig.METADATA_ENABLE_PROP.key -> useMetaFileList.toString))
|
||||
queryOpts ++ Map(HoodieMetadataConfig.ENABLE.key -> useMetaFileList.toString))
|
||||
|
||||
val partitionFilter1 = And(
|
||||
EqualTo(attribute("dt"), literal("2021-03-01")),
|
||||
@@ -198,7 +198,7 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
|
||||
assertEquals(getFileCountInPartitionPath("2021-03-01/10"), filesAfterPrune.size)
|
||||
|
||||
val readDF1 = spark.read.format("hudi")
|
||||
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key(), useMetaFileList)
|
||||
.option(HoodieMetadataConfig.ENABLE.key(), useMetaFileList)
|
||||
.load(basePath)
|
||||
assertEquals(10, readDF1.count())
|
||||
assertEquals(5, readDF1.filter("dt = '2021-03-01' and hh = '10'").count())
|
||||
@@ -213,9 +213,9 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
|
||||
.option(RECORDKEY_FIELD.key, "id")
|
||||
.option(PRECOMBINE_FIELD.key, "version")
|
||||
.option(PARTITIONPATH_FIELD.key, "dt,hh")
|
||||
.option(KEYGENERATOR_CLASS.key, classOf[ComplexKeyGenerator].getName)
|
||||
.option(KEYGENERATOR_CLASS_NAME.key, classOf[ComplexKeyGenerator].getName)
|
||||
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING.key, "false")
|
||||
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key(), useMetaFileList)
|
||||
.option(HoodieMetadataConfig.ENABLE.key(), useMetaFileList)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
@@ -235,7 +235,7 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
|
||||
assertEquals(getFileCountInPartitionPaths("2021/03/01/10", "2021/03/02/10"),
|
||||
filesAfterPrune2.length)
|
||||
val readDF2 = spark.read.format("hudi")
|
||||
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, useMetaFileList)
|
||||
.option(HoodieMetadataConfig.ENABLE.key, useMetaFileList)
|
||||
.load(basePath)
|
||||
|
||||
assertEquals(10, readDF2.count())
|
||||
|
||||
@@ -56,7 +56,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
|
||||
DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
|
||||
HoodieWriteConfig.TABLE_NAME.key -> "hoodie_test"
|
||||
HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
|
||||
)
|
||||
|
||||
val verificationCol: String = "driver"
|
||||
@@ -117,7 +117,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
|
||||
val tableMetaClient = HoodieTableMetaClient.builder().setConf(spark.sparkContext.hadoopConfiguration).setBasePath(basePath).build()
|
||||
val actualSchema = new TableSchemaResolver(tableMetaClient).getTableAvroSchemaWithoutMetadataFields
|
||||
val (structName, nameSpace) = AvroConversionUtils.getAvroRecordNameAndNamespace(commonOpts(HoodieWriteConfig.TABLE_NAME.key))
|
||||
val (structName, nameSpace) = AvroConversionUtils.getAvroRecordNameAndNamespace(commonOpts(HoodieWriteConfig.TBL_NAME.key))
|
||||
spark.sparkContext.getConf.registerKryoClasses(
|
||||
Array(classOf[org.apache.avro.generic.GenericData],
|
||||
classOf[org.apache.avro.Schema]))
|
||||
@@ -166,7 +166,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
|
||||
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
@@ -175,7 +175,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
|
||||
// Snapshot query
|
||||
val snapshotDF1 = spark.read.format("org.apache.hudi")
|
||||
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
|
||||
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
|
||||
.load(basePath + "/*/*/*")
|
||||
assertEquals(100, snapshotDF1.count())
|
||||
|
||||
@@ -185,13 +185,13 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
|
||||
updateDf.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
|
||||
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val commitInstantTime2 = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
|
||||
val snapshotDF2 = spark.read.format("hudi")
|
||||
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
|
||||
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
|
||||
.load(basePath + "/*/*/*")
|
||||
assertEquals(100, snapshotDF2.count())
|
||||
assertEquals(updatedVerificationVal, snapshotDF2.filter(col("_row_key") === verificationRowKey).select(verificationCol).first.getString(0))
|
||||
@@ -203,7 +203,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
|
||||
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
@@ -212,7 +212,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
|
||||
// Snapshot Query
|
||||
val snapshotDF3 = spark.read.format("org.apache.hudi")
|
||||
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
|
||||
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
|
||||
.load(basePath + "/*/*/*")
|
||||
assertEquals(100, snapshotDF3.count()) // still 100, since we only updated
|
||||
|
||||
@@ -242,7 +242,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
val emptyDF = spark.read.json(spark.sparkContext.parallelize(emptyRecords, 1))
|
||||
emptyDF.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
|
||||
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
@@ -510,7 +510,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(HoodieWriteConfig.HOODIE_AUTO_COMMIT.key, "true")
|
||||
.option(HoodieWriteConfig.AUTO_COMMIT_ENABLE.key, "true")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
@@ -523,7 +523,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
|
||||
inputDF.write.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS.key, keyGenerator)
|
||||
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key, keyGenerator)
|
||||
.mode(SaveMode.Overwrite)
|
||||
}
|
||||
|
||||
@@ -660,7 +660,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING.key, partitionEncode)
|
||||
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
|
||||
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
val commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
@@ -668,7 +668,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
val countIn20160315 = records1.asScala.count(record => record.getPartitionPath == "2016/03/15")
|
||||
// query the partition by filter
|
||||
val count1 = spark.read.format("hudi")
|
||||
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
|
||||
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
|
||||
.load(basePath)
|
||||
.filter("partition = '2016/03/15'")
|
||||
.count()
|
||||
@@ -677,7 +677,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
// query the partition by path
|
||||
val partitionPath = if (partitionEncode) "2016%2F03%2F15" else "2016/03/15"
|
||||
val count2 = spark.read.format("hudi")
|
||||
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
|
||||
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
|
||||
.load(basePath + s"/$partitionPath")
|
||||
.count()
|
||||
assertEquals(countIn20160315, count2)
|
||||
@@ -689,7 +689,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING.key, partitionEncode)
|
||||
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
|
||||
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
// Incremental query without "*" in path
|
||||
|
||||
@@ -17,39 +17,39 @@
|
||||
|
||||
package org.apache.hudi.functional
|
||||
|
||||
import collection.JavaConverters._
|
||||
import org.apache.hadoop.fs.FileSystem
|
||||
import org.apache.hudi.bootstrap.SparkParquetBootstrapDataProvider
|
||||
import org.apache.hudi.client.bootstrap.selector.FullRecordBootstrapModeSelector
|
||||
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers}
|
||||
import org.apache.hudi.common.fs.FSUtils
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline
|
||||
import org.apache.hudi.config.{HoodieBootstrapConfig, HoodieCompactionConfig, HoodieWriteConfig}
|
||||
import org.apache.hudi.keygen.SimpleKeyGenerator
|
||||
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers}
|
||||
import org.apache.spark.api.java.JavaSparkContext
|
||||
import org.apache.spark.sql.functions.{col, lit}
|
||||
import org.apache.spark.sql.{SaveMode, SparkSession}
|
||||
import org.junit.jupiter.api.Assertions.assertEquals
|
||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
||||
import org.junit.jupiter.api.io.TempDir
|
||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
||||
|
||||
import java.time.Instant
|
||||
import java.util.Collections
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
class TestDataSourceForBootstrap {
|
||||
|
||||
var spark: SparkSession = _
|
||||
val commonOpts: Map[String, String] = Map(
|
||||
HoodieWriteConfig.INSERT_PARALLELISM.key -> "4",
|
||||
HoodieWriteConfig.UPSERT_PARALLELISM.key -> "4",
|
||||
HoodieWriteConfig.DELETE_PARALLELISM.key -> "4",
|
||||
HoodieWriteConfig.BULKINSERT_PARALLELISM.key -> "4",
|
||||
HoodieWriteConfig.FINALIZE_WRITE_PARALLELISM.key -> "4",
|
||||
HoodieBootstrapConfig.BOOTSTRAP_PARALLELISM.key -> "4",
|
||||
HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key -> "4",
|
||||
HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key -> "4",
|
||||
HoodieWriteConfig.DELETE_PARALLELISM_VALUE.key -> "4",
|
||||
HoodieWriteConfig.BULKINSERT_PARALLELISM_VALUE.key -> "4",
|
||||
HoodieWriteConfig.FINALIZE_WRITE_PARALLELISM_VALUE.key -> "4",
|
||||
HoodieBootstrapConfig.PARALLELISM_VALUE.key -> "4",
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
|
||||
DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
|
||||
HoodieWriteConfig.TABLE_NAME.key -> "hoodie_test"
|
||||
HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
|
||||
)
|
||||
var basePath: String = _
|
||||
var srcPath: String = _
|
||||
@@ -118,7 +118,7 @@ class TestDataSourceForBootstrap {
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS.key, "org.apache.hudi.keygen.NonpartitionedKeyGenerator")
|
||||
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key, "org.apache.hudi.keygen.NonpartitionedKeyGenerator")
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
@@ -429,10 +429,10 @@ class TestDataSourceForBootstrap {
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "datestr")
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH.key, srcPath)
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS.key, classOf[SimpleKeyGenerator].getName)
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_MODE_SELECTOR.key, classOf[FullRecordBootstrapModeSelector].getName)
|
||||
.option(HoodieBootstrapConfig.FULL_BOOTSTRAP_INPUT_PROVIDER.key, classOf[SparkParquetBootstrapDataProvider].getName)
|
||||
.option(HoodieBootstrapConfig.BASE_PATH.key, srcPath)
|
||||
.option(HoodieBootstrapConfig.KEYGEN_CLASS_NAME.key, classOf[SimpleKeyGenerator].getName)
|
||||
.option(HoodieBootstrapConfig.MODE_SELECTOR_CLASS_NAME.key, classOf[FullRecordBootstrapModeSelector].getName)
|
||||
.option(HoodieBootstrapConfig.FULL_BOOTSTRAP_INPUT_PROVIDER_CLASS_NAME.key, classOf[SparkParquetBootstrapDataProvider].getName)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
@@ -480,8 +480,8 @@ class TestDataSourceForBootstrap {
|
||||
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE.key, tableType)
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, partitionColumns.getOrElse(""))
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH.key, srcPath)
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS.key, classOf[SimpleKeyGenerator].getName)
|
||||
.option(HoodieBootstrapConfig.BASE_PATH.key, srcPath)
|
||||
.option(HoodieBootstrapConfig.KEYGEN_CLASS_NAME.key, classOf[SimpleKeyGenerator].getName)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
|
||||
@@ -17,12 +17,12 @@
|
||||
|
||||
package org.apache.hudi.functional
|
||||
|
||||
import org.apache.hudi.{DataSourceWriteOptions, HoodieDataSourceHelpers}
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase
|
||||
import org.apache.hudi.{DataSourceWriteOptions, HoodieDataSourceHelpers}
|
||||
import org.apache.spark.sql.{SaveMode, SparkSession}
|
||||
import org.junit.jupiter.api.{AfterEach, BeforeEach}
|
||||
import org.junit.jupiter.api.Assertions.assertEquals
|
||||
import org.junit.jupiter.api.{AfterEach, BeforeEach}
|
||||
import org.junit.jupiter.params.ParameterizedTest
|
||||
import org.junit.jupiter.params.provider.ValueSource
|
||||
|
||||
@@ -34,7 +34,7 @@ class TestEmptyCommit extends HoodieClientTestBase {
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
|
||||
DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
|
||||
HoodieWriteConfig.TABLE_NAME.key -> "hoodie_test"
|
||||
HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
|
||||
)
|
||||
|
||||
@BeforeEach override def setUp() {
|
||||
|
||||
@@ -18,30 +18,28 @@
|
||||
package org.apache.hudi.functional
|
||||
|
||||
import org.apache.hadoop.fs.Path
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
import org.apache.hudi.DataSourceWriteOptions.{KEYGENERATOR_CLASS, PARTITIONPATH_FIELD, PAYLOAD_CLASS, PRECOMBINE_FIELD, RECORDKEY_FIELD}
|
||||
import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig
|
||||
import org.apache.hudi.common.fs.FSUtils
|
||||
import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, HoodieTableType}
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator
|
||||
import org.apache.hudi.config.{HoodieIndexConfig, HoodieWriteConfig}
|
||||
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers, HoodieSparkUtils}
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
|
||||
import org.apache.hudi.config.{HoodieIndexConfig, HoodieWriteConfig}
|
||||
import org.apache.hudi.index.HoodieIndex.IndexType
|
||||
import org.apache.hudi.keygen.NonpartitionedKeyGenerator
|
||||
import org.apache.hudi.testutils.{DataSourceTestUtils, HoodieClientTestBase}
|
||||
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers}
|
||||
import org.apache.log4j.LogManager
|
||||
import org.apache.spark.sql._
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue}
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
|
||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
||||
import org.junit.jupiter.params.ParameterizedTest
|
||||
import org.junit.jupiter.params.provider.{CsvSource, ValueSource}
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
/**
|
||||
* Tests on Spark DataSource for MOR table.
|
||||
@@ -56,7 +54,7 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
|
||||
DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
|
||||
HoodieWriteConfig.TABLE_NAME.key -> "hoodie_test"
|
||||
HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
|
||||
)
|
||||
|
||||
val verificationCol: String = "driver"
|
||||
@@ -89,7 +87,7 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
.option("hoodie.compact.inline", "false") // else fails due to compaction & deltacommit instant times being same
|
||||
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
|
||||
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
@@ -98,7 +96,7 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
// Read RO View
|
||||
val hudiRODF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL)
|
||||
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
|
||||
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
|
||||
.load(basePath + "/*/*/*")
|
||||
|
||||
assertEquals(100, hudiRODF1.count()) // still 100, since we only updated
|
||||
@@ -111,7 +109,7 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
|
||||
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
@@ -119,7 +117,7 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
val updateCommitTime = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
val hudiSnapshotDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
|
||||
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
|
||||
.load(basePath + "/*/*/*")
|
||||
|
||||
val updateCommitTimes = hudiSnapshotDF2.select("_hoodie_commit_time").distinct().collectAsList().map(r => r.getString(0)).toList
|
||||
@@ -131,12 +129,12 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
|
||||
inputDF3.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
|
||||
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
val hudiSnapshotDF3 = spark.read.format("hudi")
|
||||
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
|
||||
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
|
||||
.load(basePath + "/*/*/*")
|
||||
assertEquals(100, hudiSnapshotDF3.count())
|
||||
assertEquals(updatedVerificationVal, hudiSnapshotDF3.filter(col("_row_key") === verificationRowKey).select(verificationCol).first.getString(0))
|
||||
@@ -420,7 +418,7 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
.option("hoodie.compact.inline", "false") // else fails due to compaction & deltacommit instant times being same
|
||||
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.PAYLOAD_CLASS.key, classOf[DefaultHoodieRecordPayload].getName)
|
||||
.option(DataSourceWriteOptions.PAYLOAD_CLASS_NAME.key, classOf[DefaultHoodieRecordPayload].getName)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF1 = spark.read.format("org.apache.hudi")
|
||||
@@ -582,12 +580,12 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
df.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
// use DefaultHoodieRecordPayload here
|
||||
.option(PAYLOAD_CLASS.key, classOf[DefaultHoodieRecordPayload].getCanonicalName)
|
||||
.option(PAYLOAD_CLASS_NAME.key, classOf[DefaultHoodieRecordPayload].getCanonicalName)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.option(RECORDKEY_FIELD.key, "id")
|
||||
.option(PRECOMBINE_FIELD.key, "version")
|
||||
.option(PARTITIONPATH_FIELD.key, "")
|
||||
.option(KEYGENERATOR_CLASS.key, classOf[NonpartitionedKeyGenerator].getName)
|
||||
.option(KEYGENERATOR_CLASS_NAME.key, classOf[NonpartitionedKeyGenerator].getName)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
}
|
||||
@@ -639,7 +637,7 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING.key, partitionEncode)
|
||||
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
|
||||
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
val commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
@@ -647,7 +645,7 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
val countIn20160315 = records1.asScala.count(record => record.getPartitionPath == "2016/03/15")
|
||||
// query the partition by filter
|
||||
val count1 = spark.read.format("hudi")
|
||||
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
|
||||
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
|
||||
.load(basePath)
|
||||
.filter("partition = '2016/03/15'")
|
||||
.count()
|
||||
@@ -656,7 +654,7 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
// query the partition by path
|
||||
val partitionPath = if (partitionEncode) "2016%2F03%2F15" else "2016/03/15"
|
||||
val count2 = spark.read.format("hudi")
|
||||
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
|
||||
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
|
||||
.load(basePath + s"/$partitionPath")
|
||||
.count()
|
||||
assertEquals(countIn20160315, count2)
|
||||
@@ -669,7 +667,7 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING.key, partitionEncode)
|
||||
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
|
||||
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
// Incremental query without "*" in path
|
||||
|
||||
@@ -21,7 +21,7 @@ import org.apache.hudi.DataSourceWriteOptions
|
||||
import org.apache.hudi.DataSourceWriteOptions.{PRECOMBINE_FIELD, RECORDKEY_FIELD}
|
||||
import org.apache.hudi.common.model.HoodieTableType.{COPY_ON_WRITE, MERGE_ON_READ}
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient
|
||||
import org.apache.hudi.config.HoodieWriteConfig.{DELETE_PARALLELISM, INSERT_PARALLELISM, TABLE_NAME, UPSERT_PARALLELISM}
|
||||
import org.apache.hudi.config.HoodieWriteConfig.{DELETE_PARALLELISM_VALUE, INSERT_PARALLELISM_VALUE, TBL_NAME, UPSERT_PARALLELISM_VALUE}
|
||||
import org.apache.log4j.Level
|
||||
import org.apache.spark.sql.streaming.StreamTest
|
||||
import org.apache.spark.sql.{Row, SaveMode}
|
||||
@@ -32,9 +32,9 @@ class TestStreamingSource extends StreamTest {
|
||||
private val commonOptions = Map(
|
||||
RECORDKEY_FIELD.key -> "id",
|
||||
PRECOMBINE_FIELD.key -> "ts",
|
||||
INSERT_PARALLELISM.key -> "4",
|
||||
UPSERT_PARALLELISM.key -> "4",
|
||||
DELETE_PARALLELISM.key -> "4"
|
||||
INSERT_PARALLELISM_VALUE.key -> "4",
|
||||
UPSERT_PARALLELISM_VALUE.key -> "4",
|
||||
DELETE_PARALLELISM_VALUE.key -> "4"
|
||||
)
|
||||
private val columns = Seq("id", "name", "price", "ts")
|
||||
|
||||
@@ -50,7 +50,7 @@ class TestStreamingSource extends StreamTest {
|
||||
HoodieTableMetaClient.withPropertyBuilder()
|
||||
.setTableType(COPY_ON_WRITE)
|
||||
.setTableName(getTableName(tablePath))
|
||||
.setPayloadClassName(DataSourceWriteOptions.PAYLOAD_CLASS.defaultValue)
|
||||
.setPayloadClassName(DataSourceWriteOptions.PAYLOAD_CLASS_NAME.defaultValue)
|
||||
.initTable(spark.sessionState.newHadoopConf(), tablePath)
|
||||
|
||||
addData(tablePath, Seq(("1", "a1", "10", "000")))
|
||||
@@ -100,7 +100,7 @@ class TestStreamingSource extends StreamTest {
|
||||
HoodieTableMetaClient.withPropertyBuilder()
|
||||
.setTableType(MERGE_ON_READ)
|
||||
.setTableName(getTableName(tablePath))
|
||||
.setPayloadClassName(DataSourceWriteOptions.PAYLOAD_CLASS.defaultValue)
|
||||
.setPayloadClassName(DataSourceWriteOptions.PAYLOAD_CLASS_NAME.defaultValue)
|
||||
.initTable(spark.sessionState.newHadoopConf(), tablePath)
|
||||
|
||||
addData(tablePath, Seq(("1", "a1", "10", "000")))
|
||||
@@ -143,7 +143,7 @@ class TestStreamingSource extends StreamTest {
|
||||
.write
|
||||
.format("org.apache.hudi")
|
||||
.options(commonOptions)
|
||||
.option(TABLE_NAME.key, getTableName(inputPath))
|
||||
.option(TBL_NAME.key, getTableName(inputPath))
|
||||
.mode(SaveMode.Append)
|
||||
.save(inputPath)
|
||||
}
|
||||
|
||||
@@ -50,7 +50,7 @@ class TestStructuredStreaming extends HoodieClientTestBase {
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
|
||||
DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
|
||||
HoodieWriteConfig.TABLE_NAME.key -> "hoodie_test"
|
||||
HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
|
||||
)
|
||||
|
||||
@BeforeEach override def setUp() {
|
||||
@@ -194,11 +194,11 @@ class TestStructuredStreaming extends HoodieClientTestBase {
|
||||
def getClusteringOpts(isInlineClustering: String, isAsyncClustering: String, isAsyncCompaction: String,
|
||||
clusteringNumCommit: String, fileMaxRecordNum: Int):Map[String, String] = {
|
||||
commonOpts + (HoodieClusteringConfig.INLINE_CLUSTERING.key -> isInlineClustering,
|
||||
HoodieClusteringConfig.INLINE_CLUSTERING_MAX_COMMIT.key -> clusteringNumCommit,
|
||||
HoodieClusteringConfig.INLINE_CLUSTERING_MAX_COMMITS.key -> clusteringNumCommit,
|
||||
DataSourceWriteOptions.ASYNC_CLUSTERING_ENABLE.key -> isAsyncClustering,
|
||||
DataSourceWriteOptions.ASYNC_COMPACT_ENABLE.key -> isAsyncCompaction,
|
||||
HoodieClusteringConfig.ASYNC_CLUSTERING_MAX_COMMIT_PROP.key -> clusteringNumCommit,
|
||||
HoodieStorageConfig.PARQUET_FILE_MAX_BYTES.key -> dataGen.getEstimatedFileSizeInBytes(fileMaxRecordNum).toString
|
||||
HoodieClusteringConfig.ASYNC_CLUSTERING_MAX_COMMITS.key -> clusteringNumCommit,
|
||||
HoodieStorageConfig.PARQUET_MAX_FILE_SIZE.key -> dataGen.getEstimatedFileSizeInBytes(fileMaxRecordNum).toString
|
||||
)
|
||||
}
|
||||
|
||||
|
||||
@@ -17,13 +17,13 @@
|
||||
|
||||
package org.apache.hudi.functional
|
||||
|
||||
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions}
|
||||
import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.common.model.HoodieTableType
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.keygen.{ComplexKeyGenerator, NonpartitionedKeyGenerator}
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase
|
||||
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions}
|
||||
import org.apache.spark.sql.{Row, SaveMode, SparkSession}
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
|
||||
import org.junit.jupiter.api.{AfterEach, BeforeEach}
|
||||
@@ -42,7 +42,7 @@ class TestTimeTravelQuery extends HoodieClientTestBase {
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
|
||||
DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
|
||||
HoodieWriteConfig.TABLE_NAME.key -> "hoodie_test"
|
||||
HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
|
||||
)
|
||||
|
||||
@BeforeEach override def setUp() {
|
||||
@@ -74,7 +74,7 @@ class TestTimeTravelQuery extends HoodieClientTestBase {
|
||||
.option(RECORDKEY_FIELD.key, "id")
|
||||
.option(PRECOMBINE_FIELD.key, "version")
|
||||
.option(PARTITIONPATH_FIELD.key, "")
|
||||
.option(KEYGENERATOR_CLASS.key, classOf[NonpartitionedKeyGenerator].getName)
|
||||
.option(KEYGENERATOR_CLASS_NAME.key, classOf[NonpartitionedKeyGenerator].getName)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
@@ -88,7 +88,7 @@ class TestTimeTravelQuery extends HoodieClientTestBase {
|
||||
.option(RECORDKEY_FIELD.key, "id")
|
||||
.option(PRECOMBINE_FIELD.key, "version")
|
||||
.option(PARTITIONPATH_FIELD.key, "")
|
||||
.option(KEYGENERATOR_CLASS.key, classOf[NonpartitionedKeyGenerator].getName)
|
||||
.option(KEYGENERATOR_CLASS_NAME.key, classOf[NonpartitionedKeyGenerator].getName)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
metaClient.reloadActiveTimeline()
|
||||
@@ -102,7 +102,7 @@ class TestTimeTravelQuery extends HoodieClientTestBase {
|
||||
.option(RECORDKEY_FIELD.key, "id")
|
||||
.option(PRECOMBINE_FIELD.key, "version")
|
||||
.option(PARTITIONPATH_FIELD.key, "")
|
||||
.option(KEYGENERATOR_CLASS.key, classOf[NonpartitionedKeyGenerator].getName)
|
||||
.option(KEYGENERATOR_CLASS_NAME.key, classOf[NonpartitionedKeyGenerator].getName)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
metaClient.reloadActiveTimeline()
|
||||
@@ -148,7 +148,7 @@ class TestTimeTravelQuery extends HoodieClientTestBase {
|
||||
.option(RECORDKEY_FIELD.key, "id")
|
||||
.option(PRECOMBINE_FIELD.key, "version")
|
||||
.option(PARTITIONPATH_FIELD.key, "dt")
|
||||
.option(KEYGENERATOR_CLASS.key, classOf[ComplexKeyGenerator].getName)
|
||||
.option(KEYGENERATOR_CLASS_NAME.key, classOf[ComplexKeyGenerator].getName)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
@@ -162,7 +162,7 @@ class TestTimeTravelQuery extends HoodieClientTestBase {
|
||||
.option(RECORDKEY_FIELD.key, "id")
|
||||
.option(PRECOMBINE_FIELD.key, "version")
|
||||
.option(PARTITIONPATH_FIELD.key, "dt")
|
||||
.option(KEYGENERATOR_CLASS.key, classOf[ComplexKeyGenerator].getName)
|
||||
.option(KEYGENERATOR_CLASS_NAME.key, classOf[ComplexKeyGenerator].getName)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
metaClient.reloadActiveTimeline()
|
||||
@@ -176,7 +176,7 @@ class TestTimeTravelQuery extends HoodieClientTestBase {
|
||||
.option(RECORDKEY_FIELD.key, "id")
|
||||
.option(PRECOMBINE_FIELD.key, "version")
|
||||
.option(PARTITIONPATH_FIELD.key, "dt")
|
||||
.option(KEYGENERATOR_CLASS.key, classOf[ComplexKeyGenerator].getName)
|
||||
.option(KEYGENERATOR_CLASS_NAME.key, classOf[ComplexKeyGenerator].getName)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
metaClient.reloadActiveTimeline()
|
||||
|
||||
@@ -18,8 +18,6 @@
|
||||
package org.apache.spark.sql.hudi
|
||||
|
||||
import org.apache.hudi.DataSourceWriteOptions._
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
import org.apache.hudi.common.model.HoodieRecord
|
||||
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
@@ -28,7 +26,9 @@ import org.apache.hudi.keygen.{ComplexKeyGenerator, NonpartitionedKeyGenerator,
|
||||
import org.apache.spark.sql.SaveMode
|
||||
import org.apache.spark.sql.catalyst.TableIdentifier
|
||||
import org.apache.spark.sql.catalyst.catalog.CatalogTableType
|
||||
import org.apache.spark.sql.types.{DoubleType, IntegerType, LongType, StringType, StructField}
|
||||
import org.apache.spark.sql.types._
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
class TestCreateTable extends TestHoodieSqlBase {
|
||||
|
||||
@@ -288,14 +288,14 @@ class TestCreateTable extends TestHoodieSqlBase {
|
||||
val df = Seq((1, "a1", 10, 1000, partitionValue)).toDF("id", "name", "value", "ts", "dt")
|
||||
// Write a table by spark dataframe.
|
||||
df.write.format("hudi")
|
||||
.option(HoodieWriteConfig.TABLE_NAME.key, tableName)
|
||||
.option(HoodieWriteConfig.TBL_NAME.key, tableName)
|
||||
.option(TABLE_TYPE.key, COW_TABLE_TYPE_OPT_VAL)
|
||||
.option(RECORDKEY_FIELD.key, "id")
|
||||
.option(PRECOMBINE_FIELD.key, "ts")
|
||||
.option(PARTITIONPATH_FIELD.key, "dt")
|
||||
.option(KEYGENERATOR_CLASS.key, classOf[SimpleKeyGenerator].getName)
|
||||
.option(HoodieWriteConfig.INSERT_PARALLELISM.key, "1")
|
||||
.option(HoodieWriteConfig.UPSERT_PARALLELISM.key, "1")
|
||||
.option(KEYGENERATOR_CLASS_NAME.key, classOf[SimpleKeyGenerator].getName)
|
||||
.option(HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key, "1")
|
||||
.option(HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key, "1")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(tablePath)
|
||||
|
||||
@@ -319,9 +319,9 @@ class TestCreateTable extends TestHoodieSqlBase {
|
||||
.setConf(spark.sessionState.newHadoopConf())
|
||||
.build()
|
||||
val properties = metaClient.getTableConfig.getProps.asScala.toMap
|
||||
assertResult(true)(properties.contains(HoodieTableConfig.HOODIE_TABLE_CREATE_SCHEMA.key))
|
||||
assertResult("dt")(properties(HoodieTableConfig.HOODIE_TABLE_PARTITION_FIELDS_PROP.key))
|
||||
assertResult("ts")(properties(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD_PROP.key))
|
||||
assertResult(true)(properties.contains(HoodieTableConfig.CREATE_SCHEMA.key))
|
||||
assertResult("dt")(properties(HoodieTableConfig.PARTITION_FIELDS.key))
|
||||
assertResult("ts")(properties(HoodieTableConfig.PRECOMBINE_FIELD.key))
|
||||
|
||||
// Test insert into
|
||||
spark.sql(s"insert into $tableName values(2, 'a2', 10, 1000, '$partitionValue')")
|
||||
@@ -365,14 +365,14 @@ class TestCreateTable extends TestHoodieSqlBase {
|
||||
val df = Seq((1, "a1", 10, 1000, day, 12)).toDF("id", "name", "value", "ts", "day", "hh")
|
||||
// Write a table by spark dataframe.
|
||||
df.write.format("hudi")
|
||||
.option(HoodieWriteConfig.TABLE_NAME.key, tableName)
|
||||
.option(HoodieWriteConfig.TBL_NAME.key, tableName)
|
||||
.option(TABLE_TYPE.key, MOR_TABLE_TYPE_OPT_VAL)
|
||||
.option(RECORDKEY_FIELD.key, "id")
|
||||
.option(PRECOMBINE_FIELD.key, "ts")
|
||||
.option(PARTITIONPATH_FIELD.key, "day,hh")
|
||||
.option(KEYGENERATOR_CLASS.key, classOf[ComplexKeyGenerator].getName)
|
||||
.option(HoodieWriteConfig.INSERT_PARALLELISM.key, "1")
|
||||
.option(HoodieWriteConfig.UPSERT_PARALLELISM.key, "1")
|
||||
.option(KEYGENERATOR_CLASS_NAME.key, classOf[ComplexKeyGenerator].getName)
|
||||
.option(HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key, "1")
|
||||
.option(HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key, "1")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(tablePath)
|
||||
|
||||
@@ -396,9 +396,9 @@ class TestCreateTable extends TestHoodieSqlBase {
|
||||
.setConf(spark.sessionState.newHadoopConf())
|
||||
.build()
|
||||
val properties = metaClient.getTableConfig.getProps.asScala.toMap
|
||||
assertResult(true)(properties.contains(HoodieTableConfig.HOODIE_TABLE_CREATE_SCHEMA.key))
|
||||
assertResult("day,hh")(properties(HoodieTableConfig.HOODIE_TABLE_PARTITION_FIELDS_PROP.key))
|
||||
assertResult("ts")(properties(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD_PROP.key))
|
||||
assertResult(true)(properties.contains(HoodieTableConfig.CREATE_SCHEMA.key))
|
||||
assertResult("day,hh")(properties(HoodieTableConfig.PARTITION_FIELDS.key))
|
||||
assertResult("ts")(properties(HoodieTableConfig.PRECOMBINE_FIELD.key))
|
||||
|
||||
// Test insert into
|
||||
spark.sql(s"insert into $tableName values(2, 'a2', 10, 1000, '$day', 12)")
|
||||
@@ -440,14 +440,14 @@ class TestCreateTable extends TestHoodieSqlBase {
|
||||
import spark.implicits._
|
||||
val df = Seq((1, "a1", 10, 1000)).toDF("id", "name", "value", "ts")
|
||||
df.write.format("hudi")
|
||||
.option(HoodieWriteConfig.TABLE_NAME.key, tableName)
|
||||
.option(HoodieWriteConfig.TBL_NAME.key, tableName)
|
||||
.option(TABLE_TYPE.key, COW_TABLE_TYPE_OPT_VAL)
|
||||
.option(RECORDKEY_FIELD.key, "id")
|
||||
.option(PRECOMBINE_FIELD.key, "ts")
|
||||
.option(PARTITIONPATH_FIELD.key, "")
|
||||
.option(KEYGENERATOR_CLASS.key, classOf[NonpartitionedKeyGenerator].getName)
|
||||
.option(HoodieWriteConfig.INSERT_PARALLELISM.key, "1")
|
||||
.option(HoodieWriteConfig.UPSERT_PARALLELISM.key, "1")
|
||||
.option(KEYGENERATOR_CLASS_NAME.key, classOf[NonpartitionedKeyGenerator].getName)
|
||||
.option(HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key, "1")
|
||||
.option(HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key, "1")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(tmp.getCanonicalPath)
|
||||
|
||||
@@ -470,8 +470,8 @@ class TestCreateTable extends TestHoodieSqlBase {
|
||||
.setConf(spark.sessionState.newHadoopConf())
|
||||
.build()
|
||||
val properties = metaClient.getTableConfig.getProps.asScala.toMap
|
||||
assertResult(true)(properties.contains(HoodieTableConfig.HOODIE_TABLE_CREATE_SCHEMA.key))
|
||||
assertResult("ts")(properties(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD_PROP.key))
|
||||
assertResult(true)(properties.contains(HoodieTableConfig.CREATE_SCHEMA.key))
|
||||
assertResult("ts")(properties(HoodieTableConfig.PRECOMBINE_FIELD.key))
|
||||
|
||||
// Test insert into
|
||||
spark.sql(s"insert into $tableName values(2, 'a2', 10, 1000)")
|
||||
|
||||
Reference in New Issue
Block a user