[HUDI-1040] Make Hudi support Spark 3 (#2208)
* Fix flaky MOR unit test * Update Spark APIs to make it be compatible with both spark2 & spark3 * Refactor bulk insert v2 part to make Hudi be able to compile with Spark3 * Add spark3 profile to handle fasterxml & spark version * Create hudi-spark-common module & refactor hudi-spark related modules Co-authored-by: Wenning Ding <wenningd@amazon.com>
This commit is contained in:
@@ -0,0 +1,275 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* 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;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.hive.MultiPartKeysValueExtractor;
|
||||
import org.apache.hudi.hive.NonPartitionedExtractor;
|
||||
import org.apache.hudi.keygen.NonpartitionedKeyGenerator;
|
||||
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.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.DataFrameWriter;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.SaveMode;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings;
|
||||
import static org.apache.hudi.common.testutils.Transformations.randomSelectAsHoodieKeys;
|
||||
|
||||
/**
|
||||
* Sample program that writes & reads hoodie tables via the Spark datasource.
|
||||
*/
|
||||
public class HoodieJavaApp {
|
||||
|
||||
@Parameter(names = {"--table-path", "-p"}, description = "path for Hoodie sample table")
|
||||
private String tablePath = "file:///tmp/hoodie/sample-table";
|
||||
|
||||
@Parameter(names = {"--table-name", "-n"}, description = "table name for Hoodie sample table")
|
||||
private String tableName = "hoodie_test";
|
||||
|
||||
@Parameter(names = {"--table-type", "-t"}, description = "One of COPY_ON_WRITE or MERGE_ON_READ")
|
||||
private String tableType = HoodieTableType.COPY_ON_WRITE.name();
|
||||
|
||||
@Parameter(names = {"--hive-sync", "-hv"}, description = "Enable syncing to hive")
|
||||
private Boolean enableHiveSync = false;
|
||||
|
||||
@Parameter(names = {"--hive-db", "-hd"}, description = "hive database")
|
||||
private String hiveDB = "default";
|
||||
|
||||
@Parameter(names = {"--hive-table", "-ht"}, description = "hive table")
|
||||
private String hiveTable = "hoodie_sample_test";
|
||||
|
||||
@Parameter(names = {"--hive-user", "-hu"}, description = "hive username")
|
||||
private String hiveUser = "hive";
|
||||
|
||||
@Parameter(names = {"--hive-password", "-hp"}, description = "hive password")
|
||||
private String hivePass = "hive";
|
||||
|
||||
@Parameter(names = {"--hive-url", "-hl"}, description = "hive JDBC URL")
|
||||
private String hiveJdbcUrl = "jdbc:hive2://localhost:10000";
|
||||
|
||||
@Parameter(names = {"--non-partitioned", "-np"}, description = "Use non-partitioned Table")
|
||||
private Boolean nonPartitionedTable = false;
|
||||
|
||||
@Parameter(names = {"--use-multi-partition-keys", "-mp"}, description = "Use Multiple Partition Keys")
|
||||
private Boolean useMultiPartitionKeys = false;
|
||||
|
||||
@Parameter(names = {"--help", "-h"}, help = true)
|
||||
public Boolean help = false;
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieJavaApp.class);
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
HoodieJavaApp cli = new HoodieJavaApp();
|
||||
JCommander cmd = new JCommander(cli, null, args);
|
||||
|
||||
if (cli.help) {
|
||||
cmd.usage();
|
||||
System.exit(1);
|
||||
}
|
||||
cli.run();
|
||||
}
|
||||
|
||||
public void run() throws Exception {
|
||||
|
||||
// Spark session setup..
|
||||
SparkSession spark = SparkSession.builder().appName("Hoodie Spark APP")
|
||||
.config("spark.serializer", "org.apache.spark.serializer.KryoSerializer").master("local[1]").getOrCreate();
|
||||
JavaSparkContext jssc = new JavaSparkContext(spark.sparkContext());
|
||||
spark.sparkContext().setLogLevel("WARN");
|
||||
FileSystem fs = FileSystem.get(jssc.hadoopConfiguration());
|
||||
|
||||
// Generator of some records to be loaded in.
|
||||
HoodieTestDataGenerator dataGen = null;
|
||||
if (nonPartitionedTable) {
|
||||
// All data goes to base-path
|
||||
dataGen = new HoodieTestDataGenerator(new String[]{""});
|
||||
} else {
|
||||
dataGen = new HoodieTestDataGenerator();
|
||||
}
|
||||
|
||||
// Explicitly clear up the hoodie table path if it exists.
|
||||
fs.delete(new Path(tablePath), true);
|
||||
|
||||
/**
|
||||
* Commit with only inserts
|
||||
*/
|
||||
// Generate some input..
|
||||
List<HoodieRecord> recordsSoFar = new ArrayList<>(dataGen.generateInserts("001"/* ignore */, 100));
|
||||
List<String> records1 = recordsToStrings(recordsSoFar);
|
||||
Dataset<Row> inputDF1 = spark.read().json(jssc.parallelize(records1, 2));
|
||||
|
||||
// Save as hoodie dataset (copy on write)
|
||||
// specify the hoodie source
|
||||
DataFrameWriter<Row> writer = inputDF1.write().format("org.apache.hudi")
|
||||
// any hoodie client config can be passed like this
|
||||
.option("hoodie.insert.shuffle.parallelism", "2")
|
||||
// full list in HoodieWriteConfig & its package
|
||||
.option("hoodie.upsert.shuffle.parallelism", "2")
|
||||
// Hoodie Table Type
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY(), tableType)
|
||||
// insert
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY(), DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL())
|
||||
// This is the record key
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key")
|
||||
// this is the partition to place it into
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition")
|
||||
// use to combine duplicate records in input/with disk val
|
||||
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY(), "timestamp")
|
||||
// Used by hive sync and queries
|
||||
.option(HoodieWriteConfig.TABLE_NAME, tableName)
|
||||
// Add Key Extractor
|
||||
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(),
|
||||
nonPartitionedTable ? NonpartitionedKeyGenerator.class.getCanonicalName()
|
||||
: SimpleKeyGenerator.class.getCanonicalName())
|
||||
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_OPT_KEY(), "false")
|
||||
// This will remove any existing data at path below, and create a
|
||||
.mode(SaveMode.Overwrite);
|
||||
|
||||
updateHiveSyncConfig(writer);
|
||||
// new dataset if needed
|
||||
writer.save(tablePath); // ultimately where the dataset will be placed
|
||||
String commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, tablePath);
|
||||
LOG.info("First commit at instant time :" + commitInstantTime1);
|
||||
|
||||
/**
|
||||
* Commit that updates records
|
||||
*/
|
||||
List<HoodieRecord> recordsToBeUpdated = dataGen.generateUpdates("002"/* ignore */, 100);
|
||||
recordsSoFar.addAll(recordsToBeUpdated);
|
||||
List<String> records2 = recordsToStrings(recordsToBeUpdated);
|
||||
Dataset<Row> inputDF2 = spark.read().json(jssc.parallelize(records2, 2));
|
||||
writer = inputDF2.write().format("org.apache.hudi").option("hoodie.insert.shuffle.parallelism", "2")
|
||||
.option("hoodie.upsert.shuffle.parallelism", "2")
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY(), tableType) // Hoodie Table Type
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key")
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition")
|
||||
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY(), "timestamp")
|
||||
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(),
|
||||
nonPartitionedTable ? NonpartitionedKeyGenerator.class.getCanonicalName()
|
||||
: SimpleKeyGenerator.class.getCanonicalName()) // Add Key Extractor
|
||||
.option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, "1")
|
||||
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_OPT_KEY(), "false")
|
||||
.option(HoodieWriteConfig.TABLE_NAME, tableName).mode(SaveMode.Append);
|
||||
|
||||
updateHiveSyncConfig(writer);
|
||||
writer.save(tablePath);
|
||||
String commitInstantTime2 = HoodieDataSourceHelpers.latestCommit(fs, tablePath);
|
||||
LOG.info("Second commit at instant time :" + commitInstantTime2);
|
||||
|
||||
/**
|
||||
* Commit that Deletes some records
|
||||
*/
|
||||
List<String> deletes = randomSelectAsHoodieKeys(recordsSoFar, 20).stream()
|
||||
.map(hr -> "{\"_row_key\":\"" + hr.getRecordKey() + "\",\"partition\":\"" + hr.getPartitionPath() + "\"}")
|
||||
.collect(Collectors.toList());
|
||||
Dataset<Row> inputDF3 = spark.read().json(jssc.parallelize(deletes, 2));
|
||||
writer = inputDF3.write().format("org.apache.hudi").option("hoodie.insert.shuffle.parallelism", "2")
|
||||
.option("hoodie.upsert.shuffle.parallelism", "2")
|
||||
.option("hoodie.delete.shuffle.parallelism", "2")
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY(), tableType) // Hoodie Table Type
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY(), "delete")
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key")
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition")
|
||||
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY(), "_row_key")
|
||||
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(),
|
||||
nonPartitionedTable ? NonpartitionedKeyGenerator.class.getCanonicalName()
|
||||
: SimpleKeyGenerator.class.getCanonicalName()) // Add Key Extractor
|
||||
.option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, "1")
|
||||
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_OPT_KEY(), "false")
|
||||
.option(HoodieWriteConfig.TABLE_NAME, tableName).mode(SaveMode.Append);
|
||||
|
||||
updateHiveSyncConfig(writer);
|
||||
writer.save(tablePath);
|
||||
String commitInstantTime3 = HoodieDataSourceHelpers.latestCommit(fs, tablePath);
|
||||
LOG.info("Third commit at instant time :" + commitInstantTime3);
|
||||
|
||||
/**
|
||||
* Read & do some queries
|
||||
*/
|
||||
Dataset<Row> snapshotQueryDF = spark.read().format("org.apache.hudi")
|
||||
// pass any path glob, can include hoodie & non-hoodie
|
||||
// datasets
|
||||
.load(tablePath + (nonPartitionedTable ? "/*" : "/*/*/*/*"));
|
||||
snapshotQueryDF.registerTempTable("hoodie_ro");
|
||||
spark.sql("describe hoodie_ro").show();
|
||||
// all trips whose fare amount was greater than 2.
|
||||
spark.sql("select fare.amount, begin_lon, begin_lat, timestamp from hoodie_ro where fare.amount > 2.0").show();
|
||||
|
||||
if (tableType.equals(HoodieTableType.COPY_ON_WRITE.name())) {
|
||||
/**
|
||||
* Consume incrementally, only changes in commit 2 above. Currently only supported for COPY_ON_WRITE TABLE
|
||||
*/
|
||||
Dataset<Row> incQueryDF = spark.read().format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY(), DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL())
|
||||
// Only changes in write 2 above
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY(), commitInstantTime1)
|
||||
// For incremental view, pass in the root/base path of dataset
|
||||
.load(tablePath);
|
||||
|
||||
LOG.info("You will only see records from : " + commitInstantTime2);
|
||||
incQueryDF.groupBy(incQueryDF.col("_hoodie_commit_time")).count().show();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Setup configs for syncing to hive.
|
||||
*/
|
||||
private DataFrameWriter<Row> updateHiveSyncConfig(DataFrameWriter<Row> writer) {
|
||||
if (enableHiveSync) {
|
||||
LOG.info("Enabling Hive sync to " + hiveJdbcUrl);
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY(), hiveTable)
|
||||
.option(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY(), hiveDB)
|
||||
.option(DataSourceWriteOptions.HIVE_URL_OPT_KEY(), hiveJdbcUrl)
|
||||
.option(DataSourceWriteOptions.HIVE_USER_OPT_KEY(), hiveUser)
|
||||
.option(DataSourceWriteOptions.HIVE_PASS_OPT_KEY(), hivePass)
|
||||
.option(DataSourceWriteOptions.HIVE_SYNC_ENABLED_OPT_KEY(), "true");
|
||||
if (nonPartitionedTable) {
|
||||
writer = writer
|
||||
.option(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY(),
|
||||
NonPartitionedExtractor.class.getCanonicalName())
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "");
|
||||
} else if (useMultiPartitionKeys) {
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY(), "year,month,day").option(
|
||||
DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY(),
|
||||
MultiPartKeysValueExtractor.class.getCanonicalName());
|
||||
} else {
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY(), "dateStr");
|
||||
}
|
||||
}
|
||||
return writer;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,192 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.hudi.DataSourceWriteOptions;
|
||||
import org.apache.hudi.HoodieDataSourceHelpers;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.hive.MultiPartKeysValueExtractor;
|
||||
import org.apache.hudi.hive.NonPartitionedExtractor;
|
||||
import org.apache.hudi.keygen.NonpartitionedKeyGenerator;
|
||||
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.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.DataFrameWriter;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import static org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings;
|
||||
|
||||
public class HoodieJavaGenerateApp {
|
||||
@Parameter(names = {"--table-path", "-p"}, description = "Path for Hoodie sample table")
|
||||
private String tablePath = "file:///tmp/hoodie/sample-table";
|
||||
|
||||
@Parameter(names = {"--table-name", "-n"}, description = "Table name for Hoodie sample table")
|
||||
private String tableName = "hoodie_test";
|
||||
|
||||
@Parameter(names = {"--table-type", "-t"}, description = "One of COPY_ON_WRITE or MERGE_ON_READ")
|
||||
private String tableType = HoodieTableType.COPY_ON_WRITE.name();
|
||||
|
||||
@Parameter(names = {"--hive-sync", "-hs"}, description = "Enable syncing to hive")
|
||||
private Boolean enableHiveSync = false;
|
||||
|
||||
@Parameter(names = {"--hive-db", "-hd"}, description = "Hive database")
|
||||
private String hiveDB = "default";
|
||||
|
||||
@Parameter(names = {"--hive-table", "-ht"}, description = "Hive table")
|
||||
private String hiveTable = "hoodie_sample_test";
|
||||
|
||||
@Parameter(names = {"--hive-user", "-hu"}, description = "Hive username")
|
||||
private String hiveUser = "hive";
|
||||
|
||||
@Parameter(names = {"--hive-password", "-hp"}, description = "Hive password")
|
||||
private String hivePass = "hive";
|
||||
|
||||
@Parameter(names = {"--hive-url", "-hl"}, description = "Hive JDBC URL")
|
||||
private String hiveJdbcUrl = "jdbc:hive2://localhost:10000";
|
||||
|
||||
@Parameter(names = {"--non-partitioned", "-np"}, description = "Use non-partitioned Table")
|
||||
private Boolean nonPartitionedTable = false;
|
||||
|
||||
@Parameter(names = {"--use-multi-partition-keys", "-mp"}, description = "Use Multiple Partition Keys")
|
||||
private Boolean useMultiPartitionKeys = false;
|
||||
|
||||
@Parameter(names = {"--commit-type", "-ct"}, description = "How may commits will run")
|
||||
private String commitType = "overwrite";
|
||||
|
||||
@Parameter(names = {"--help", "-h"}, help = true)
|
||||
public Boolean help = false;
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieJavaGenerateApp.class);
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
HoodieJavaGenerateApp cli = new HoodieJavaGenerateApp();
|
||||
JCommander cmd = new JCommander(cli, null, args);
|
||||
|
||||
if (cli.help) {
|
||||
cmd.usage();
|
||||
System.exit(1);
|
||||
}
|
||||
try (SparkSession spark = cli.getOrCreateSparkSession()) {
|
||||
cli.insert(spark);
|
||||
}
|
||||
}
|
||||
|
||||
private SparkSession getOrCreateSparkSession() {
|
||||
// Spark session setup..
|
||||
SparkSession spark = SparkSession.builder().appName("Hoodie Spark APP")
|
||||
.config("spark.serializer", "org.apache.spark.serializer.KryoSerializer").master("local[1]").getOrCreate();
|
||||
spark.sparkContext().setLogLevel("WARN");
|
||||
return spark;
|
||||
}
|
||||
|
||||
private HoodieTestDataGenerator getDataGenerate() {
|
||||
// Generator of some records to be loaded in.
|
||||
if (nonPartitionedTable) {
|
||||
// All data goes to base-path
|
||||
return new HoodieTestDataGenerator(new String[]{""});
|
||||
} else {
|
||||
return new HoodieTestDataGenerator();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Setup configs for syncing to hive.
|
||||
*/
|
||||
private DataFrameWriter<Row> updateHiveSyncConfig(DataFrameWriter<Row> writer) {
|
||||
if (enableHiveSync) {
|
||||
LOG.info("Enabling Hive sync to " + hiveJdbcUrl);
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY(), hiveTable)
|
||||
.option(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY(), hiveDB)
|
||||
.option(DataSourceWriteOptions.HIVE_URL_OPT_KEY(), hiveJdbcUrl)
|
||||
.option(DataSourceWriteOptions.HIVE_USER_OPT_KEY(), hiveUser)
|
||||
.option(DataSourceWriteOptions.HIVE_PASS_OPT_KEY(), hivePass)
|
||||
.option(DataSourceWriteOptions.HIVE_SYNC_ENABLED_OPT_KEY(), "true");
|
||||
if (nonPartitionedTable) {
|
||||
writer = writer
|
||||
.option(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY(),
|
||||
NonPartitionedExtractor.class.getCanonicalName())
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "");
|
||||
} else if (useMultiPartitionKeys) {
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY(), "year,month,day").option(
|
||||
DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY(),
|
||||
MultiPartKeysValueExtractor.class.getCanonicalName());
|
||||
} else {
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY(), "dateStr");
|
||||
}
|
||||
}
|
||||
return writer;
|
||||
}
|
||||
|
||||
private void insert(SparkSession spark) throws IOException {
|
||||
HoodieTestDataGenerator dataGen = getDataGenerate();
|
||||
|
||||
JavaSparkContext jssc = new JavaSparkContext(spark.sparkContext());
|
||||
|
||||
// Generate some input..
|
||||
String instantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
List<HoodieRecord> recordsSoFar = new ArrayList<>(dataGen.generateInserts(instantTime/* ignore */, 100));
|
||||
List<String> records1 = recordsToStrings(recordsSoFar);
|
||||
Dataset<Row> inputDF1 = spark.read().json(jssc.parallelize(records1, 2));
|
||||
|
||||
// Save as hoodie dataset (copy on write)
|
||||
// specify the hoodie source
|
||||
DataFrameWriter<Row> writer = inputDF1.write().format("org.apache.hudi")
|
||||
// any hoodie client config can be passed like this
|
||||
.option("hoodie.insert.shuffle.parallelism", "2")
|
||||
// full list in HoodieWriteConfig & its package
|
||||
.option("hoodie.upsert.shuffle.parallelism", "2")
|
||||
// Hoodie Table Type
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY(), tableType)
|
||||
// insert
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY(), DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL())
|
||||
// This is the record key
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key")
|
||||
// this is the partition to place it into
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition")
|
||||
// use to combine duplicate records in input/with disk val
|
||||
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY(), "timestamp")
|
||||
// Used by hive sync and queries
|
||||
.option(HoodieWriteConfig.TABLE_NAME, tableName)
|
||||
// Add Key Extractor
|
||||
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(),
|
||||
nonPartitionedTable ? NonpartitionedKeyGenerator.class.getCanonicalName()
|
||||
: SimpleKeyGenerator.class.getCanonicalName())
|
||||
.mode(commitType);
|
||||
|
||||
updateHiveSyncConfig(writer);
|
||||
// new dataset if needed
|
||||
writer.save(tablePath); // ultimately where the dataset will be placed
|
||||
FileSystem fs = FileSystem.get(jssc.hadoopConfiguration());
|
||||
String commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, tablePath);
|
||||
LOG.info("Commit at instant time :" + commitInstantTime1);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,398 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hudi.DataSourceReadOptions;
|
||||
import org.apache.hudi.DataSourceWriteOptions;
|
||||
import org.apache.hudi.HoodieDataSourceHelpers;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.TableNotFoundException;
|
||||
import org.apache.hudi.hive.MultiPartKeysValueExtractor;
|
||||
|
||||
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;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Row;
|
||||
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.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 static org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings;
|
||||
|
||||
/**
|
||||
* Sample program that writes & reads hoodie tables via the Spark datasource streaming.
|
||||
*/
|
||||
public class HoodieJavaStreamingApp {
|
||||
|
||||
@Parameter(names = {"--table-path", "-p"}, description = "path for Hoodie sample table")
|
||||
private String tablePath = "/tmp/hoodie/streaming/sample-table";
|
||||
|
||||
@Parameter(names = {"--streaming-source-path", "-ssp"}, description = "path for streaming source file folder")
|
||||
private String streamingSourcePath = "/tmp/hoodie/streaming/source";
|
||||
|
||||
@Parameter(names = {"--streaming-checkpointing-path", "-scp"},
|
||||
description = "path for streaming checking pointing folder")
|
||||
private String streamingCheckpointingPath = "/tmp/hoodie/streaming/checkpoint";
|
||||
|
||||
@Parameter(names = {"--streaming-duration-in-ms", "-sdm"},
|
||||
description = "time in millisecond for the streaming duration")
|
||||
private Long streamingDurationInMs = 15000L;
|
||||
|
||||
@Parameter(names = {"--table-name", "-n"}, description = "table name for Hoodie sample table")
|
||||
private String tableName = "hoodie_test";
|
||||
|
||||
@Parameter(names = {"--table-type", "-t"}, description = "One of COPY_ON_WRITE or MERGE_ON_READ")
|
||||
private String tableType = HoodieTableType.MERGE_ON_READ.name();
|
||||
|
||||
@Parameter(names = {"--hive-sync", "-hv"}, description = "Enable syncing to hive")
|
||||
private Boolean enableHiveSync = false;
|
||||
|
||||
@Parameter(names = {"--hive-db", "-hd"}, description = "hive database")
|
||||
private String hiveDB = "default";
|
||||
|
||||
@Parameter(names = {"--hive-table", "-ht"}, description = "hive table")
|
||||
private String hiveTable = "hoodie_sample_test";
|
||||
|
||||
@Parameter(names = {"--hive-user", "-hu"}, description = "hive username")
|
||||
private String hiveUser = "hive";
|
||||
|
||||
@Parameter(names = {"--hive-password", "-hp"}, description = "hive password")
|
||||
private String hivePass = "hive";
|
||||
|
||||
@Parameter(names = {"--hive-url", "-hl"}, description = "hive JDBC URL")
|
||||
private String hiveJdbcUrl = "jdbc:hive2://localhost:10000";
|
||||
|
||||
@Parameter(names = {"--use-multi-partition-keys", "-mp"}, description = "Use Multiple Partition Keys")
|
||||
private Boolean useMultiPartitionKeys = false;
|
||||
|
||||
@Parameter(names = {"--help", "-h"}, help = true)
|
||||
public Boolean help = false;
|
||||
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieJavaStreamingApp.class);
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
HoodieJavaStreamingApp cli = new HoodieJavaStreamingApp();
|
||||
JCommander cmd = new JCommander(cli, null, args);
|
||||
|
||||
if (cli.help) {
|
||||
cmd.usage();
|
||||
System.exit(1);
|
||||
}
|
||||
int errStatus = 0;
|
||||
try {
|
||||
cli.run();
|
||||
} catch (Exception ex) {
|
||||
LOG.error("Got error running app ", ex);
|
||||
errStatus = -1;
|
||||
} finally {
|
||||
System.exit(errStatus);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @throws Exception
|
||||
*/
|
||||
public void run() throws Exception {
|
||||
// Spark session setup..
|
||||
SparkSession spark = SparkSession.builder().appName("Hoodie Spark Streaming APP")
|
||||
.config("spark.serializer", "org.apache.spark.serializer.KryoSerializer").master("local[1]").getOrCreate();
|
||||
JavaSparkContext jssc = new JavaSparkContext(spark.sparkContext());
|
||||
|
||||
// folder path clean up and creation, preparing the environment
|
||||
FileSystem fs = FileSystem.get(jssc.hadoopConfiguration());
|
||||
fs.delete(new Path(streamingSourcePath), true);
|
||||
fs.delete(new Path(streamingCheckpointingPath), true);
|
||||
fs.delete(new Path(tablePath), true);
|
||||
fs.mkdirs(new Path(streamingSourcePath));
|
||||
|
||||
// Generator of some records to be loaded in.
|
||||
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
|
||||
|
||||
List<String> records1 = recordsToStrings(dataGen.generateInserts("001", 100));
|
||||
Dataset<Row> inputDF1 = spark.read().json(jssc.parallelize(records1, 2));
|
||||
|
||||
List<String> records2 = recordsToStrings(dataGen.generateUpdatesForAllRecords("002"));
|
||||
Dataset<Row> inputDF2 = spark.read().json(jssc.parallelize(records2, 2));
|
||||
|
||||
|
||||
String ckptPath = streamingCheckpointingPath + "/stream1";
|
||||
String srcPath = streamingSourcePath + "/stream1";
|
||||
fs.mkdirs(new Path(ckptPath));
|
||||
fs.mkdirs(new Path(srcPath));
|
||||
|
||||
// setup the input for streaming
|
||||
Dataset<Row> streamingInput = spark.readStream().schema(inputDF1.schema()).json(srcPath + "/*");
|
||||
|
||||
// start streaming and showing
|
||||
ExecutorService executor = Executors.newFixedThreadPool(2);
|
||||
int numInitialCommits = 0;
|
||||
|
||||
// thread for spark strucutured streaming
|
||||
try {
|
||||
Future<Void> streamFuture = executor.submit(() -> {
|
||||
LOG.info("===== Streaming Starting =====");
|
||||
stream(streamingInput, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL(), ckptPath);
|
||||
LOG.info("===== Streaming Ends =====");
|
||||
return null;
|
||||
});
|
||||
|
||||
// thread for adding data to the streaming source and showing results over time
|
||||
Future<Integer> showFuture = executor.submit(() -> {
|
||||
LOG.info("===== Showing Starting =====");
|
||||
int numCommits = addInputAndValidateIngestion(spark, fs, srcPath,0, 100, inputDF1, inputDF2, true);
|
||||
LOG.info("===== Showing Ends =====");
|
||||
return numCommits;
|
||||
});
|
||||
|
||||
// let the threads run
|
||||
streamFuture.get();
|
||||
numInitialCommits = showFuture.get();
|
||||
} finally {
|
||||
executor.shutdownNow();
|
||||
}
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jssc.hadoopConfiguration(), tablePath);
|
||||
if (tableType.equals(HoodieTableType.MERGE_ON_READ.name())) {
|
||||
// Ensure we have successfully completed one compaction commit
|
||||
ValidationUtils.checkArgument(metaClient.getActiveTimeline().getCommitTimeline().getInstants().count() == 1);
|
||||
} else {
|
||||
ValidationUtils.checkArgument(metaClient.getActiveTimeline().getCommitTimeline().getInstants().count() >= 1);
|
||||
}
|
||||
|
||||
// Deletes Stream
|
||||
// Need to restart application to ensure spark does not assume there are multiple streams active.
|
||||
spark.close();
|
||||
SparkSession newSpark = SparkSession.builder().appName("Hoodie Spark Streaming APP")
|
||||
.config("spark.serializer", "org.apache.spark.serializer.KryoSerializer").master("local[1]").getOrCreate();
|
||||
jssc = new JavaSparkContext(newSpark.sparkContext());
|
||||
String ckptPath2 = streamingCheckpointingPath + "/stream2";
|
||||
String srcPath2 = srcPath + "/stream2";
|
||||
fs.mkdirs(new Path(ckptPath2));
|
||||
fs.mkdirs(new Path(srcPath2));
|
||||
Dataset<Row> delStreamingInput = newSpark.readStream().schema(inputDF1.schema()).json(srcPath2 + "/*");
|
||||
List<String> deletes = recordsToStrings(dataGen.generateUniqueUpdates("002", 20));
|
||||
Dataset<Row> inputDF3 = newSpark.read().json(jssc.parallelize(deletes, 2));
|
||||
executor = Executors.newFixedThreadPool(2);
|
||||
|
||||
// thread for spark strucutured streaming
|
||||
try {
|
||||
Future<Void> streamFuture = executor.submit(() -> {
|
||||
LOG.info("===== Streaming Starting =====");
|
||||
stream(delStreamingInput, DataSourceWriteOptions.DELETE_OPERATION_OPT_VAL(), ckptPath2);
|
||||
LOG.info("===== Streaming Ends =====");
|
||||
return null;
|
||||
});
|
||||
|
||||
final int numCommits = numInitialCommits;
|
||||
// thread for adding data to the streaming source and showing results over time
|
||||
Future<Void> showFuture = executor.submit(() -> {
|
||||
LOG.info("===== Showing Starting =====");
|
||||
addInputAndValidateIngestion(newSpark, fs, srcPath2, numCommits, 80, inputDF3, null, false);
|
||||
LOG.info("===== Showing Ends =====");
|
||||
return null;
|
||||
});
|
||||
|
||||
// let the threads run
|
||||
streamFuture.get();
|
||||
showFuture.get();
|
||||
} finally {
|
||||
executor.shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
private void waitTillNCommits(FileSystem fs, int numCommits, int timeoutSecs, int sleepSecsAfterEachRun)
|
||||
throws InterruptedException {
|
||||
long beginTime = System.currentTimeMillis();
|
||||
long currTime = beginTime;
|
||||
long timeoutMsecs = timeoutSecs * 1000;
|
||||
|
||||
while ((currTime - beginTime) < timeoutMsecs) {
|
||||
try {
|
||||
HoodieTimeline timeline = HoodieDataSourceHelpers.allCompletedCommitsCompactions(fs, tablePath);
|
||||
LOG.info("Timeline :" + timeline.getInstants().collect(Collectors.toList()));
|
||||
if (timeline.countInstants() >= numCommits) {
|
||||
return;
|
||||
}
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), tablePath, true);
|
||||
System.out.println("Instants :" + metaClient.getActiveTimeline().getInstants().collect(Collectors.toList()));
|
||||
} catch (TableNotFoundException te) {
|
||||
LOG.info("Got table not found exception. Retrying");
|
||||
} finally {
|
||||
Thread.sleep(sleepSecsAfterEachRun * 1000);
|
||||
currTime = System.currentTimeMillis();
|
||||
}
|
||||
}
|
||||
throw new IllegalStateException("Timedout waiting for " + numCommits + " commits to appear in " + tablePath);
|
||||
}
|
||||
|
||||
/**
|
||||
* Adding data to the streaming source and showing results over time.
|
||||
*
|
||||
* @param spark
|
||||
* @param fs
|
||||
* @param inputDF1
|
||||
* @param inputDF2
|
||||
* @throws Exception
|
||||
*/
|
||||
public int addInputAndValidateIngestion(SparkSession spark, FileSystem fs, String srcPath,
|
||||
int initialCommits, int expRecords,
|
||||
Dataset<Row> inputDF1, Dataset<Row> inputDF2, boolean instantTimeValidation) throws Exception {
|
||||
// Ensure, we always write only one file. This is very important to ensure a single batch is reliably read
|
||||
// atomically by one iteration of spark streaming.
|
||||
inputDF1.coalesce(1).write().mode(SaveMode.Append).json(srcPath);
|
||||
|
||||
int numExpCommits = initialCommits + 1;
|
||||
// wait for spark streaming to process one microbatch
|
||||
waitTillNCommits(fs, numExpCommits, 180, 3);
|
||||
String commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, tablePath);
|
||||
LOG.info("First commit at instant time :" + commitInstantTime1);
|
||||
|
||||
String commitInstantTime2 = commitInstantTime1;
|
||||
if (null != inputDF2) {
|
||||
numExpCommits += 1;
|
||||
inputDF2.write().mode(SaveMode.Append).json(srcPath);
|
||||
// wait for spark streaming to process one microbatch
|
||||
Thread.sleep(3000);
|
||||
waitTillNCommits(fs, numExpCommits, 180, 3);
|
||||
commitInstantTime2 = HoodieDataSourceHelpers.latestCommit(fs, tablePath);
|
||||
LOG.info("Second commit at instant time :" + commitInstantTime2);
|
||||
}
|
||||
|
||||
if (tableType.equals(HoodieTableType.MERGE_ON_READ.name())) {
|
||||
numExpCommits += 1;
|
||||
// Wait for compaction to also finish and track latest timestamp as commit timestamp
|
||||
waitTillNCommits(fs, numExpCommits, 180, 3);
|
||||
commitInstantTime2 = HoodieDataSourceHelpers.latestCommit(fs, tablePath);
|
||||
LOG.info("Compaction commit at instant time :" + commitInstantTime2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Read & do some queries
|
||||
*/
|
||||
Dataset<Row> hoodieROViewDF = spark.read().format("hudi")
|
||||
// pass any path glob, can include hoodie & non-hoodie
|
||||
// datasets
|
||||
.load(tablePath + "/*/*/*/*");
|
||||
hoodieROViewDF.registerTempTable("hoodie_ro");
|
||||
spark.sql("describe hoodie_ro").show();
|
||||
// all trips whose fare amount was greater than 2.
|
||||
spark.sql("select fare.amount, begin_lon, begin_lat, timestamp from hoodie_ro where fare.amount > 2.0").show();
|
||||
|
||||
if (instantTimeValidation) {
|
||||
System.out.println("Showing all records. Latest Instant Time =" + commitInstantTime2);
|
||||
spark.sql("select * from hoodie_ro").show(200, false);
|
||||
long numRecordsAtInstant2 =
|
||||
spark.sql("select * from hoodie_ro where _hoodie_commit_time = " + commitInstantTime2).count();
|
||||
ValidationUtils.checkArgument(numRecordsAtInstant2 == expRecords,
|
||||
"Expecting " + expRecords + " records, Got " + numRecordsAtInstant2);
|
||||
}
|
||||
|
||||
long numRecords = spark.sql("select * from hoodie_ro").count();
|
||||
ValidationUtils.checkArgument(numRecords == expRecords,
|
||||
"Expecting " + expRecords + " records, Got " + numRecords);
|
||||
|
||||
if (tableType.equals(HoodieTableType.COPY_ON_WRITE.name())) {
|
||||
/**
|
||||
* Consume incrementally, only changes in commit 2 above. Currently only supported for COPY_ON_WRITE TABLE
|
||||
*/
|
||||
Dataset<Row> hoodieIncViewDF = spark.read().format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY(), DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL())
|
||||
// Only changes in write 2 above
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY(), commitInstantTime1)
|
||||
// For incremental view, pass in the root/base path of dataset
|
||||
.load(tablePath);
|
||||
|
||||
LOG.info("You will only see records from : " + commitInstantTime2);
|
||||
hoodieIncViewDF.groupBy(hoodieIncViewDF.col("_hoodie_commit_time")).count().show();
|
||||
}
|
||||
return numExpCommits;
|
||||
}
|
||||
|
||||
/**
|
||||
* Hoodie spark streaming job.
|
||||
*
|
||||
* @param streamingInput
|
||||
* @throws Exception
|
||||
*/
|
||||
public void stream(Dataset<Row> streamingInput, String operationType, String checkpointLocation) throws Exception {
|
||||
|
||||
DataStreamWriter<Row> writer = streamingInput.writeStream().format("org.apache.hudi")
|
||||
.option("hoodie.insert.shuffle.parallelism", "2").option("hoodie.upsert.shuffle.parallelism", "2")
|
||||
.option("hoodie.delete.shuffle.parallelism", "2")
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY(), operationType)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY(), tableType)
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key")
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition")
|
||||
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY(), "timestamp")
|
||||
.option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, "1")
|
||||
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_OPT_KEY(), "true")
|
||||
.option(HoodieWriteConfig.TABLE_NAME, tableName).option("checkpointLocation", checkpointLocation)
|
||||
.outputMode(OutputMode.Append());
|
||||
|
||||
updateHiveSyncConfig(writer);
|
||||
StreamingQuery query = writer.trigger(Trigger.ProcessingTime(500)).start(tablePath);
|
||||
query.awaitTermination(streamingDurationInMs);
|
||||
}
|
||||
|
||||
/**
|
||||
* Setup configs for syncing to hive.
|
||||
*
|
||||
* @param writer
|
||||
* @return
|
||||
*/
|
||||
private DataStreamWriter<Row> updateHiveSyncConfig(DataStreamWriter<Row> writer) {
|
||||
if (enableHiveSync) {
|
||||
LOG.info("Enabling Hive sync to " + hiveJdbcUrl);
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY(), hiveTable)
|
||||
.option(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY(), hiveDB)
|
||||
.option(DataSourceWriteOptions.HIVE_URL_OPT_KEY(), hiveJdbcUrl)
|
||||
.option(DataSourceWriteOptions.HIVE_USER_OPT_KEY(), hiveUser)
|
||||
.option(DataSourceWriteOptions.HIVE_PASS_OPT_KEY(), hivePass)
|
||||
.option(DataSourceWriteOptions.HIVE_SYNC_ENABLED_OPT_KEY(), "true");
|
||||
if (useMultiPartitionKeys) {
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY(), "year,month,day").option(
|
||||
DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY(),
|
||||
MultiPartKeysValueExtractor.class.getCanonicalName());
|
||||
} else {
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY(), "dateStr");
|
||||
}
|
||||
}
|
||||
return writer;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,187 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi;
|
||||
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.client.SparkRDDWriteClient;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.table.BulkInsertPartitioner;
|
||||
|
||||
import org.apache.avro.Conversions;
|
||||
import org.apache.avro.LogicalTypes;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericData;
|
||||
import org.apache.avro.generic.GenericFixed;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.extension.ExtendWith;
|
||||
import org.mockito.ArgumentCaptor;
|
||||
import org.mockito.Captor;
|
||||
import org.mockito.Mock;
|
||||
import org.mockito.junit.jupiter.MockitoExtension;
|
||||
|
||||
import java.math.BigDecimal;
|
||||
import java.time.LocalDate;
|
||||
|
||||
import static org.hamcrest.CoreMatchers.containsString;
|
||||
import static org.hamcrest.CoreMatchers.equalTo;
|
||||
import static org.hamcrest.CoreMatchers.instanceOf;
|
||||
import static org.hamcrest.CoreMatchers.is;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.mockito.ArgumentMatchers.any;
|
||||
import static org.mockito.ArgumentMatchers.anyString;
|
||||
import static org.mockito.Mockito.times;
|
||||
import static org.mockito.Mockito.verify;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
@ExtendWith(MockitoExtension.class)
|
||||
public class TestDataSourceUtils {
|
||||
|
||||
@Mock
|
||||
private SparkRDDWriteClient hoodieWriteClient;
|
||||
|
||||
@Mock
|
||||
private JavaRDD<HoodieRecord> hoodieRecords;
|
||||
|
||||
@Captor
|
||||
private ArgumentCaptor<Option> optionCaptor;
|
||||
private HoodieWriteConfig config;
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() {
|
||||
config = HoodieWriteConfig.newBuilder().withPath("/").build();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAvroRecordsFieldConversion() {
|
||||
// There are fields event_date1, event_date2, event_date3 with logical type as Date. event_date1 & event_date3 are
|
||||
// of UNION schema type, which is a union of null and date type in different orders. event_date2 is non-union
|
||||
// date type. event_cost1, event_cost2, event3 are decimal logical types with UNION schema, which is similar to
|
||||
// the event_date.
|
||||
String avroSchemaString = "{\"type\": \"record\"," + "\"name\": \"events\"," + "\"fields\": [ "
|
||||
+ "{\"name\": \"event_date1\", \"type\" : [{\"type\" : \"int\", \"logicalType\" : \"date\"}, \"null\"]},"
|
||||
+ "{\"name\": \"event_date2\", \"type\" : {\"type\": \"int\", \"logicalType\" : \"date\"}},"
|
||||
+ "{\"name\": \"event_date3\", \"type\" : [\"null\", {\"type\" : \"int\", \"logicalType\" : \"date\"}]},"
|
||||
+ "{\"name\": \"event_name\", \"type\": \"string\"},"
|
||||
+ "{\"name\": \"event_organizer\", \"type\": \"string\"},"
|
||||
+ "{\"name\": \"event_cost1\", \"type\": "
|
||||
+ "[{\"type\": \"fixed\", \"name\": \"dc\", \"size\": 5, \"logicalType\": \"decimal\", \"precision\": 10, \"scale\": 6}, \"null\"]},"
|
||||
+ "{\"name\": \"event_cost2\", \"type\": "
|
||||
+ "{\"type\": \"fixed\", \"name\": \"ef\", \"size\": 5, \"logicalType\": \"decimal\", \"precision\": 10, \"scale\": 6}},"
|
||||
+ "{\"name\": \"event_cost3\", \"type\": "
|
||||
+ "[\"null\", {\"type\": \"fixed\", \"name\": \"fg\", \"size\": 5, \"logicalType\": \"decimal\", \"precision\": 10, \"scale\": 6}]}"
|
||||
+ "]}";
|
||||
|
||||
Schema avroSchema = new Schema.Parser().parse(avroSchemaString);
|
||||
GenericRecord record = new GenericData.Record(avroSchema);
|
||||
record.put("event_date1", 18000);
|
||||
record.put("event_date2", 18001);
|
||||
record.put("event_date3", 18002);
|
||||
record.put("event_name", "Hudi Meetup");
|
||||
record.put("event_organizer", "Hudi PMC");
|
||||
|
||||
BigDecimal bigDecimal = new BigDecimal("123.184331");
|
||||
Schema decimalSchema = avroSchema.getField("event_cost1").schema().getTypes().get(0);
|
||||
Conversions.DecimalConversion decimalConversions = new Conversions.DecimalConversion();
|
||||
GenericFixed genericFixed = decimalConversions.toFixed(bigDecimal, decimalSchema, LogicalTypes.decimal(10, 6));
|
||||
record.put("event_cost1", genericFixed);
|
||||
record.put("event_cost2", genericFixed);
|
||||
record.put("event_cost3", genericFixed);
|
||||
|
||||
assertEquals(LocalDate.ofEpochDay(18000).toString(), HoodieAvroUtils.getNestedFieldValAsString(record, "event_date1",
|
||||
true));
|
||||
assertEquals(LocalDate.ofEpochDay(18001).toString(), HoodieAvroUtils.getNestedFieldValAsString(record, "event_date2",
|
||||
true));
|
||||
assertEquals(LocalDate.ofEpochDay(18002).toString(), HoodieAvroUtils.getNestedFieldValAsString(record, "event_date3",
|
||||
true));
|
||||
assertEquals("Hudi Meetup", HoodieAvroUtils.getNestedFieldValAsString(record, "event_name", true));
|
||||
assertEquals("Hudi PMC", HoodieAvroUtils.getNestedFieldValAsString(record, "event_organizer", true));
|
||||
assertEquals(bigDecimal.toString(), HoodieAvroUtils.getNestedFieldValAsString(record, "event_cost1", true));
|
||||
assertEquals(bigDecimal.toString(), HoodieAvroUtils.getNestedFieldValAsString(record, "event_cost2", true));
|
||||
assertEquals(bigDecimal.toString(), HoodieAvroUtils.getNestedFieldValAsString(record, "event_cost3", true));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDoWriteOperationWithoutUserDefinedBulkInsertPartitioner() throws HoodieException {
|
||||
when(hoodieWriteClient.getConfig()).thenReturn(config);
|
||||
|
||||
DataSourceUtils.doWriteOperation(hoodieWriteClient, hoodieRecords, "test-time",
|
||||
WriteOperationType.BULK_INSERT);
|
||||
|
||||
verify(hoodieWriteClient, times(1)).bulkInsert(any(hoodieRecords.getClass()), anyString(),
|
||||
optionCaptor.capture());
|
||||
assertThat(optionCaptor.getValue(), is(equalTo(Option.empty())));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDoWriteOperationWithNonExistUserDefinedBulkInsertPartitioner() throws HoodieException {
|
||||
setAndVerifyHoodieWriteClientWith("NonExistClassName");
|
||||
|
||||
Exception exception = assertThrows(HoodieException.class, () -> {
|
||||
DataSourceUtils.doWriteOperation(hoodieWriteClient, hoodieRecords, "test-time",
|
||||
WriteOperationType.BULK_INSERT);
|
||||
});
|
||||
|
||||
assertThat(exception.getMessage(), containsString("Could not create UserDefinedBulkInsertPartitioner"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDoWriteOperationWithUserDefinedBulkInsertPartitioner() throws HoodieException {
|
||||
setAndVerifyHoodieWriteClientWith(NoOpBulkInsertPartitioner.class.getName());
|
||||
|
||||
DataSourceUtils.doWriteOperation(hoodieWriteClient, hoodieRecords, "test-time",
|
||||
WriteOperationType.BULK_INSERT);
|
||||
|
||||
verify(hoodieWriteClient, times(1)).bulkInsert(any(hoodieRecords.getClass()), anyString(),
|
||||
optionCaptor.capture());
|
||||
assertThat(optionCaptor.getValue().get(), is(instanceOf(NoOpBulkInsertPartitioner.class)));
|
||||
}
|
||||
|
||||
private void setAndVerifyHoodieWriteClientWith(final String partitionerClassName) {
|
||||
config = HoodieWriteConfig.newBuilder().withPath(config.getBasePath())
|
||||
.withUserDefinedBulkInsertPartitionerClass(partitionerClassName)
|
||||
.build();
|
||||
when(hoodieWriteClient.getConfig()).thenReturn(config);
|
||||
|
||||
assertThat(config.getUserDefinedBulkInsertPartitionerClass(), is(equalTo(partitionerClassName)));
|
||||
}
|
||||
|
||||
public static class NoOpBulkInsertPartitioner<T extends HoodieRecordPayload>
|
||||
implements BulkInsertPartitioner<JavaRDD<HoodieRecord<T>>> {
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> repartitionRecords(JavaRDD<HoodieRecord<T>> records, int outputSparkPartitions) {
|
||||
return records;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean arePartitionRecordsSorted() {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,156 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.util.FileIOUtils;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.testutils.DataSourceTestUtils;
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.junit.jupiter.api.Assertions.fail;
|
||||
|
||||
/**
|
||||
* Tests {@link HoodieDatasetBulkInsertHelper}.
|
||||
*/
|
||||
public class TestHoodieDatasetBulkInsertHelper extends HoodieClientTestBase {
|
||||
|
||||
private String schemaStr;
|
||||
private Schema schema;
|
||||
private StructType structType;
|
||||
|
||||
public TestHoodieDatasetBulkInsertHelper() throws IOException {
|
||||
init();
|
||||
}
|
||||
|
||||
private void init() throws IOException {
|
||||
schemaStr = FileIOUtils.readAsUTFString(getClass().getResourceAsStream("/exampleSchema.txt"));
|
||||
schema = DataSourceTestUtils.getStructTypeExampleSchema();
|
||||
structType = AvroConversionUtils.convertAvroSchemaToStructType(schema);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBulkInsertHelper() throws IOException {
|
||||
HoodieWriteConfig config = getConfigBuilder(schemaStr).withProps(getPropsAllSet()).build();
|
||||
List<Row> rows = DataSourceTestUtils.generateRandomRows(10);
|
||||
Dataset<Row> dataset = sqlContext.createDataFrame(rows, structType);
|
||||
Dataset<Row> result = HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName", "testNamespace");
|
||||
StructType resultSchema = result.schema();
|
||||
|
||||
assertEquals(result.count(), 10);
|
||||
assertEquals(resultSchema.fieldNames().length, structType.fieldNames().length + HoodieRecord.HOODIE_META_COLUMNS.size());
|
||||
|
||||
for (Map.Entry<String, Integer> entry : HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.entrySet()) {
|
||||
assertTrue(resultSchema.fieldIndex(entry.getKey()) == entry.getValue());
|
||||
}
|
||||
|
||||
int metadataRecordKeyIndex = resultSchema.fieldIndex(HoodieRecord.RECORD_KEY_METADATA_FIELD);
|
||||
int metadataParitionPathIndex = resultSchema.fieldIndex(HoodieRecord.PARTITION_PATH_METADATA_FIELD);
|
||||
int metadataCommitTimeIndex = resultSchema.fieldIndex(HoodieRecord.COMMIT_TIME_METADATA_FIELD);
|
||||
int metadataCommitSeqNoIndex = resultSchema.fieldIndex(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD);
|
||||
int metadataFilenameIndex = resultSchema.fieldIndex(HoodieRecord.FILENAME_METADATA_FIELD);
|
||||
|
||||
result.toJavaRDD().foreach(entry -> {
|
||||
assertTrue(entry.get(metadataRecordKeyIndex).equals(entry.getAs("_row_key")));
|
||||
assertTrue(entry.get(metadataParitionPathIndex).equals(entry.getAs("partition")));
|
||||
assertTrue(entry.get(metadataCommitSeqNoIndex).equals(""));
|
||||
assertTrue(entry.get(metadataCommitTimeIndex).equals(""));
|
||||
assertTrue(entry.get(metadataFilenameIndex).equals(""));
|
||||
});
|
||||
}
|
||||
|
||||
private Map<String, String> getPropsAllSet() {
|
||||
return getProps(true, true, true, true);
|
||||
}
|
||||
|
||||
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_OPT_KEY(), "org.apache.hudi.keygen.SimpleKeyGenerator");
|
||||
props.put(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key");
|
||||
props.put(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition");
|
||||
} else {
|
||||
if (setKeyGen) {
|
||||
props.put(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(), "org.apache.hudi.keygen.SimpleKeyGenerator");
|
||||
}
|
||||
if (setRecordKey) {
|
||||
props.put(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key");
|
||||
}
|
||||
if (setPartitionPath) {
|
||||
props.put(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition");
|
||||
}
|
||||
}
|
||||
return props;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNoPropsSet() {
|
||||
HoodieWriteConfig config = getConfigBuilder(schemaStr).build();
|
||||
List<Row> rows = DataSourceTestUtils.generateRandomRows(10);
|
||||
Dataset<Row> dataset = sqlContext.createDataFrame(rows, structType);
|
||||
try {
|
||||
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName", "testNamespace");
|
||||
fail("Should have thrown exception");
|
||||
} catch (Exception e) {
|
||||
// ignore
|
||||
}
|
||||
|
||||
config = getConfigBuilder(schemaStr).withProps(getProps(false, false, true, true)).build();
|
||||
rows = DataSourceTestUtils.generateRandomRows(10);
|
||||
dataset = sqlContext.createDataFrame(rows, structType);
|
||||
try {
|
||||
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName", "testNamespace");
|
||||
fail("Should have thrown exception");
|
||||
} catch (Exception e) {
|
||||
// ignore
|
||||
}
|
||||
|
||||
config = getConfigBuilder(schemaStr).withProps(getProps(false, true, false, true)).build();
|
||||
rows = DataSourceTestUtils.generateRandomRows(10);
|
||||
dataset = sqlContext.createDataFrame(rows, structType);
|
||||
try {
|
||||
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName", "testNamespace");
|
||||
fail("Should have thrown exception");
|
||||
} catch (Exception e) {
|
||||
// ignore
|
||||
}
|
||||
|
||||
config = getConfigBuilder(schemaStr).withProps(getProps(false, true, true, false)).build();
|
||||
rows = DataSourceTestUtils.generateRandomRows(10);
|
||||
dataset = sqlContext.createDataFrame(rows, structType);
|
||||
try {
|
||||
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName", "testNamespace");
|
||||
fail("Should have thrown exception");
|
||||
} catch (Exception e) {
|
||||
// ignore
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,584 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.client;
|
||||
|
||||
import org.apache.hudi.DataSourceWriteOptions;
|
||||
import org.apache.hudi.avro.model.HoodieFileStatus;
|
||||
import org.apache.hudi.client.bootstrap.BootstrapMode;
|
||||
import org.apache.hudi.client.bootstrap.FullRecordBootstrapDataProvider;
|
||||
import org.apache.hudi.client.bootstrap.selector.BootstrapModeSelector;
|
||||
import org.apache.hudi.client.bootstrap.selector.FullRecordBootstrapModeSelector;
|
||||
import org.apache.hudi.client.bootstrap.selector.MetadataOnlyBootstrapModeSelector;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.bootstrap.FileStatusUtils;
|
||||
import org.apache.hudi.common.bootstrap.index.BootstrapIndex;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
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.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieBootstrapConfig;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.hadoop.HoodieParquetInputFormat;
|
||||
import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hadoop.io.LongWritable;
|
||||
import org.apache.hudi.index.HoodieIndex.IndexType;
|
||||
import org.apache.hudi.keygen.NonpartitionedKeyGenerator;
|
||||
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.parquet.avro.AvroParquetReader;
|
||||
import org.apache.parquet.avro.AvroReadSupport;
|
||||
import org.apache.parquet.avro.AvroSchemaConverter;
|
||||
import org.apache.parquet.hadoop.ParquetFileReader;
|
||||
import org.apache.parquet.schema.MessageType;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.SQLContext;
|
||||
import org.apache.spark.sql.Column;
|
||||
import org.apache.spark.sql.api.java.UDF1;
|
||||
import org.apache.spark.sql.types.DataTypes;
|
||||
|
||||
import org.apache.spark.sql.SaveMode;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.io.TempDir;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.URLEncoder;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.time.Instant;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
import java.util.Spliterators;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
import java.util.stream.StreamSupport;
|
||||
|
||||
import static java.util.stream.Collectors.mapping;
|
||||
import static java.util.stream.Collectors.toList;
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.generateGenericRecord;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.apache.spark.sql.functions.callUDF;
|
||||
|
||||
/**
|
||||
* Tests Bootstrap Client functionality.
|
||||
*/
|
||||
public class TestBootstrap extends HoodieClientTestBase {
|
||||
|
||||
public static final String TRIP_HIVE_COLUMN_TYPES = "bigint,string,string,string,double,double,double,double,"
|
||||
+ "struct<amount:double,currency:string>,array<struct<amount:double,currency:string>>,boolean";
|
||||
|
||||
@TempDir
|
||||
public java.nio.file.Path tmpFolder;
|
||||
|
||||
protected String bootstrapBasePath = null;
|
||||
|
||||
private HoodieParquetInputFormat roInputFormat;
|
||||
private JobConf roJobConf;
|
||||
|
||||
private HoodieParquetRealtimeInputFormat rtInputFormat;
|
||||
private JobConf rtJobConf;
|
||||
private SparkSession spark;
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
bootstrapBasePath = tmpFolder.toAbsolutePath().toString() + "/data";
|
||||
initPath();
|
||||
initSparkContexts();
|
||||
initTestDataGenerator();
|
||||
initMetaClient();
|
||||
// initialize parquet input format
|
||||
reloadInputFormats();
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void tearDown() throws IOException {
|
||||
cleanupSparkContexts();
|
||||
cleanupClients();
|
||||
cleanupTestDataGenerator();
|
||||
}
|
||||
|
||||
private void reloadInputFormats() {
|
||||
// initialize parquet input format
|
||||
roInputFormat = new HoodieParquetInputFormat();
|
||||
roJobConf = new JobConf(jsc.hadoopConfiguration());
|
||||
roInputFormat.setConf(roJobConf);
|
||||
|
||||
rtInputFormat = new HoodieParquetRealtimeInputFormat();
|
||||
rtJobConf = new JobConf(jsc.hadoopConfiguration());
|
||||
rtInputFormat.setConf(rtJobConf);
|
||||
}
|
||||
|
||||
public Schema generateNewDataSetAndReturnSchema(long timestamp, int numRecords, List<String> partitionPaths,
|
||||
String srcPath) throws Exception {
|
||||
boolean isPartitioned = partitionPaths != null && !partitionPaths.isEmpty();
|
||||
Dataset<Row> df = generateTestRawTripDataset(timestamp, 0, numRecords, partitionPaths, jsc, sqlContext);
|
||||
df.printSchema();
|
||||
if (isPartitioned) {
|
||||
df.write().partitionBy("datestr").format("parquet").mode(SaveMode.Overwrite).save(srcPath);
|
||||
} else {
|
||||
df.write().format("parquet").mode(SaveMode.Overwrite).save(srcPath);
|
||||
}
|
||||
String filePath = FileStatusUtils.toPath(BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(),
|
||||
srcPath, context).stream().findAny().map(p -> p.getValue().stream().findAny())
|
||||
.orElse(null).get().getPath()).toString();
|
||||
ParquetFileReader reader = ParquetFileReader.open(metaClient.getHadoopConf(), new Path(filePath));
|
||||
MessageType schema = reader.getFooter().getFileMetaData().getSchema();
|
||||
return new AvroSchemaConverter().convert(schema);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMetadataBootstrapUnpartitionedCOW() throws Exception {
|
||||
testBootstrapCommon(false, false, EffectiveMode.METADATA_BOOTSTRAP_MODE);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMetadataBootstrapWithUpdatesCOW() throws Exception {
|
||||
testBootstrapCommon(true, false, EffectiveMode.METADATA_BOOTSTRAP_MODE);
|
||||
}
|
||||
|
||||
private enum EffectiveMode {
|
||||
FULL_BOOTSTRAP_MODE,
|
||||
METADATA_BOOTSTRAP_MODE,
|
||||
MIXED_BOOTSTRAP_MODE
|
||||
}
|
||||
|
||||
private void testBootstrapCommon(boolean partitioned, boolean deltaCommit, EffectiveMode mode) throws Exception {
|
||||
|
||||
if (deltaCommit) {
|
||||
metaClient = HoodieTestUtils.init(basePath, HoodieTableType.MERGE_ON_READ, bootstrapBasePath);
|
||||
} else {
|
||||
metaClient = HoodieTestUtils.init(basePath, HoodieTableType.COPY_ON_WRITE, bootstrapBasePath);
|
||||
}
|
||||
|
||||
int totalRecords = 100;
|
||||
String keyGeneratorClass = partitioned ? SimpleKeyGenerator.class.getCanonicalName()
|
||||
: NonpartitionedKeyGenerator.class.getCanonicalName();
|
||||
final String bootstrapModeSelectorClass;
|
||||
final String bootstrapCommitInstantTs;
|
||||
final boolean checkNumRawFiles;
|
||||
final boolean isBootstrapIndexCreated;
|
||||
final int numInstantsAfterBootstrap;
|
||||
final List<String> bootstrapInstants;
|
||||
switch (mode) {
|
||||
case FULL_BOOTSTRAP_MODE:
|
||||
bootstrapModeSelectorClass = FullRecordBootstrapModeSelector.class.getCanonicalName();
|
||||
bootstrapCommitInstantTs = HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS;
|
||||
checkNumRawFiles = false;
|
||||
isBootstrapIndexCreated = false;
|
||||
numInstantsAfterBootstrap = 1;
|
||||
bootstrapInstants = Arrays.asList(bootstrapCommitInstantTs);
|
||||
break;
|
||||
case METADATA_BOOTSTRAP_MODE:
|
||||
bootstrapModeSelectorClass = MetadataOnlyBootstrapModeSelector.class.getCanonicalName();
|
||||
bootstrapCommitInstantTs = HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS;
|
||||
checkNumRawFiles = true;
|
||||
isBootstrapIndexCreated = true;
|
||||
numInstantsAfterBootstrap = 1;
|
||||
bootstrapInstants = Arrays.asList(bootstrapCommitInstantTs);
|
||||
break;
|
||||
default:
|
||||
bootstrapModeSelectorClass = TestRandomBootstapModeSelector.class.getName();
|
||||
bootstrapCommitInstantTs = HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS;
|
||||
checkNumRawFiles = false;
|
||||
isBootstrapIndexCreated = true;
|
||||
numInstantsAfterBootstrap = 2;
|
||||
bootstrapInstants = Arrays.asList(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS,
|
||||
HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS);
|
||||
break;
|
||||
}
|
||||
List<String> partitions = Arrays.asList("2020/04/01", "2020/04/02", "2020/04/03");
|
||||
long timestamp = Instant.now().toEpochMilli();
|
||||
Schema schema = generateNewDataSetAndReturnSchema(timestamp, totalRecords, partitions, bootstrapBasePath);
|
||||
HoodieWriteConfig config = getConfigBuilder(schema.toString())
|
||||
.withAutoCommit(true)
|
||||
.withSchema(schema.toString())
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
|
||||
.withMaxNumDeltaCommitsBeforeCompaction(1)
|
||||
.build())
|
||||
.withBootstrapConfig(HoodieBootstrapConfig.newBuilder()
|
||||
.withBootstrapBasePath(bootstrapBasePath)
|
||||
.withBootstrapKeyGenClass(keyGeneratorClass)
|
||||
.withFullBootstrapInputProvider(TestFullBootstrapDataProvider.class.getName())
|
||||
.withBootstrapParallelism(3)
|
||||
.withBootstrapModeSelector(bootstrapModeSelectorClass).build())
|
||||
.build();
|
||||
SparkRDDWriteClient client = new SparkRDDWriteClient(context, config);
|
||||
client.bootstrap(Option.empty());
|
||||
checkBootstrapResults(totalRecords, schema, bootstrapCommitInstantTs, checkNumRawFiles, numInstantsAfterBootstrap,
|
||||
numInstantsAfterBootstrap, timestamp, timestamp, deltaCommit, bootstrapInstants);
|
||||
|
||||
// Rollback Bootstrap
|
||||
FSUtils.deleteInstantFile(metaClient.getFs(), metaClient.getMetaPath(), new HoodieInstant(State.COMPLETED,
|
||||
deltaCommit ? HoodieTimeline.DELTA_COMMIT_ACTION : HoodieTimeline.COMMIT_ACTION, bootstrapCommitInstantTs));
|
||||
client.rollBackInflightBootstrap();
|
||||
metaClient.reloadActiveTimeline();
|
||||
assertEquals(0, metaClient.getCommitsTimeline().countInstants());
|
||||
assertEquals(0L, BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(), basePath, context)
|
||||
.stream().flatMap(f -> f.getValue().stream()).count());
|
||||
|
||||
BootstrapIndex index = BootstrapIndex.getBootstrapIndex(metaClient);
|
||||
assertFalse(index.useIndex());
|
||||
|
||||
// Run bootstrap again
|
||||
client = new SparkRDDWriteClient(context, config);
|
||||
client.bootstrap(Option.empty());
|
||||
|
||||
metaClient.reloadActiveTimeline();
|
||||
index = BootstrapIndex.getBootstrapIndex(metaClient);
|
||||
if (isBootstrapIndexCreated) {
|
||||
assertTrue(index.useIndex());
|
||||
} else {
|
||||
assertFalse(index.useIndex());
|
||||
}
|
||||
|
||||
checkBootstrapResults(totalRecords, schema, bootstrapCommitInstantTs, checkNumRawFiles, numInstantsAfterBootstrap,
|
||||
numInstantsAfterBootstrap, timestamp, timestamp, deltaCommit, bootstrapInstants);
|
||||
|
||||
// Upsert case
|
||||
long updateTimestamp = Instant.now().toEpochMilli();
|
||||
String updateSPath = tmpFolder.toAbsolutePath().toString() + "/data2";
|
||||
generateNewDataSetAndReturnSchema(updateTimestamp, totalRecords, partitions, updateSPath);
|
||||
JavaRDD<HoodieRecord> updateBatch =
|
||||
generateInputBatch(jsc, BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(), updateSPath, context),
|
||||
schema);
|
||||
String newInstantTs = client.startCommit();
|
||||
client.upsert(updateBatch, newInstantTs);
|
||||
checkBootstrapResults(totalRecords, schema, newInstantTs, false, numInstantsAfterBootstrap + 1,
|
||||
updateTimestamp, deltaCommit ? timestamp : updateTimestamp, deltaCommit);
|
||||
|
||||
if (deltaCommit) {
|
||||
Option<String> compactionInstant = client.scheduleCompaction(Option.empty());
|
||||
assertTrue(compactionInstant.isPresent());
|
||||
client.compact(compactionInstant.get());
|
||||
checkBootstrapResults(totalRecords, schema, compactionInstant.get(), checkNumRawFiles,
|
||||
numInstantsAfterBootstrap + 2, 2, updateTimestamp, updateTimestamp, !deltaCommit,
|
||||
Arrays.asList(compactionInstant.get()));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMetadataBootstrapWithUpdatesMOR() throws Exception {
|
||||
testBootstrapCommon(true, true, EffectiveMode.METADATA_BOOTSTRAP_MODE);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFullBootstrapOnlyCOW() throws Exception {
|
||||
testBootstrapCommon(true, false, EffectiveMode.FULL_BOOTSTRAP_MODE);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFullBootstrapWithUpdatesMOR() throws Exception {
|
||||
testBootstrapCommon(true, true, EffectiveMode.FULL_BOOTSTRAP_MODE);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMetaAndFullBootstrapCOW() throws Exception {
|
||||
testBootstrapCommon(true, false, EffectiveMode.MIXED_BOOTSTRAP_MODE);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMetadataAndFullBootstrapWithUpdatesMOR() throws Exception {
|
||||
testBootstrapCommon(true, true, EffectiveMode.MIXED_BOOTSTRAP_MODE);
|
||||
}
|
||||
|
||||
private void checkBootstrapResults(int totalRecords, Schema schema, String maxInstant, boolean checkNumRawFiles,
|
||||
int expNumInstants, long expTimestamp, long expROTimestamp, boolean isDeltaCommit) throws Exception {
|
||||
checkBootstrapResults(totalRecords, schema, maxInstant, checkNumRawFiles, expNumInstants, expNumInstants,
|
||||
expTimestamp, expROTimestamp, isDeltaCommit, Arrays.asList(maxInstant));
|
||||
}
|
||||
|
||||
private void checkBootstrapResults(int totalRecords, Schema schema, String instant, boolean checkNumRawFiles,
|
||||
int expNumInstants, int numVersions, long expTimestamp, long expROTimestamp, boolean isDeltaCommit,
|
||||
List<String> instantsWithValidRecords) throws Exception {
|
||||
metaClient.reloadActiveTimeline();
|
||||
assertEquals(expNumInstants, metaClient.getCommitsTimeline().filterCompletedInstants().countInstants());
|
||||
assertEquals(instant, metaClient.getActiveTimeline()
|
||||
.getCommitsTimeline().filterCompletedInstants().lastInstant().get().getTimestamp());
|
||||
|
||||
Dataset<Row> bootstrapped = sqlContext.read().format("parquet").load(basePath);
|
||||
Dataset<Row> original = sqlContext.read().format("parquet").load(bootstrapBasePath);
|
||||
bootstrapped.registerTempTable("bootstrapped");
|
||||
original.registerTempTable("original");
|
||||
if (checkNumRawFiles) {
|
||||
List<HoodieFileStatus> files = BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(),
|
||||
bootstrapBasePath, context).stream().flatMap(x -> x.getValue().stream()).collect(Collectors.toList());
|
||||
assertEquals(files.size() * numVersions,
|
||||
sqlContext.sql("select distinct _hoodie_file_name from bootstrapped").count());
|
||||
}
|
||||
|
||||
if (!isDeltaCommit) {
|
||||
String predicate = String.join(", ",
|
||||
instantsWithValidRecords.stream().map(p -> "\"" + p + "\"").collect(Collectors.toList()));
|
||||
assertEquals(totalRecords, sqlContext.sql("select * from bootstrapped where _hoodie_commit_time IN "
|
||||
+ "(" + predicate + ")").count());
|
||||
Dataset<Row> missingOriginal = sqlContext.sql("select a._row_key from original a where a._row_key not "
|
||||
+ "in (select _hoodie_record_key from bootstrapped)");
|
||||
assertEquals(0, missingOriginal.count());
|
||||
Dataset<Row> missingBootstrapped = sqlContext.sql("select a._hoodie_record_key from bootstrapped a "
|
||||
+ "where a._hoodie_record_key not in (select _row_key from original)");
|
||||
assertEquals(0, missingBootstrapped.count());
|
||||
//sqlContext.sql("select * from bootstrapped").show(10, false);
|
||||
}
|
||||
|
||||
// RO Input Format Read
|
||||
reloadInputFormats();
|
||||
List<GenericRecord> records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
|
||||
jsc.hadoopConfiguration(),
|
||||
FSUtils.getAllPartitionPaths(metaClient.getFs(), basePath, false).stream()
|
||||
.map(f -> basePath + "/" + f).collect(Collectors.toList()),
|
||||
basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES, false, new ArrayList<>());
|
||||
assertEquals(totalRecords, records.size());
|
||||
Set<String> seenKeys = new HashSet<>();
|
||||
for (GenericRecord r : records) {
|
||||
assertEquals(r.get("_row_key").toString(), r.get("_hoodie_record_key").toString(), "Record :" + r);
|
||||
assertEquals(expROTimestamp, ((LongWritable)r.get("timestamp")).get(), 0.1, "Record :" + r);
|
||||
assertFalse(seenKeys.contains(r.get("_hoodie_record_key").toString()));
|
||||
seenKeys.add(r.get("_hoodie_record_key").toString());
|
||||
}
|
||||
assertEquals(totalRecords, seenKeys.size());
|
||||
|
||||
//RT Input Format Read
|
||||
reloadInputFormats();
|
||||
seenKeys = new HashSet<>();
|
||||
records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
|
||||
jsc.hadoopConfiguration(),
|
||||
FSUtils.getAllPartitionPaths(metaClient.getFs(), basePath, false).stream()
|
||||
.map(f -> basePath + "/" + f).collect(Collectors.toList()),
|
||||
basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, false, new ArrayList<>());
|
||||
assertEquals(totalRecords, records.size());
|
||||
for (GenericRecord r : records) {
|
||||
assertEquals(r.get("_row_key").toString(), r.get("_hoodie_record_key").toString(), "Realtime Record :" + r);
|
||||
assertEquals(expTimestamp, ((LongWritable)r.get("timestamp")).get(),0.1, "Realtime Record :" + r);
|
||||
assertFalse(seenKeys.contains(r.get("_hoodie_record_key").toString()));
|
||||
seenKeys.add(r.get("_hoodie_record_key").toString());
|
||||
}
|
||||
assertEquals(totalRecords, seenKeys.size());
|
||||
|
||||
// RO Input Format Read - Project only Hoodie Columns
|
||||
reloadInputFormats();
|
||||
records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
|
||||
jsc.hadoopConfiguration(),
|
||||
FSUtils.getAllPartitionPaths(metaClient.getFs(), basePath, false).stream()
|
||||
.map(f -> basePath + "/" + f).collect(Collectors.toList()),
|
||||
basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES,
|
||||
true, HoodieRecord.HOODIE_META_COLUMNS);
|
||||
assertEquals(totalRecords, records.size());
|
||||
seenKeys = new HashSet<>();
|
||||
for (GenericRecord r : records) {
|
||||
assertFalse(seenKeys.contains(r.get("_hoodie_record_key").toString()));
|
||||
seenKeys.add(r.get("_hoodie_record_key").toString());
|
||||
}
|
||||
assertEquals(totalRecords, seenKeys.size());
|
||||
|
||||
//RT Input Format Read - Project only Hoodie Columns
|
||||
reloadInputFormats();
|
||||
seenKeys = new HashSet<>();
|
||||
records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
|
||||
jsc.hadoopConfiguration(),
|
||||
FSUtils.getAllPartitionPaths(metaClient.getFs(), basePath, false).stream()
|
||||
.map(f -> basePath + "/" + f).collect(Collectors.toList()),
|
||||
basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, true,
|
||||
HoodieRecord.HOODIE_META_COLUMNS);
|
||||
assertEquals(totalRecords, records.size());
|
||||
for (GenericRecord r : records) {
|
||||
assertFalse(seenKeys.contains(r.get("_hoodie_record_key").toString()));
|
||||
seenKeys.add(r.get("_hoodie_record_key").toString());
|
||||
}
|
||||
assertEquals(totalRecords, seenKeys.size());
|
||||
|
||||
// RO Input Format Read - Project only non-hoodie column
|
||||
reloadInputFormats();
|
||||
records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
|
||||
jsc.hadoopConfiguration(),
|
||||
FSUtils.getAllPartitionPaths(metaClient.getFs(), basePath, false).stream()
|
||||
.map(f -> basePath + "/" + f).collect(Collectors.toList()),
|
||||
basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES, true,
|
||||
Arrays.asList("_row_key"));
|
||||
assertEquals(totalRecords, records.size());
|
||||
seenKeys = new HashSet<>();
|
||||
for (GenericRecord r : records) {
|
||||
assertFalse(seenKeys.contains(r.get("_row_key").toString()));
|
||||
seenKeys.add(r.get("_row_key").toString());
|
||||
}
|
||||
assertEquals(totalRecords, seenKeys.size());
|
||||
|
||||
//RT Input Format Read - Project only non-hoodie column
|
||||
reloadInputFormats();
|
||||
seenKeys = new HashSet<>();
|
||||
records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
|
||||
jsc.hadoopConfiguration(),
|
||||
FSUtils.getAllPartitionPaths(metaClient.getFs(), basePath, false).stream()
|
||||
.map(f -> basePath + "/" + f).collect(Collectors.toList()),
|
||||
basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, true,
|
||||
Arrays.asList("_row_key"));
|
||||
assertEquals(totalRecords, records.size());
|
||||
for (GenericRecord r : records) {
|
||||
assertFalse(seenKeys.contains(r.get("_row_key").toString()));
|
||||
seenKeys.add(r.get("_row_key").toString());
|
||||
}
|
||||
assertEquals(totalRecords, seenKeys.size());
|
||||
}
|
||||
|
||||
public static class TestFullBootstrapDataProvider extends FullRecordBootstrapDataProvider<JavaRDD<HoodieRecord>> {
|
||||
|
||||
public TestFullBootstrapDataProvider(TypedProperties props, HoodieSparkEngineContext context) {
|
||||
super(props, context);
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord> generateInputRecords(String tableName, String sourceBasePath,
|
||||
List<Pair<String, List<HoodieFileStatus>>> partitionPaths) {
|
||||
String filePath = FileStatusUtils.toPath(partitionPaths.stream().flatMap(p -> p.getValue().stream())
|
||||
.findAny().get().getPath()).toString();
|
||||
ParquetFileReader reader = null;
|
||||
JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
|
||||
try {
|
||||
reader = ParquetFileReader.open(jsc.hadoopConfiguration(), new Path(filePath));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
MessageType parquetSchema = reader.getFooter().getFileMetaData().getSchema();
|
||||
Schema schema = new AvroSchemaConverter().convert(parquetSchema);
|
||||
return generateInputBatch(jsc, partitionPaths, schema);
|
||||
}
|
||||
}
|
||||
|
||||
private static JavaRDD<HoodieRecord> generateInputBatch(JavaSparkContext jsc,
|
||||
List<Pair<String, List<HoodieFileStatus>>> partitionPaths, Schema writerSchema) {
|
||||
List<Pair<String, Path>> fullFilePathsWithPartition = partitionPaths.stream().flatMap(p -> p.getValue().stream()
|
||||
.map(x -> Pair.of(p.getKey(), FileStatusUtils.toPath(x.getPath())))).collect(Collectors.toList());
|
||||
return jsc.parallelize(fullFilePathsWithPartition.stream().flatMap(p -> {
|
||||
try {
|
||||
Configuration conf = jsc.hadoopConfiguration();
|
||||
AvroReadSupport.setAvroReadSchema(conf, writerSchema);
|
||||
Iterator<GenericRecord> recIterator = new ParquetReaderIterator(
|
||||
AvroParquetReader.<GenericRecord>builder(p.getValue()).withConf(conf).build());
|
||||
return StreamSupport.stream(Spliterators.spliteratorUnknownSize(recIterator, 0), false).map(gr -> {
|
||||
try {
|
||||
String key = gr.get("_row_key").toString();
|
||||
String pPath = p.getKey();
|
||||
return new HoodieRecord<>(new HoodieKey(key, pPath), new RawTripTestPayload(gr.toString(), key, pPath,
|
||||
HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
});
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException(ioe.getMessage(), ioe);
|
||||
}
|
||||
}).collect(Collectors.toList()));
|
||||
}
|
||||
|
||||
public static class TestRandomBootstapModeSelector extends BootstrapModeSelector {
|
||||
|
||||
private int currIdx = new Random().nextInt(2);
|
||||
|
||||
public TestRandomBootstapModeSelector(HoodieWriteConfig writeConfig) {
|
||||
super(writeConfig);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<BootstrapMode, List<String>> select(List<Pair<String, List<HoodieFileStatus>>> partitions) {
|
||||
List<Pair<BootstrapMode, String>> selections = new ArrayList<>();
|
||||
partitions.stream().forEach(p -> {
|
||||
final BootstrapMode mode;
|
||||
if (currIdx == 0) {
|
||||
mode = BootstrapMode.METADATA_ONLY;
|
||||
} else {
|
||||
mode = BootstrapMode.FULL_RECORD;
|
||||
}
|
||||
currIdx = (currIdx + 1) % 2;
|
||||
selections.add(Pair.of(mode, p.getKey()));
|
||||
});
|
||||
return selections.stream().collect(Collectors.groupingBy(Pair::getKey, mapping(Pair::getValue, toList())));
|
||||
}
|
||||
}
|
||||
|
||||
public HoodieWriteConfig.Builder getConfigBuilder(String schemaStr) {
|
||||
HoodieWriteConfig.Builder builder = getConfigBuilder(schemaStr, IndexType.BLOOM)
|
||||
.withExternalSchemaTrasformation(true);
|
||||
TypedProperties properties = new TypedProperties();
|
||||
properties.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key");
|
||||
properties.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "datestr");
|
||||
builder = builder.withProps(properties);
|
||||
return builder;
|
||||
}
|
||||
|
||||
public static Dataset<Row> generateTestRawTripDataset(long timestamp, int from, int to, List<String> partitionPaths,
|
||||
JavaSparkContext jsc, SQLContext sqlContext) {
|
||||
boolean isPartitioned = partitionPaths != null && !partitionPaths.isEmpty();
|
||||
final List<String> records = new ArrayList<>();
|
||||
IntStream.range(from, to).forEach(i -> {
|
||||
String id = "" + i;
|
||||
records.add(generateGenericRecord("trip_" + id, "rider_" + id, "driver_" + id,
|
||||
timestamp, false, false).toString());
|
||||
});
|
||||
if (isPartitioned) {
|
||||
sqlContext.udf().register("partgen",
|
||||
(UDF1<String, String>) (val) -> URLEncoder.encode(partitionPaths.get(
|
||||
Integer.parseInt(val.split("_")[1]) % partitionPaths.size()), StandardCharsets.UTF_8.toString()),
|
||||
DataTypes.StringType);
|
||||
}
|
||||
JavaRDD rdd = jsc.parallelize(records);
|
||||
Dataset<Row> df = sqlContext.read().json(rdd);
|
||||
if (isPartitioned) {
|
||||
df = df.withColumn("datestr", callUDF("partgen", new Column("_row_key")));
|
||||
// Order the columns to ensure generated avro schema aligns with Hive schema
|
||||
df = df.select("timestamp", "_row_key", "rider", "driver", "begin_lat", "begin_lon",
|
||||
"end_lat", "end_lon", "fare", "tip_history", "_hoodie_is_deleted", "datestr");
|
||||
} else {
|
||||
// Order the columns to ensure generated avro schema aligns with Hive schema
|
||||
df = df.select("timestamp", "_row_key", "rider", "driver", "begin_lat", "begin_lon",
|
||||
"end_lat", "end_lon", "fare", "tip_history", "_hoodie_is_deleted");
|
||||
}
|
||||
return df;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,132 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.payload;
|
||||
|
||||
import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericData;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.junit.jupiter.api.Assertions.fail;
|
||||
|
||||
public class TestAWSDmsAvroPayload {
|
||||
|
||||
private static final String AVRO_SCHEMA_STRING = "{\"type\": \"record\","
|
||||
+ "\"name\": \"events\"," + "\"fields\": [ "
|
||||
+ "{\"name\": \"field1\", \"type\" : \"int\"},"
|
||||
+ "{\"name\": \"Op\", \"type\": \"string\"}"
|
||||
+ "]}";
|
||||
|
||||
@Test
|
||||
public void testInsert() {
|
||||
|
||||
Schema avroSchema = new Schema.Parser().parse(AVRO_SCHEMA_STRING);
|
||||
GenericRecord record = new GenericData.Record(avroSchema);
|
||||
record.put("field1", 0);
|
||||
record.put("Op", "I");
|
||||
|
||||
AWSDmsAvroPayload payload = new AWSDmsAvroPayload(Option.of(record));
|
||||
|
||||
try {
|
||||
Option<IndexedRecord> outputPayload = payload.getInsertValue(avroSchema);
|
||||
assertTrue((int) outputPayload.get().get(0) == 0);
|
||||
assertTrue(outputPayload.get().get(1).toString().equals("I"));
|
||||
} catch (Exception e) {
|
||||
fail("Unexpected exception");
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpdate() {
|
||||
Schema avroSchema = new Schema.Parser().parse(AVRO_SCHEMA_STRING);
|
||||
GenericRecord newRecord = new GenericData.Record(avroSchema);
|
||||
newRecord.put("field1", 1);
|
||||
newRecord.put("Op", "U");
|
||||
|
||||
GenericRecord oldRecord = new GenericData.Record(avroSchema);
|
||||
oldRecord.put("field1", 0);
|
||||
oldRecord.put("Op", "I");
|
||||
|
||||
AWSDmsAvroPayload payload = new AWSDmsAvroPayload(Option.of(newRecord));
|
||||
|
||||
try {
|
||||
Option<IndexedRecord> outputPayload = payload.combineAndGetUpdateValue(oldRecord, avroSchema);
|
||||
assertTrue((int) outputPayload.get().get(0) == 1);
|
||||
assertTrue(outputPayload.get().get(1).toString().equals("U"));
|
||||
} catch (Exception e) {
|
||||
fail("Unexpected exception");
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDelete() {
|
||||
Schema avroSchema = new Schema.Parser().parse(AVRO_SCHEMA_STRING);
|
||||
GenericRecord deleteRecord = new GenericData.Record(avroSchema);
|
||||
deleteRecord.put("field1", 2);
|
||||
deleteRecord.put("Op", "D");
|
||||
|
||||
GenericRecord oldRecord = new GenericData.Record(avroSchema);
|
||||
oldRecord.put("field1", 2);
|
||||
oldRecord.put("Op", "U");
|
||||
|
||||
AWSDmsAvroPayload payload = new AWSDmsAvroPayload(Option.of(deleteRecord));
|
||||
|
||||
try {
|
||||
Option<IndexedRecord> outputPayload = payload.combineAndGetUpdateValue(oldRecord, avroSchema);
|
||||
// expect nothing to be comitted to table
|
||||
assertFalse(outputPayload.isPresent());
|
||||
} catch (Exception e) {
|
||||
fail("Unexpected exception");
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPreCombineWithDelete() {
|
||||
Schema avroSchema = new Schema.Parser().parse(AVRO_SCHEMA_STRING);
|
||||
GenericRecord deleteRecord = new GenericData.Record(avroSchema);
|
||||
deleteRecord.put("field1", 4);
|
||||
deleteRecord.put("Op", "D");
|
||||
|
||||
GenericRecord oldRecord = new GenericData.Record(avroSchema);
|
||||
oldRecord.put("field1", 4);
|
||||
oldRecord.put("Op", "I");
|
||||
|
||||
AWSDmsAvroPayload payload = new AWSDmsAvroPayload(Option.of(deleteRecord));
|
||||
AWSDmsAvroPayload insertPayload = new AWSDmsAvroPayload(Option.of(oldRecord));
|
||||
|
||||
try {
|
||||
OverwriteWithLatestAvroPayload output = payload.preCombine(insertPayload);
|
||||
Option<IndexedRecord> outputPayload = output.getInsertValue(avroSchema);
|
||||
// expect nothing to be comitted to table
|
||||
assertFalse(outputPayload.isPresent());
|
||||
} catch (Exception e) {
|
||||
fail("Unexpected exception");
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,61 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.testutils;
|
||||
|
||||
import org.apache.hudi.common.util.FileIOUtils;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.RowFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Date;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
import java.util.UUID;
|
||||
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH;
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH;
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH;
|
||||
|
||||
/**
|
||||
* Test utils for data source tests.
|
||||
*/
|
||||
public class DataSourceTestUtils {
|
||||
|
||||
public static Schema getStructTypeExampleSchema() throws IOException {
|
||||
return new Schema.Parser().parse(FileIOUtils.readAsUTFString(DataSourceTestUtils.class.getResourceAsStream("/exampleSchema.txt")));
|
||||
}
|
||||
|
||||
public static List<Row> generateRandomRows(int count) {
|
||||
Random random = new Random();
|
||||
List<Row> toReturn = new ArrayList<>();
|
||||
List<String> partitions = Arrays.asList(new String[] {DEFAULT_FIRST_PARTITION_PATH, DEFAULT_SECOND_PARTITION_PATH, DEFAULT_THIRD_PARTITION_PATH});
|
||||
for (int i = 0; i < count; i++) {
|
||||
Object[] values = new Object[3];
|
||||
values[0] = UUID.randomUUID().toString();
|
||||
values[1] = partitions.get(random.nextInt(3));
|
||||
values[2] = new Date().getTime();
|
||||
toReturn.add(RowFactory.create(values));
|
||||
}
|
||||
return toReturn;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,36 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
{
|
||||
"namespace": "example.schema",
|
||||
"type": "record",
|
||||
"name": "trip",
|
||||
"fields": [
|
||||
{
|
||||
"name": "_row_key",
|
||||
"type": "string"
|
||||
},
|
||||
{
|
||||
"name": "partition",
|
||||
"type": "string"
|
||||
},
|
||||
{
|
||||
"name": "ts",
|
||||
"type": ["long", "null"]
|
||||
}
|
||||
]
|
||||
}
|
||||
@@ -0,0 +1,30 @@
|
||||
###
|
||||
# Licensed to the Apache Software Foundation (ASF) under one
|
||||
# or more contributor license agreements. See the NOTICE file
|
||||
# distributed with this work for additional information
|
||||
# regarding copyright ownership. The ASF licenses this file
|
||||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing, software
|
||||
# distributed under the License is distributed on an "AS IS" BASIS,
|
||||
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
###
|
||||
log4j.rootLogger=WARN, CONSOLE
|
||||
log4j.logger.org.apache.hudi=DEBUG
|
||||
log4j.logger.org.apache.hadoop.hbase=ERROR
|
||||
|
||||
# CONSOLE is set to be a ConsoleAppender.
|
||||
log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
|
||||
# CONSOLE uses PatternLayout.
|
||||
log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.CONSOLE.layout.ConversionPattern=[%-5p] %d %c %x - %m%n
|
||||
log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter
|
||||
log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true
|
||||
log4j.appender.CONSOLE.filter.a.LevelMin=WARN
|
||||
log4j.appender.CONSOLE.filter.a.LevelMax=FATAL
|
||||
@@ -0,0 +1,31 @@
|
||||
###
|
||||
# Licensed to the Apache Software Foundation (ASF) under one
|
||||
# or more contributor license agreements. See the NOTICE file
|
||||
# distributed with this work for additional information
|
||||
# regarding copyright ownership. The ASF licenses this file
|
||||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing, software
|
||||
# distributed under the License is distributed on an "AS IS" BASIS,
|
||||
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
###
|
||||
log4j.rootLogger=WARN, CONSOLE
|
||||
log4j.logger.org.apache=INFO
|
||||
log4j.logger.org.apache.hudi=DEBUG
|
||||
log4j.logger.org.apache.hadoop.hbase=ERROR
|
||||
|
||||
# A1 is set to be a ConsoleAppender.
|
||||
log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
|
||||
# A1 uses PatternLayout.
|
||||
log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.CONSOLE.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
|
||||
log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter
|
||||
log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true
|
||||
log4j.appender.CONSOLE.filter.a.LevelMin=WARN
|
||||
log4j.appender.CONSOLE.filter.a.LevelMax=FATAL
|
||||
@@ -0,0 +1,58 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi
|
||||
|
||||
import java.time.LocalDate
|
||||
|
||||
import org.apache.avro.Schema
|
||||
import org.apache.avro.generic.GenericData
|
||||
import org.apache.spark.sql.catalyst.expressions.GenericRow
|
||||
import org.scalatest.{FunSuite, Matchers}
|
||||
|
||||
class TestAvroConversionHelper extends FunSuite with Matchers {
|
||||
|
||||
val dateSchema = s"""
|
||||
{
|
||||
"namespace": "logical",
|
||||
"type": "record",
|
||||
"name": "test",
|
||||
"fields": [
|
||||
{"name": "date", "type": {"type": "int", "logicalType": "date"}}
|
||||
]
|
||||
}
|
||||
"""
|
||||
|
||||
val dateInputData = Seq(7, 365, 0)
|
||||
|
||||
test("Logical type: date") {
|
||||
val schema = new Schema.Parser().parse(dateSchema)
|
||||
val convertor = AvroConversionHelper.createConverterToRow(schema, AvroConversionUtils.convertAvroSchemaToStructType(schema))
|
||||
|
||||
val dateOutputData = dateInputData.map(x => {
|
||||
val record = new GenericData.Record(schema) {{ put("date", x) }}
|
||||
convertor(record).asInstanceOf[GenericRow].get(0)
|
||||
})
|
||||
|
||||
println(s"trans data from int [ ${dateInputData.mkString(", ")} ] to date [ ${dateOutputData.mkString(", ")} ]")
|
||||
|
||||
assert(dateOutputData(0).toString === LocalDate.ofEpochDay(dateInputData(0)).toString)
|
||||
assert(dateOutputData(1).toString === LocalDate.ofEpochDay(dateInputData(1)).toString)
|
||||
assert(dateOutputData(2).toString === LocalDate.ofEpochDay(dateInputData(2)).toString)
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,581 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi
|
||||
|
||||
import org.apache.avro.generic.GenericRecord
|
||||
import org.apache.hudi.avro.HoodieAvroUtils
|
||||
import org.apache.hudi.common.config.TypedProperties
|
||||
import org.apache.hudi.common.model.{EmptyHoodieRecordPayload, HoodieKey, OverwriteWithLatestAvroPayload}
|
||||
import org.apache.hudi.common.testutils.SchemaTestUtil
|
||||
import org.apache.hudi.common.util.Option
|
||||
import org.apache.hudi.exception.{HoodieException, HoodieKeyException}
|
||||
import org.apache.hudi.keygen._
|
||||
import org.apache.hudi.testutils.KeyGeneratorTestUtilities
|
||||
import org.apache.spark.sql.Row
|
||||
import org.junit.jupiter.api.Assertions.assertEquals
|
||||
import org.junit.jupiter.api.{BeforeEach, Test}
|
||||
import org.scalatest.Assertions.fail
|
||||
|
||||
/**
|
||||
* Tests on the default key generator, payload classes.
|
||||
*/
|
||||
class TestDataSourceDefaults {
|
||||
|
||||
val schema = SchemaTestUtil.getComplexEvolvedSchema
|
||||
val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema)
|
||||
var baseRecord: GenericRecord = _
|
||||
var baseRow: Row = _
|
||||
val testStructName = "testStructName"
|
||||
val testNamespace = "testNamespace"
|
||||
|
||||
@BeforeEach def initialize(): Unit = {
|
||||
baseRecord = SchemaTestUtil
|
||||
.generateAvroRecordFromJson(schema, 1, "001", "f1")
|
||||
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
|
||||
}
|
||||
|
||||
private def getKeyConfig(recordKeyFieldName: String, partitionPathField: String, hiveStylePartitioning: String): TypedProperties = {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, recordKeyFieldName)
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, partitionPathField)
|
||||
props.setProperty(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING_OPT_KEY, hiveStylePartitioning)
|
||||
props
|
||||
}
|
||||
|
||||
@Test def testSimpleKeyGenerator() = {
|
||||
|
||||
// top level, valid fields
|
||||
var keyGen = new SimpleKeyGenerator(getKeyConfig("field1", "name", "false"))
|
||||
val hk1 = keyGen.getKey(baseRecord)
|
||||
assertEquals("field1", hk1.getRecordKey)
|
||||
assertEquals("name1", hk1.getPartitionPath)
|
||||
|
||||
assertEquals("field1", keyGen.getRecordKey(baseRow))
|
||||
assertEquals("name1", keyGen.getPartitionPath(baseRow))
|
||||
|
||||
// partition path field not specified
|
||||
try {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1")
|
||||
new SimpleKeyGenerator(props).getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: IllegalArgumentException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// partition path field not specified using Row
|
||||
try {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1")
|
||||
val keyGen = new SimpleKeyGenerator(props)
|
||||
keyGen.getRecordKey(baseRow)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: IllegalArgumentException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// recordkey field not specified
|
||||
try {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "partitionField")
|
||||
new SimpleKeyGenerator(props).getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: IllegalArgumentException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// recordkey field not specified using Row
|
||||
try {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "partitionField")
|
||||
val keyGen = new SimpleKeyGenerator(props)
|
||||
keyGen.getPartitionPath(baseRow)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: IllegalArgumentException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// nested field as record key and partition path
|
||||
val hk2 = new SimpleKeyGenerator(getKeyConfig("testNestedRecord.userId", "testNestedRecord.isAdmin", "false"))
|
||||
.getKey(baseRecord)
|
||||
assertEquals("UserId1@001", hk2.getRecordKey)
|
||||
assertEquals("false", hk2.getPartitionPath)
|
||||
|
||||
// Nested record key not found
|
||||
try {
|
||||
new SimpleKeyGenerator(getKeyConfig("testNestedRecord.NotThere", "testNestedRecord.isAdmin", "false"))
|
||||
.getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: HoodieException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// if partition path can't be found, return default partition path
|
||||
val hk3 = new SimpleKeyGenerator(getKeyConfig("testNestedRecord.userId", "testNestedRecord.notThere", "false"))
|
||||
.getKey(baseRecord)
|
||||
assertEquals("default", hk3.getPartitionPath)
|
||||
|
||||
// if partition path can't be found, return default partition path using row
|
||||
keyGen = new SimpleKeyGenerator(getKeyConfig("testNestedRecord.userId", "testNestedRecord.notThere", "false"))
|
||||
val hk3_row = keyGen.getPartitionPath(baseRow)
|
||||
assertEquals("default", hk3_row)
|
||||
|
||||
// if enable hive style partitioning
|
||||
val hk4 = new SimpleKeyGenerator(getKeyConfig("field1", "name", "true")).getKey(baseRecord)
|
||||
assertEquals("name=name1", hk4.getPartitionPath)
|
||||
|
||||
// if enable hive style partitioning using row
|
||||
keyGen = new SimpleKeyGenerator(getKeyConfig("field1", "name", "true"))
|
||||
val hk4_row = keyGen.getPartitionPath(baseRow)
|
||||
assertEquals("name=name1", hk4_row)
|
||||
|
||||
// if partition is null, return default partition path
|
||||
baseRecord.put("name", "")
|
||||
val hk5 = new SimpleKeyGenerator(getKeyConfig("field1", "name", "false"))
|
||||
.getKey(baseRecord)
|
||||
assertEquals("default", hk5.getPartitionPath)
|
||||
|
||||
// if partition is null, return default partition path using Row
|
||||
keyGen = new SimpleKeyGenerator(getKeyConfig("field1", "name", "false"))
|
||||
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
|
||||
val hk5_row = keyGen.getPartitionPath(baseRow)
|
||||
assertEquals("default", hk5_row)
|
||||
|
||||
// if partition is empty, return default partition path
|
||||
baseRecord.put("name", null)
|
||||
val hk6 = new SimpleKeyGenerator(getKeyConfig("field1", "name", "false"))
|
||||
.getKey(baseRecord)
|
||||
assertEquals("default", hk6.getPartitionPath)
|
||||
|
||||
// if partition is empty, return default partition path using Row
|
||||
keyGen = new SimpleKeyGenerator(getKeyConfig("field1", "name", "false"))
|
||||
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
|
||||
val hk6_row = keyGen.getPartitionPath(baseRow)
|
||||
assertEquals("default", hk6_row)
|
||||
|
||||
// if record key is empty, throw error
|
||||
try {
|
||||
baseRecord.put("field1", "")
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1")
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "name")
|
||||
new SimpleKeyGenerator(props).getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: HoodieKeyException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// if record key is empty, throw error. Using Row
|
||||
try {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1")
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "name")
|
||||
keyGen = new SimpleKeyGenerator(props)
|
||||
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
|
||||
keyGen.getRecordKey(baseRow)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: HoodieKeyException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// if record key is null, throw error
|
||||
try {
|
||||
baseRecord.put("field1", null)
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1")
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "name")
|
||||
new SimpleKeyGenerator(props).getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: HoodieKeyException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// if record key is null, throw error. Using Row
|
||||
try {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1")
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "name")
|
||||
keyGen = new SimpleKeyGenerator(props)
|
||||
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
|
||||
keyGen.getRecordKey(baseRow)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: HoodieKeyException =>
|
||||
// do nothing
|
||||
}
|
||||
}
|
||||
|
||||
@Test def testUserDefinedKeyGeneratorWorksWithRows(): Unit = {
|
||||
val keyGen = new UserDefinedKeyGenerator(getKeyConfig("field1", "name", "false"))
|
||||
assertEquals("field1", keyGen.getRecordKey(baseRow))
|
||||
assertEquals("name1", keyGen.getPartitionPath(baseRow))
|
||||
}
|
||||
|
||||
class UserDefinedKeyGenerator(props: TypedProperties) extends KeyGenerator(props) with SparkKeyGeneratorInterface {
|
||||
val recordKeyProp: String = props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY)
|
||||
val partitionPathProp: String = props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY)
|
||||
val STRUCT_NAME: String = "hoodieRowTopLevelField"
|
||||
val NAMESPACE: String = "hoodieRow"
|
||||
var converterFn: Function1[Any, Any] = _
|
||||
|
||||
override def getKey(record: GenericRecord): HoodieKey = {
|
||||
new HoodieKey(HoodieAvroUtils.getNestedFieldValAsString(record, recordKeyProp, true),
|
||||
HoodieAvroUtils.getNestedFieldValAsString(record, partitionPathProp, true))
|
||||
}
|
||||
|
||||
override def getRecordKey(row: Row): String = {
|
||||
if (null == converterFn) converterFn = AvroConversionHelper.createConverterToAvro(row.schema, STRUCT_NAME, NAMESPACE)
|
||||
val genericRecord = converterFn.apply(row).asInstanceOf[GenericRecord]
|
||||
getKey(genericRecord).getRecordKey
|
||||
}
|
||||
|
||||
override def getPartitionPath(row: Row): String = {
|
||||
if (null == converterFn) converterFn = AvroConversionHelper.createConverterToAvro(row.schema, STRUCT_NAME, NAMESPACE)
|
||||
val genericRecord = converterFn.apply(row).asInstanceOf[GenericRecord]
|
||||
getKey(genericRecord).getPartitionPath
|
||||
}
|
||||
}
|
||||
|
||||
@Test def testComplexKeyGenerator() = {
|
||||
// top level, valid fields
|
||||
var keyGen = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name", "false"))
|
||||
val hk1 = keyGen.getKey(baseRecord)
|
||||
assertEquals("field1:field1,name:name1", hk1.getRecordKey)
|
||||
assertEquals("field1/name1", hk1.getPartitionPath)
|
||||
|
||||
// top level, valid fields with Row
|
||||
assertEquals("field1:field1,name:name1", keyGen.getRecordKey(baseRow))
|
||||
assertEquals("field1/name1", keyGen.getPartitionPath(baseRow))
|
||||
|
||||
// partition path field not specified
|
||||
try {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1")
|
||||
new ComplexKeyGenerator(props).getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: IllegalArgumentException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// partition path field not specified using Row
|
||||
try {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1")
|
||||
val keyGen = new ComplexKeyGenerator(props)
|
||||
keyGen.getRecordKey(baseRow)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: IllegalArgumentException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// recordkey field not specified
|
||||
try {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "partitionField")
|
||||
new ComplexKeyGenerator(props).getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: IllegalArgumentException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// recordkey field not specified
|
||||
try {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "partitionField")
|
||||
val keyGen = new ComplexKeyGenerator(props)
|
||||
keyGen.getPartitionPath(baseRow)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: IllegalArgumentException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// nested field as record key and partition path
|
||||
keyGen = new ComplexKeyGenerator(getKeyConfig("testNestedRecord.userId,testNestedRecord.isAdmin", "testNestedRecord.userId,testNestedRecord.isAdmin", "false"))
|
||||
val hk2 = keyGen.getKey(baseRecord)
|
||||
assertEquals("testNestedRecord.userId:UserId1@001,testNestedRecord.isAdmin:false", hk2.getRecordKey)
|
||||
assertEquals("UserId1@001/false", hk2.getPartitionPath)
|
||||
|
||||
// nested field as record key and partition path
|
||||
assertEquals("testNestedRecord.userId:UserId1@001,testNestedRecord.isAdmin:false", keyGen.getRecordKey(baseRow))
|
||||
assertEquals("UserId1@001/false", keyGen.getPartitionPath(baseRow))
|
||||
|
||||
// Nested record key not found
|
||||
try {
|
||||
new ComplexKeyGenerator(getKeyConfig("testNestedRecord.NotThere", "testNestedRecord.isAdmin", "false"))
|
||||
.getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: HoodieException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// Nested record key not found
|
||||
try {
|
||||
val keyGen = new ComplexKeyGenerator(getKeyConfig("testNestedRecord.NotThere", "testNestedRecord.isAdmin", "false"))
|
||||
keyGen.getRecordKey(baseRow)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: HoodieException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// if partition path can't be found, return default partition path
|
||||
keyGen = new ComplexKeyGenerator(getKeyConfig("testNestedRecord.userId", "testNestedRecord.notThere", "false"))
|
||||
val hk3 = keyGen.getKey(baseRecord)
|
||||
assertEquals("default", hk3.getPartitionPath)
|
||||
|
||||
assertEquals("default", keyGen.getPartitionPath(baseRow))
|
||||
|
||||
// if enable hive style partitioning
|
||||
keyGen = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name", "true"))
|
||||
val hk4 = keyGen.getKey(baseRecord)
|
||||
assertEquals("field1:field1,name:name1", hk4.getRecordKey)
|
||||
assertEquals("field1=field1/name=name1", hk4.getPartitionPath)
|
||||
|
||||
assertEquals("field1:field1,name:name1", keyGen.getRecordKey(baseRow))
|
||||
assertEquals("field1=field1/name=name1", keyGen.getPartitionPath(baseRow))
|
||||
|
||||
// if one part of the record key is empty, replace with "__empty__"
|
||||
baseRecord.put("name", "")
|
||||
keyGen = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name", "false"))
|
||||
val hk5 = keyGen.getKey(baseRecord)
|
||||
assertEquals("field1:field1,name:__empty__", hk5.getRecordKey)
|
||||
assertEquals("field1/default", hk5.getPartitionPath)
|
||||
|
||||
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
|
||||
assertEquals("field1:field1,name:__empty__", keyGen.getRecordKey(baseRow))
|
||||
assertEquals("field1/default", keyGen.getPartitionPath(baseRow))
|
||||
|
||||
// if one part of the record key is null, replace with "__null__"
|
||||
baseRecord.put("name", null)
|
||||
keyGen = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name", "false"))
|
||||
val hk6 = keyGen.getKey(baseRecord)
|
||||
assertEquals("field1:field1,name:__null__", hk6.getRecordKey)
|
||||
assertEquals("field1/default", hk6.getPartitionPath)
|
||||
|
||||
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
|
||||
assertEquals("field1:field1,name:__null__", keyGen.getRecordKey(baseRow))
|
||||
assertEquals("field1/default", keyGen.getPartitionPath(baseRow))
|
||||
|
||||
// if all parts of the composite record key are null/empty, throw error
|
||||
try {
|
||||
baseRecord.put("name", "")
|
||||
baseRecord.put("field1", null)
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1,name")
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "field1,name")
|
||||
new ComplexKeyGenerator(props).getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: HoodieKeyException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// if all parts of the composite record key are null/empty, throw error
|
||||
try {
|
||||
baseRecord.put("name", "")
|
||||
baseRecord.put("field1", null)
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1,name")
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "field1,name")
|
||||
keyGen = new ComplexKeyGenerator(props)
|
||||
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
|
||||
keyGen.getRecordKey(baseRow)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: HoodieKeyException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// reset name and field1 values.
|
||||
baseRecord.put("name", "name1")
|
||||
baseRecord.put("field1", "field1")
|
||||
keyGen = new ComplexKeyGenerator(getKeyConfig("field1, name", "field1, name", "false"))
|
||||
val hk7 = keyGen.getKey(baseRecord)
|
||||
assertEquals("field1:field1,name:name1", hk7.getRecordKey)
|
||||
assertEquals("field1/name1", hk7.getPartitionPath)
|
||||
|
||||
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
|
||||
assertEquals("field1:field1,name:name1", keyGen.getRecordKey(baseRow))
|
||||
assertEquals("field1/name1", keyGen.getPartitionPath(baseRow))
|
||||
|
||||
keyGen = new ComplexKeyGenerator(getKeyConfig("field1,", "field1,", "false"))
|
||||
val hk8 = keyGen.getKey(baseRecord)
|
||||
assertEquals("field1:field1", hk8.getRecordKey)
|
||||
assertEquals("field1", hk8.getPartitionPath)
|
||||
|
||||
assertEquals("field1:field1", keyGen.getRecordKey(baseRow))
|
||||
assertEquals("field1", keyGen.getPartitionPath(baseRow))
|
||||
}
|
||||
|
||||
@Test def testGlobalDeleteKeyGenerator() = {
|
||||
// top level, partition value included but not actually used
|
||||
var keyGen = new GlobalDeleteKeyGenerator(getKeyConfig("field1,name", "field1,name", "false"))
|
||||
val hk1 = keyGen.getKey(baseRecord)
|
||||
assertEquals("field1:field1,name:name1", hk1.getRecordKey)
|
||||
assertEquals("", hk1.getPartitionPath)
|
||||
|
||||
assertEquals("field1:field1,name:name1", keyGen.getRecordKey(baseRow))
|
||||
assertEquals("", keyGen.getPartitionPath(baseRow))
|
||||
|
||||
// top level, partition value not included
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1,name")
|
||||
keyGen = new GlobalDeleteKeyGenerator(props)
|
||||
val hk2 = keyGen.getKey(baseRecord)
|
||||
assertEquals("field1:field1,name:name1", hk2.getRecordKey)
|
||||
assertEquals("", hk2.getPartitionPath)
|
||||
|
||||
assertEquals("field1:field1,name:name1", keyGen.getRecordKey(baseRow))
|
||||
assertEquals("", keyGen.getPartitionPath(baseRow))
|
||||
|
||||
// if one part of the record key is empty, replace with "__empty__"
|
||||
baseRecord.put("name", "")
|
||||
keyGen = new GlobalDeleteKeyGenerator(getKeyConfig("field1,name", "field1,name", "false"))
|
||||
val hk3 = keyGen.getKey(baseRecord)
|
||||
assertEquals("field1:field1,name:__empty__", hk3.getRecordKey)
|
||||
assertEquals("", hk3.getPartitionPath)
|
||||
|
||||
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
|
||||
assertEquals("field1:field1,name:__empty__", keyGen.getRecordKey(baseRow))
|
||||
assertEquals("", keyGen.getPartitionPath(baseRow))
|
||||
|
||||
// if one part of the record key is null, replace with "__null__"
|
||||
baseRecord.put("name", null)
|
||||
keyGen = new GlobalDeleteKeyGenerator(getKeyConfig("field1,name", "field1,name", "false"))
|
||||
val hk4 = keyGen.getKey(baseRecord)
|
||||
assertEquals("field1:field1,name:__null__", hk4.getRecordKey)
|
||||
assertEquals("", hk4.getPartitionPath)
|
||||
|
||||
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
|
||||
assertEquals("field1:field1,name:__null__", keyGen.getRecordKey(baseRow))
|
||||
assertEquals("", keyGen.getPartitionPath(baseRow))
|
||||
|
||||
// recordkey field not specified
|
||||
try {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "partitionField")
|
||||
new GlobalDeleteKeyGenerator(props).getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: IllegalArgumentException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// recordkey field not specified
|
||||
try {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "partitionField")
|
||||
val keyGen = new GlobalDeleteKeyGenerator(props)
|
||||
keyGen.getRecordKey(baseRow)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: IllegalArgumentException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// Nested record key not found
|
||||
try {
|
||||
new GlobalDeleteKeyGenerator(getKeyConfig("testNestedRecord.NotThere", "testNestedRecord.isAdmin", "false"))
|
||||
.getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: HoodieException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// Nested record key not found
|
||||
try {
|
||||
val keyGen = new GlobalDeleteKeyGenerator(getKeyConfig("testNestedRecord.NotThere", "testNestedRecord.isAdmin", "false"))
|
||||
keyGen.getRecordKey(baseRow)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: HoodieException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// if all parts of the composite record key are null/empty, throw error
|
||||
try {
|
||||
baseRecord.put("name", "")
|
||||
baseRecord.put("field1", null)
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1,name")
|
||||
new GlobalDeleteKeyGenerator(props).getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: HoodieKeyException =>
|
||||
// do nothing
|
||||
}
|
||||
|
||||
// if all parts of the composite record key are null/empty, throw error
|
||||
try {
|
||||
baseRecord.put("name", "")
|
||||
baseRecord.put("field1", null)
|
||||
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1,name")
|
||||
val keyGen = new GlobalDeleteKeyGenerator(props)
|
||||
keyGen.getRecordKey(baseRow)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
case e: HoodieKeyException =>
|
||||
// do nothing
|
||||
}
|
||||
}
|
||||
|
||||
@Test def testOverwriteWithLatestAvroPayload() = {
|
||||
val overWritePayload1 = new OverwriteWithLatestAvroPayload(baseRecord, 1)
|
||||
val laterRecord = SchemaTestUtil
|
||||
.generateAvroRecordFromJson(schema, 2, "001", "f1")
|
||||
val overWritePayload2 = new OverwriteWithLatestAvroPayload(laterRecord, 2)
|
||||
|
||||
// it will provide the record with greatest combine value
|
||||
val combinedPayload12 = overWritePayload1.preCombine(overWritePayload2)
|
||||
val combinedGR12 = combinedPayload12.getInsertValue(schema).get().asInstanceOf[GenericRecord]
|
||||
assertEquals("field2", combinedGR12.get("field1").toString)
|
||||
|
||||
// and it will be deterministic, to order of processing.
|
||||
val combinedPayload21 = overWritePayload2.preCombine(overWritePayload1)
|
||||
val combinedGR21 = combinedPayload21.getInsertValue(schema).get().asInstanceOf[GenericRecord]
|
||||
assertEquals("field2", combinedGR21.get("field1").toString)
|
||||
}
|
||||
|
||||
@Test def testEmptyHoodieRecordPayload() = {
|
||||
val emptyPayload1 = new EmptyHoodieRecordPayload(baseRecord, 1)
|
||||
val laterRecord = SchemaTestUtil
|
||||
.generateAvroRecordFromJson(schema, 2, "001", "f1")
|
||||
val emptyPayload2 = new EmptyHoodieRecordPayload(laterRecord, 2)
|
||||
|
||||
// it will provide an empty record
|
||||
val combinedPayload12 = emptyPayload1.preCombine(emptyPayload2)
|
||||
val combined12 = combinedPayload12.getInsertValue(schema)
|
||||
assertEquals(Option.empty(), combined12)
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,106 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi
|
||||
|
||||
import java.io.File
|
||||
import java.nio.file.Paths
|
||||
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.spark.sql.SparkSession
|
||||
import org.junit.jupiter.api.Assertions.assertEquals
|
||||
import org.junit.jupiter.api.Test
|
||||
import org.junit.jupiter.api.io.TempDir
|
||||
|
||||
class TestHoodieSparkUtils {
|
||||
|
||||
@Test
|
||||
def testGlobPaths(@TempDir tempDir: File): Unit = {
|
||||
val folders: Seq[Path] = Seq(
|
||||
new Path(Paths.get(tempDir.getAbsolutePath, "folder1").toUri),
|
||||
new Path(Paths.get(tempDir.getAbsolutePath, "folder2").toUri)
|
||||
)
|
||||
|
||||
val files: Seq[Path] = Seq(
|
||||
new Path(Paths.get(tempDir.getAbsolutePath, "folder1", "file1").toUri),
|
||||
new Path(Paths.get(tempDir.getAbsolutePath, "folder1", "file2").toUri),
|
||||
new Path(Paths.get(tempDir.getAbsolutePath, "folder2", "file3").toUri),
|
||||
new Path(Paths.get(tempDir.getAbsolutePath, "folder2", "file4").toUri)
|
||||
)
|
||||
|
||||
folders.foreach(folder => new File(folder.toUri).mkdir())
|
||||
files.foreach(file => new File(file.toUri).createNewFile())
|
||||
|
||||
var paths = Seq(tempDir.getAbsolutePath + "/*")
|
||||
var globbedPaths = HoodieSparkUtils.checkAndGlobPathIfNecessary(paths,
|
||||
new Path(paths.head).getFileSystem(new Configuration()))
|
||||
assertEquals(folders.sortWith(_.toString < _.toString), globbedPaths.sortWith(_.toString < _.toString))
|
||||
|
||||
paths = Seq(tempDir.getAbsolutePath + "/*/*")
|
||||
globbedPaths = HoodieSparkUtils.checkAndGlobPathIfNecessary(paths,
|
||||
new Path(paths.head).getFileSystem(new Configuration()))
|
||||
assertEquals(files.sortWith(_.toString < _.toString), globbedPaths.sortWith(_.toString < _.toString))
|
||||
|
||||
paths = Seq(tempDir.getAbsolutePath + "/folder1/*")
|
||||
globbedPaths = HoodieSparkUtils.checkAndGlobPathIfNecessary(paths,
|
||||
new Path(paths.head).getFileSystem(new Configuration()))
|
||||
assertEquals(Seq(files(0), files(1)).sortWith(_.toString < _.toString),
|
||||
globbedPaths.sortWith(_.toString < _.toString))
|
||||
|
||||
paths = Seq(tempDir.getAbsolutePath + "/folder2/*")
|
||||
globbedPaths = HoodieSparkUtils.checkAndGlobPathIfNecessary(paths,
|
||||
new Path(paths.head).getFileSystem(new Configuration()))
|
||||
assertEquals(Seq(files(2), files(3)).sortWith(_.toString < _.toString),
|
||||
globbedPaths.sortWith(_.toString < _.toString))
|
||||
|
||||
paths = Seq(tempDir.getAbsolutePath + "/folder1/*", tempDir.getAbsolutePath + "/folder2/*")
|
||||
globbedPaths = HoodieSparkUtils.checkAndGlobPathIfNecessary(paths,
|
||||
new Path(paths.head).getFileSystem(new Configuration()))
|
||||
assertEquals(files.sortWith(_.toString < _.toString), globbedPaths.sortWith(_.toString < _.toString))
|
||||
}
|
||||
|
||||
@Test
|
||||
def testCreateInMemoryIndex(@TempDir tempDir: File): Unit = {
|
||||
val spark = SparkSession.builder
|
||||
.appName("Hoodie Datasource test")
|
||||
.master("local[2]")
|
||||
.config("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
|
||||
.getOrCreate
|
||||
|
||||
val folders: Seq[Path] = Seq(
|
||||
new Path(Paths.get(tempDir.getAbsolutePath, "folder1").toUri),
|
||||
new Path(Paths.get(tempDir.getAbsolutePath, "folder2").toUri)
|
||||
)
|
||||
|
||||
val files: Seq[Path] = Seq(
|
||||
new Path(Paths.get(tempDir.getAbsolutePath, "folder1", "file1").toUri),
|
||||
new Path(Paths.get(tempDir.getAbsolutePath, "folder1", "file2").toUri),
|
||||
new Path(Paths.get(tempDir.getAbsolutePath, "folder2", "file3").toUri),
|
||||
new Path(Paths.get(tempDir.getAbsolutePath, "folder2", "file4").toUri)
|
||||
)
|
||||
|
||||
folders.foreach(folder => new File(folder.toUri).mkdir())
|
||||
files.foreach(file => new File(file.toUri).createNewFile())
|
||||
|
||||
val index = HoodieSparkUtils.createInMemoryFileIndex(spark, Seq(folders(0), folders(1)))
|
||||
val indexedFilePaths = index.allFiles().map(fs => fs.getPath)
|
||||
assertEquals(files.sortWith(_.toString < _.toString), indexedFilePaths.sortWith(_.toString < _.toString))
|
||||
spark.stop()
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,417 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.functional
|
||||
|
||||
import java.time.Instant
|
||||
import java.util
|
||||
import java.util.{Collections, Date, UUID}
|
||||
|
||||
import org.apache.commons.io.FileUtils
|
||||
import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.client.{SparkRDDWriteClient, TestBootstrap}
|
||||
import org.apache.hudi.common.model.{HoodieRecord, HoodieRecordPayload}
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator
|
||||
import org.apache.hudi.config.{HoodieBootstrapConfig, HoodieWriteConfig}
|
||||
import org.apache.hudi.exception.HoodieException
|
||||
import org.apache.hudi.keygen.{NonpartitionedKeyGenerator, SimpleKeyGenerator}
|
||||
import org.apache.hudi.testutils.DataSourceTestUtils
|
||||
import org.apache.hudi.{AvroConversionUtils, DataSourceUtils, DataSourceWriteOptions, HoodieSparkSqlWriter, HoodieWriterUtils}
|
||||
import org.apache.spark.SparkContext
|
||||
import org.apache.spark.api.java.JavaSparkContext
|
||||
import org.apache.spark.sql.{Row, SQLContext, SaveMode, SparkSession}
|
||||
import org.mockito.ArgumentMatchers.any
|
||||
import org.mockito.Mockito.{spy, times, verify}
|
||||
import org.scalatest.{FunSuite, Matchers}
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
|
||||
class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
|
||||
var spark: SparkSession = _
|
||||
var sc: SparkContext = _
|
||||
var sqlContext: SQLContext = _
|
||||
|
||||
test("Parameters With Write Defaults") {
|
||||
val originals = HoodieWriterUtils.parametersWithWriteDefaults(Map.empty)
|
||||
val rhsKey = "hoodie.right.hand.side.key"
|
||||
val rhsVal = "hoodie.right.hand.side.val"
|
||||
val modifier = Map(OPERATION_OPT_KEY -> INSERT_OPERATION_OPT_VAL, TABLE_TYPE_OPT_KEY -> MOR_TABLE_TYPE_OPT_VAL, rhsKey -> rhsVal)
|
||||
val modified = HoodieWriterUtils.parametersWithWriteDefaults(modifier)
|
||||
val matcher = (k: String, v: String) => modified(k) should be(v)
|
||||
|
||||
originals foreach {
|
||||
case (OPERATION_OPT_KEY, _) => matcher(OPERATION_OPT_KEY, INSERT_OPERATION_OPT_VAL)
|
||||
case (TABLE_TYPE_OPT_KEY, _) => matcher(TABLE_TYPE_OPT_KEY, MOR_TABLE_TYPE_OPT_VAL)
|
||||
case (`rhsKey`, _) => matcher(rhsKey, rhsVal)
|
||||
case (k, v) => matcher(k, v)
|
||||
}
|
||||
}
|
||||
|
||||
test("throw hoodie exception when invalid serializer") {
|
||||
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 -> "hoodie_test_tbl")
|
||||
val e = intercept[HoodieException](HoodieSparkSqlWriter.write(sqlContext, SaveMode.ErrorIfExists, options,
|
||||
session.emptyDataFrame))
|
||||
assert(e.getMessage.contains("spark.serializer"))
|
||||
} finally {
|
||||
session.stop()
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
test("throw hoodie exception when there already exist a table with different name with Append Save mode") {
|
||||
|
||||
initSparkContext("test_append_mode")
|
||||
val path = java.nio.file.Files.createTempDirectory("hoodie_test_path")
|
||||
try {
|
||||
|
||||
val hoodieFooTableName = "hoodie_foo_tbl"
|
||||
|
||||
//create a new table
|
||||
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
|
||||
HoodieWriteConfig.TABLE_NAME -> hoodieFooTableName,
|
||||
"hoodie.insert.shuffle.parallelism" -> "4",
|
||||
"hoodie.upsert.shuffle.parallelism" -> "4")
|
||||
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
||||
val dataFrame = spark.createDataFrame(Seq(Test(UUID.randomUUID().toString, new Date().getTime)))
|
||||
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, dataFrame)
|
||||
|
||||
//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 -> "hoodie_bar_tbl",
|
||||
"hoodie.insert.shuffle.parallelism" -> "4",
|
||||
"hoodie.upsert.shuffle.parallelism" -> "4")
|
||||
val barTableParams = HoodieWriterUtils.parametersWithWriteDefaults(barTableModifier)
|
||||
val dataFrame2 = spark.createDataFrame(Seq(Test(UUID.randomUUID().toString, new Date().getTime)))
|
||||
val tableAlreadyExistException = intercept[HoodieException](HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, barTableParams, dataFrame2))
|
||||
assert(tableAlreadyExistException.getMessage.contains("hoodie table with name " + hoodieFooTableName + " already exist"))
|
||||
|
||||
//on same path try append with delete operation and different("hoodie_bar_tbl") table name which should throw an exception
|
||||
val deleteTableParams = barTableParams ++ Map(OPERATION_OPT_KEY -> "delete")
|
||||
val deleteCmdException = intercept[HoodieException](HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, deleteTableParams, dataFrame2))
|
||||
assert(deleteCmdException.getMessage.contains("hoodie table with name " + hoodieFooTableName + " already exist"))
|
||||
} finally {
|
||||
spark.stop()
|
||||
FileUtils.deleteDirectory(path.toFile)
|
||||
}
|
||||
}
|
||||
|
||||
test("test bulk insert dataset with datasource impl") {
|
||||
initSparkContext("test_bulk_insert_datasource")
|
||||
val path = java.nio.file.Files.createTempDirectory("hoodie_test_path")
|
||||
try {
|
||||
|
||||
val hoodieFooTableName = "hoodie_foo_tbl"
|
||||
|
||||
//create a new table
|
||||
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
|
||||
HoodieWriteConfig.TABLE_NAME -> hoodieFooTableName,
|
||||
"hoodie.bulkinsert.shuffle.parallelism" -> "4",
|
||||
DataSourceWriteOptions.OPERATION_OPT_KEY -> DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL,
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
|
||||
DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY -> "org.apache.hudi.keygen.SimpleKeyGenerator")
|
||||
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
||||
|
||||
// generate the inserts
|
||||
val schema = DataSourceTestUtils.getStructTypeExampleSchema
|
||||
val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema)
|
||||
val records = DataSourceTestUtils.generateRandomRows(100)
|
||||
val recordsSeq = convertRowListToSeq(records)
|
||||
val df = spark.createDataFrame(sc.parallelize(recordsSeq), structType)
|
||||
// write to Hudi
|
||||
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, df)
|
||||
|
||||
// collect all parition paths to issue read of parquet files
|
||||
val partitions = Seq(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH,
|
||||
HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH)
|
||||
// Check the entire dataset has all records still
|
||||
val fullPartitionPaths = new Array[String](3)
|
||||
for (i <- 0 until fullPartitionPaths.length) {
|
||||
fullPartitionPaths(i) = String.format("%s/%s/*", path.toAbsolutePath.toString, partitions(i))
|
||||
}
|
||||
|
||||
// fetch all records from parquet files generated from write to hudi
|
||||
val actualDf = sqlContext.read.parquet(fullPartitionPaths(0), fullPartitionPaths(1), fullPartitionPaths(2))
|
||||
|
||||
// remove metadata columns so that expected and actual DFs can be compared as is
|
||||
val trimmedDf = actualDf.drop(HoodieRecord.HOODIE_META_COLUMNS.get(0)).drop(HoodieRecord.HOODIE_META_COLUMNS.get(1))
|
||||
.drop(HoodieRecord.HOODIE_META_COLUMNS.get(2)).drop(HoodieRecord.HOODIE_META_COLUMNS.get(3))
|
||||
.drop(HoodieRecord.HOODIE_META_COLUMNS.get(4))
|
||||
|
||||
assert(df.except(trimmedDf).count() == 0)
|
||||
} finally {
|
||||
spark.stop()
|
||||
FileUtils.deleteDirectory(path.toFile)
|
||||
}
|
||||
}
|
||||
|
||||
test("test insert dataset without precombine field") {
|
||||
val session = SparkSession.builder()
|
||||
.appName("test_insert_without_precombine")
|
||||
.master("local[2]")
|
||||
.config("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
|
||||
.getOrCreate()
|
||||
val path = java.nio.file.Files.createTempDirectory("hoodie_test_path")
|
||||
try {
|
||||
|
||||
val sqlContext = session.sqlContext
|
||||
val sc = session.sparkContext
|
||||
val hoodieFooTableName = "hoodie_foo_tbl"
|
||||
|
||||
//create a new table
|
||||
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
|
||||
HoodieWriteConfig.TABLE_NAME -> hoodieFooTableName,
|
||||
"hoodie.bulkinsert.shuffle.parallelism" -> "1",
|
||||
DataSourceWriteOptions.OPERATION_OPT_KEY -> DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL,
|
||||
DataSourceWriteOptions.INSERT_DROP_DUPS_OPT_KEY -> "false",
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
|
||||
DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY -> "org.apache.hudi.keygen.SimpleKeyGenerator")
|
||||
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
||||
|
||||
// generate the inserts
|
||||
val schema = DataSourceTestUtils.getStructTypeExampleSchema
|
||||
val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema)
|
||||
val records = DataSourceTestUtils.generateRandomRows(100)
|
||||
val recordsSeq = convertRowListToSeq(records)
|
||||
val df = session.createDataFrame(sc.parallelize(recordsSeq), structType)
|
||||
// write to Hudi
|
||||
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams - DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, df)
|
||||
|
||||
// collect all parition paths to issue read of parquet files
|
||||
val partitions = Seq(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH,
|
||||
HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH)
|
||||
// Check the entire dataset has all records still
|
||||
val fullPartitionPaths = new Array[String](3)
|
||||
for (i <- 0 until fullPartitionPaths.length) {
|
||||
fullPartitionPaths(i) = String.format("%s/%s/*", path.toAbsolutePath.toString, partitions(i))
|
||||
}
|
||||
|
||||
// fetch all records from parquet files generated from write to hudi
|
||||
val actualDf = session.sqlContext.read.parquet(fullPartitionPaths(0), fullPartitionPaths(1), fullPartitionPaths(2))
|
||||
|
||||
// remove metadata columns so that expected and actual DFs can be compared as is
|
||||
val trimmedDf = actualDf.drop(HoodieRecord.HOODIE_META_COLUMNS.get(0)).drop(HoodieRecord.HOODIE_META_COLUMNS.get(1))
|
||||
.drop(HoodieRecord.HOODIE_META_COLUMNS.get(2)).drop(HoodieRecord.HOODIE_META_COLUMNS.get(3))
|
||||
.drop(HoodieRecord.HOODIE_META_COLUMNS.get(4))
|
||||
|
||||
assert(df.except(trimmedDf).count() == 0)
|
||||
} finally {
|
||||
session.stop()
|
||||
FileUtils.deleteDirectory(path.toFile)
|
||||
}
|
||||
}
|
||||
|
||||
test("test bulk insert dataset with datasource impl multiple rounds") {
|
||||
initSparkContext("test_bulk_insert_datasource")
|
||||
val path = java.nio.file.Files.createTempDirectory("hoodie_test_path")
|
||||
try {
|
||||
|
||||
val hoodieFooTableName = "hoodie_foo_tbl"
|
||||
|
||||
//create a new table
|
||||
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
|
||||
HoodieWriteConfig.TABLE_NAME -> hoodieFooTableName,
|
||||
"hoodie.bulkinsert.shuffle.parallelism" -> "4",
|
||||
DataSourceWriteOptions.OPERATION_OPT_KEY -> DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL,
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
|
||||
DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY -> "org.apache.hudi.keygen.SimpleKeyGenerator")
|
||||
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
||||
|
||||
val partitions = Seq(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH,
|
||||
HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH)
|
||||
val fullPartitionPaths = new Array[String](3)
|
||||
for (i <- 0 to 2) {
|
||||
fullPartitionPaths(i) = String.format("%s/%s/*", path.toAbsolutePath.toString, partitions(i))
|
||||
}
|
||||
|
||||
val schema = DataSourceTestUtils.getStructTypeExampleSchema
|
||||
val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema)
|
||||
var totalExpectedDf = spark.createDataFrame(sc.emptyRDD[Row], structType)
|
||||
|
||||
for (_ <- 0 to 2) {
|
||||
// generate the inserts
|
||||
val records = DataSourceTestUtils.generateRandomRows(200)
|
||||
val recordsSeq = convertRowListToSeq(records)
|
||||
val df = spark.createDataFrame(sc.parallelize(recordsSeq), structType)
|
||||
// write to Hudi
|
||||
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, df)
|
||||
|
||||
// Fetch records from entire dataset
|
||||
val actualDf = sqlContext.read.parquet(fullPartitionPaths(0), fullPartitionPaths(1), fullPartitionPaths(2))
|
||||
|
||||
// remove metadata columns so that expected and actual DFs can be compared as is
|
||||
val trimmedDf = actualDf.drop(HoodieRecord.HOODIE_META_COLUMNS.get(0)).drop(HoodieRecord.HOODIE_META_COLUMNS.get(1))
|
||||
.drop(HoodieRecord.HOODIE_META_COLUMNS.get(2)).drop(HoodieRecord.HOODIE_META_COLUMNS.get(3))
|
||||
.drop(HoodieRecord.HOODIE_META_COLUMNS.get(4))
|
||||
|
||||
// find total df (union from multiple rounds)
|
||||
totalExpectedDf = totalExpectedDf.union(df)
|
||||
// find mismatch between actual and expected df
|
||||
assert(totalExpectedDf.except(trimmedDf).count() == 0)
|
||||
}
|
||||
} finally {
|
||||
spark.stop()
|
||||
FileUtils.deleteDirectory(path.toFile)
|
||||
}
|
||||
}
|
||||
|
||||
List(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.foreach(tableType => {
|
||||
test("test basic HoodieSparkSqlWriter functionality with datasource insert for " + tableType) {
|
||||
initSparkContext("test_insert_datasource")
|
||||
val path = java.nio.file.Files.createTempDirectory("hoodie_test_path")
|
||||
try {
|
||||
|
||||
val hoodieFooTableName = "hoodie_foo_tbl"
|
||||
|
||||
//create a new table
|
||||
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
|
||||
HoodieWriteConfig.TABLE_NAME -> hoodieFooTableName,
|
||||
DataSourceWriteOptions.TABLE_TYPE_OPT_KEY -> tableType,
|
||||
HoodieWriteConfig.INSERT_PARALLELISM -> "4",
|
||||
DataSourceWriteOptions.OPERATION_OPT_KEY -> DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL,
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
|
||||
DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY -> classOf[SimpleKeyGenerator].getCanonicalName)
|
||||
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
||||
|
||||
// generate the inserts
|
||||
val schema = DataSourceTestUtils.getStructTypeExampleSchema
|
||||
val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema)
|
||||
val records = DataSourceTestUtils.generateRandomRows(100)
|
||||
val recordsSeq = convertRowListToSeq(records)
|
||||
val df = spark.createDataFrame(sc.parallelize(recordsSeq), structType)
|
||||
|
||||
val client = spy(DataSourceUtils.createHoodieClient(
|
||||
new JavaSparkContext(sc),
|
||||
schema.toString,
|
||||
path.toAbsolutePath.toString,
|
||||
hoodieFooTableName,
|
||||
mapAsJavaMap(fooTableParams)).asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]])
|
||||
|
||||
// write to Hudi
|
||||
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, df, Option.empty,
|
||||
Option(client))
|
||||
// Verify that asynchronous compaction is not scheduled
|
||||
verify(client, times(0)).scheduleCompaction(any())
|
||||
// Verify that HoodieWriteClient is closed correctly
|
||||
verify(client, times(1)).close()
|
||||
|
||||
// collect all partition paths to issue read of parquet files
|
||||
val partitions = Seq(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH,
|
||||
HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH, HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH)
|
||||
// Check the entire dataset has all records still
|
||||
val fullPartitionPaths = new Array[String](3)
|
||||
for (i <- fullPartitionPaths.indices) {
|
||||
fullPartitionPaths(i) = String.format("%s/%s/*", path.toAbsolutePath.toString, partitions(i))
|
||||
}
|
||||
|
||||
// fetch all records from parquet files generated from write to hudi
|
||||
val actualDf = sqlContext.read.parquet(fullPartitionPaths(0), fullPartitionPaths(1), fullPartitionPaths(2))
|
||||
|
||||
// remove metadata columns so that expected and actual DFs can be compared as is
|
||||
val trimmedDf = actualDf.drop(HoodieRecord.HOODIE_META_COLUMNS.get(0)).drop(HoodieRecord.HOODIE_META_COLUMNS.get(1))
|
||||
.drop(HoodieRecord.HOODIE_META_COLUMNS.get(2)).drop(HoodieRecord.HOODIE_META_COLUMNS.get(3))
|
||||
.drop(HoodieRecord.HOODIE_META_COLUMNS.get(4))
|
||||
|
||||
assert(df.except(trimmedDf).count() == 0)
|
||||
} finally {
|
||||
spark.stop()
|
||||
FileUtils.deleteDirectory(path.toFile)
|
||||
}
|
||||
}
|
||||
})
|
||||
|
||||
List(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.foreach(tableType => {
|
||||
test("test HoodieSparkSqlWriter functionality with datasource bootstrap for " + tableType) {
|
||||
initSparkContext("test_bootstrap_datasource")
|
||||
val path = java.nio.file.Files.createTempDirectory("hoodie_test_path")
|
||||
val srcPath = java.nio.file.Files.createTempDirectory("hoodie_bootstrap_source_path")
|
||||
|
||||
try {
|
||||
|
||||
val hoodieFooTableName = "hoodie_foo_tbl"
|
||||
|
||||
val sourceDF = TestBootstrap.generateTestRawTripDataset(Instant.now.toEpochMilli, 0, 100, Collections.emptyList(), sc,
|
||||
spark.sqlContext)
|
||||
|
||||
// Write source data non-partitioned
|
||||
sourceDF.write
|
||||
.format("parquet")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(srcPath.toAbsolutePath.toString)
|
||||
|
||||
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
|
||||
HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP -> srcPath.toAbsolutePath.toString,
|
||||
HoodieWriteConfig.TABLE_NAME -> hoodieFooTableName,
|
||||
DataSourceWriteOptions.TABLE_TYPE_OPT_KEY -> tableType,
|
||||
HoodieBootstrapConfig.BOOTSTRAP_PARALLELISM -> "4",
|
||||
DataSourceWriteOptions.OPERATION_OPT_KEY -> DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL,
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
|
||||
HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS -> classOf[NonpartitionedKeyGenerator].getCanonicalName)
|
||||
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
||||
|
||||
val client = spy(DataSourceUtils.createHoodieClient(
|
||||
new JavaSparkContext(sc),
|
||||
null,
|
||||
path.toAbsolutePath.toString,
|
||||
hoodieFooTableName,
|
||||
mapAsJavaMap(fooTableParams)).asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]])
|
||||
|
||||
HoodieSparkSqlWriter.bootstrap(sqlContext, SaveMode.Append, fooTableParams, spark.emptyDataFrame, Option.empty,
|
||||
Option(client))
|
||||
|
||||
// Verify that HoodieWriteClient is closed correctly
|
||||
verify(client, times(1)).close()
|
||||
|
||||
// fetch all records from parquet files generated from write to hudi
|
||||
val actualDf = sqlContext.read.parquet(path.toAbsolutePath.toString)
|
||||
assert(actualDf.count == 100)
|
||||
} finally {
|
||||
spark.stop()
|
||||
FileUtils.deleteDirectory(path.toFile)
|
||||
FileUtils.deleteDirectory(srcPath.toFile)
|
||||
}
|
||||
}
|
||||
})
|
||||
|
||||
case class Test(uuid: String, ts: Long)
|
||||
|
||||
import scala.collection.JavaConverters
|
||||
|
||||
def convertRowListToSeq(inputList: util.List[Row]): Seq[Row] =
|
||||
JavaConverters.asScalaIteratorConverter(inputList.iterator).asScala.toSeq
|
||||
|
||||
def initSparkContext(appName: String): Unit = {
|
||||
spark = SparkSession.builder()
|
||||
.appName(appName)
|
||||
.master("local[2]")
|
||||
.config("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
|
||||
.getOrCreate()
|
||||
sc = spark.sparkContext
|
||||
sc.setLogLevel("ERROR")
|
||||
sqlContext = spark.sqlContext
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,323 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.functional
|
||||
|
||||
import java.sql.{Date, Timestamp}
|
||||
import java.util.function.Supplier
|
||||
import java.util.stream.Stream
|
||||
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase
|
||||
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers}
|
||||
import org.apache.spark.sql._
|
||||
import org.apache.spark.sql.functions.col
|
||||
import org.apache.spark.sql.types.{DataTypes, DateType, IntegerType, StringType, StructField, StructType, TimestampType}
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
|
||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
|
||||
/**
|
||||
* Basic tests on the spark datasource for COW table.
|
||||
*/
|
||||
|
||||
class TestCOWDataSource extends HoodieClientTestBase {
|
||||
var spark: SparkSession = null
|
||||
val commonOpts = Map(
|
||||
"hoodie.insert.shuffle.parallelism" -> "4",
|
||||
"hoodie.upsert.shuffle.parallelism" -> "4",
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
|
||||
DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY -> "timestamp",
|
||||
HoodieWriteConfig.TABLE_NAME -> "hoodie_test"
|
||||
)
|
||||
|
||||
@BeforeEach override def setUp() {
|
||||
initPath()
|
||||
initSparkContexts()
|
||||
spark = sqlContext.sparkSession
|
||||
initTestDataGenerator()
|
||||
initFileSystem()
|
||||
}
|
||||
|
||||
@AfterEach override def tearDown() = {
|
||||
cleanupSparkContexts()
|
||||
cleanupTestDataGenerator()
|
||||
cleanupFileSystem()
|
||||
}
|
||||
|
||||
@Test def testShortNameStorage() {
|
||||
// Insert Operation
|
||||
val records = recordsToStrings(dataGen.generateInserts("000", 100)).toList
|
||||
val inputDF = spark.read.json(spark.sparkContext.parallelize(records, 2))
|
||||
inputDF.write.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000"))
|
||||
}
|
||||
|
||||
@Test def testCopyOnWriteStorage() {
|
||||
// Insert Operation
|
||||
val records1 = recordsToStrings(dataGen.generateInserts("000", 100)).toList
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000"))
|
||||
val commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
|
||||
// Snapshot query
|
||||
val snapshotDF1 = spark.read.format("org.apache.hudi").load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, snapshotDF1.count())
|
||||
|
||||
val records2 = recordsToStrings(dataGen.generateUpdates("001", 100)).toList
|
||||
val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
val uniqueKeyCnt = inputDF2.select("_row_key").distinct().count()
|
||||
|
||||
// Upsert Operation
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
val commitInstantTime2 = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
assertEquals(2, HoodieDataSourceHelpers.listCommitsSince(fs, basePath, "000").size())
|
||||
|
||||
// Snapshot Query
|
||||
val snapshotDF2 = spark.read.format("org.apache.hudi")
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, snapshotDF2.count()) // still 100, since we only updated
|
||||
|
||||
// Read Incremental Query
|
||||
// we have 2 commits, try pulling the first commit (which is not the latest)
|
||||
val firstCommit = HoodieDataSourceHelpers.listCommitsSince(fs, basePath, "000").get(0)
|
||||
val hoodieIncViewDF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000")
|
||||
.option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, firstCommit)
|
||||
.load(basePath)
|
||||
assertEquals(100, hoodieIncViewDF1.count()) // 100 initial inserts must be pulled
|
||||
var countsPerCommit = hoodieIncViewDF1.groupBy("_hoodie_commit_time").count().collect()
|
||||
assertEquals(1, countsPerCommit.length)
|
||||
assertEquals(firstCommit, countsPerCommit(0).get(0))
|
||||
|
||||
// Upsert an empty dataFrame
|
||||
val emptyRecords = recordsToStrings(dataGen.generateUpdates("002", 0)).toList
|
||||
val emptyDF = spark.read.json(spark.sparkContext.parallelize(emptyRecords, 1))
|
||||
emptyDF.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
// pull the latest commit
|
||||
val hoodieIncViewDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.load(basePath)
|
||||
|
||||
assertEquals(uniqueKeyCnt, hoodieIncViewDF2.count()) // 100 records must be pulled
|
||||
countsPerCommit = hoodieIncViewDF2.groupBy("_hoodie_commit_time").count().collect()
|
||||
assertEquals(1, countsPerCommit.length)
|
||||
assertEquals(commitInstantTime2, countsPerCommit(0).get(0))
|
||||
|
||||
// pull the latest commit within certain partitions
|
||||
val hoodieIncViewDF3 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.option(DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY, "/2016/*/*/*")
|
||||
.load(basePath)
|
||||
assertEquals(hoodieIncViewDF2.filter(col("_hoodie_partition_path").contains("2016")).count(), hoodieIncViewDF3.count())
|
||||
|
||||
val timeTravelDF = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000")
|
||||
.option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, firstCommit)
|
||||
.load(basePath)
|
||||
assertEquals(100, timeTravelDF.count()) // 100 initial inserts must be pulled
|
||||
}
|
||||
|
||||
@Test def testOverWriteModeUseReplaceAction(): Unit = {
|
||||
val records1 = recordsToStrings(dataGen.generateInserts("001", 5)).toList
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
val records2 = recordsToStrings(dataGen.generateInserts("002", 5)).toList
|
||||
val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
val metaClient = new HoodieTableMetaClient(spark.sparkContext.hadoopConfiguration, basePath, true)
|
||||
val commits = metaClient.getActiveTimeline.filterCompletedInstants().getInstants.toArray
|
||||
.map(instant => (instant.asInstanceOf[HoodieInstant]).getAction)
|
||||
assertEquals(2, commits.size)
|
||||
assertEquals("commit", commits(0))
|
||||
assertEquals("replacecommit", commits(1))
|
||||
}
|
||||
|
||||
@Test def testOverWriteModeUseReplaceActionOnDisJointPartitions(): Unit = {
|
||||
// step1: Write 5 records to hoodie table for partition1 DEFAULT_FIRST_PARTITION_PATH
|
||||
val records1 = recordsToStrings(dataGen.generateInsertsForPartition("001", 5, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)).toList
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
// step2: Write 7 more rectestOverWriteModeUseReplaceActionords using SaveMode.Overwrite for partition2 DEFAULT_SECOND_PARTITION_PATH
|
||||
val records2 = recordsToStrings(dataGen.generateInsertsForPartition("002", 7, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH)).toList
|
||||
val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
val allRecords = spark.read.format("org.apache.hudi").load(basePath + "/*/*/*")
|
||||
allRecords.registerTempTable("tmpTable")
|
||||
|
||||
spark.sql(String.format("select count(*) from tmpTable")).show()
|
||||
|
||||
// step3: Query the rows count from hoodie table for partition1 DEFAULT_FIRST_PARTITION_PATH
|
||||
val recordCountForParititon1 = spark.sql(String.format("select count(*) from tmpTable where partition = '%s'", HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)).collect()
|
||||
assertEquals("0", recordCountForParititon1(0).get(0).toString)
|
||||
|
||||
// step4: Query the rows count from hoodie table for partition1 DEFAULT_SECOND_PARTITION_PATH
|
||||
val recordCountForParititon2 = spark.sql(String.format("select count(*) from tmpTable where partition = '%s'", HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH)).collect()
|
||||
assertEquals("7", recordCountForParititon2(0).get(0).toString)
|
||||
|
||||
// step5: Query the rows count from hoodie table
|
||||
val recordCount = spark.sql(String.format("select count(*) from tmpTable")).collect()
|
||||
assertEquals("7", recordCountForParititon2(0).get(0).toString)
|
||||
|
||||
// step6: Query the rows count from hoodie table for partition1 DEFAULT_SECOND_PARTITION_PATH using spark.collect and then filter mode
|
||||
val recordsForPartitionColumn = spark.sql(String.format("select partition from tmpTable")).collect()
|
||||
val filterSecondPartitionCount = recordsForPartitionColumn.filter(row => row.get(0).equals(HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH)).size
|
||||
assertEquals(7,filterSecondPartitionCount)
|
||||
|
||||
val metaClient = new HoodieTableMetaClient(spark.sparkContext.hadoopConfiguration, basePath, true)
|
||||
val commits = metaClient.getActiveTimeline.filterCompletedInstants().getInstants.toArray
|
||||
.map(instant => instant.asInstanceOf[HoodieInstant].getAction)
|
||||
assertEquals(2, commits.size)
|
||||
assertEquals("commit", commits(0))
|
||||
assertEquals("replacecommit", commits(1))
|
||||
}
|
||||
|
||||
@Test def testDropInsertDup(): Unit = {
|
||||
val insert1Cnt = 10
|
||||
val insert2DupKeyCnt = 9
|
||||
val insert2NewKeyCnt = 2
|
||||
|
||||
val totalUniqueKeyToGenerate = insert1Cnt + insert2NewKeyCnt
|
||||
val allRecords = dataGen.generateInserts("001", totalUniqueKeyToGenerate)
|
||||
val inserts1 = allRecords.subList(0, insert1Cnt)
|
||||
val inserts2New = dataGen.generateSameKeyInserts("002", allRecords.subList(insert1Cnt, insert1Cnt + insert2NewKeyCnt))
|
||||
val inserts2Dup = dataGen.generateSameKeyInserts("002", inserts1.subList(0, insert2DupKeyCnt))
|
||||
|
||||
val records1 = recordsToStrings(inserts1).toList
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
val hoodieROViewDF1 = spark.read.format("org.apache.hudi")
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(insert1Cnt, hoodieROViewDF1.count())
|
||||
|
||||
val commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
val records2 = recordsToStrings(inserts2Dup ++ inserts2New).toList
|
||||
val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.INSERT_DROP_DUPS_OPT_KEY, "true")
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val hoodieROViewDF2 = spark.read.format("org.apache.hudi")
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(hoodieROViewDF2.count(), totalUniqueKeyToGenerate)
|
||||
|
||||
val hoodieIncViewDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.load(basePath)
|
||||
assertEquals(hoodieIncViewDF2.count(), insert2NewKeyCnt)
|
||||
}
|
||||
|
||||
@Test def testComplexDataTypeWriteAndReadConsistency(): Unit = {
|
||||
val schema = StructType(StructField("_row_key", StringType, true) :: StructField("name", StringType, true)
|
||||
:: StructField("timeStampValue", TimestampType, true) :: StructField("dateValue", DateType, true)
|
||||
:: StructField("decimalValue", DataTypes.createDecimalType(15, 10), true) :: StructField("timestamp", IntegerType, true)
|
||||
:: StructField("partition", IntegerType, true) :: Nil)
|
||||
|
||||
val records = Seq(Row("11", "Andy", Timestamp.valueOf("1970-01-01 13:31:24"), Date.valueOf("1991-11-07"), BigDecimal.valueOf(1.0), 11, 1),
|
||||
Row("22", "lisi", Timestamp.valueOf("1970-01-02 13:31:24"), Date.valueOf("1991-11-08"), BigDecimal.valueOf(2.0), 11, 1),
|
||||
Row("33", "zhangsan", Timestamp.valueOf("1970-01-03 13:31:24"), Date.valueOf("1991-11-09"), BigDecimal.valueOf(3.0), 11, 1))
|
||||
val rdd = jsc.parallelize(records)
|
||||
val recordsDF = spark.createDataFrame(rdd, schema)
|
||||
recordsDF.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
val recordsReadDF = spark.read.format("org.apache.hudi")
|
||||
.load(basePath + "/*/*")
|
||||
recordsReadDF.printSchema()
|
||||
recordsReadDF.schema.foreach(f => {
|
||||
f.name match {
|
||||
case "timeStampValue" =>
|
||||
assertEquals(f.dataType, org.apache.spark.sql.types.TimestampType)
|
||||
case "dateValue" =>
|
||||
assertEquals(f.dataType, org.apache.spark.sql.types.DateType)
|
||||
case "decimalValue" =>
|
||||
assertEquals(f.dataType, org.apache.spark.sql.types.DecimalType(15, 10))
|
||||
case _ =>
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
@Test def testWithAutoCommitOn(): Unit = {
|
||||
val records1 = recordsToStrings(dataGen.generateInserts("000", 100)).toList
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(HoodieWriteConfig.HOODIE_AUTO_COMMIT_PROP, "true")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000"))
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,630 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.functional
|
||||
|
||||
import java.time.Instant
|
||||
import java.util.Collections
|
||||
|
||||
import collection.JavaConverters._
|
||||
import org.apache.hadoop.fs.FileSystem
|
||||
import org.apache.hudi.bootstrap.SparkParquetBootstrapDataProvider
|
||||
import org.apache.hudi.client.TestBootstrap
|
||||
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.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
|
||||
|
||||
class TestDataSourceForBootstrap {
|
||||
|
||||
var spark: SparkSession = _
|
||||
val commonOpts = Map(
|
||||
HoodieWriteConfig.INSERT_PARALLELISM -> "4",
|
||||
HoodieWriteConfig.UPSERT_PARALLELISM -> "4",
|
||||
HoodieWriteConfig.DELETE_PARALLELISM -> "4",
|
||||
HoodieWriteConfig.BULKINSERT_PARALLELISM -> "4",
|
||||
HoodieWriteConfig.FINALIZE_WRITE_PARALLELISM -> "4",
|
||||
HoodieBootstrapConfig.BOOTSTRAP_PARALLELISM -> "4",
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
|
||||
DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY -> "timestamp",
|
||||
HoodieWriteConfig.TABLE_NAME -> "hoodie_test"
|
||||
)
|
||||
var basePath: String = _
|
||||
var srcPath: String = _
|
||||
var fs: FileSystem = _
|
||||
|
||||
@BeforeEach def initialize(@TempDir tempDir: java.nio.file.Path) {
|
||||
spark = SparkSession.builder
|
||||
.appName("Hoodie Datasource test")
|
||||
.master("local[2]")
|
||||
.config("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
|
||||
.getOrCreate
|
||||
basePath = tempDir.toAbsolutePath.toString + "/base"
|
||||
srcPath = tempDir.toAbsolutePath.toString + "/src"
|
||||
fs = FSUtils.getFs(basePath, spark.sparkContext.hadoopConfiguration)
|
||||
}
|
||||
|
||||
@AfterEach def tearDown(): Unit ={
|
||||
// Close spark session
|
||||
if (spark != null) {
|
||||
spark.stop()
|
||||
spark = null
|
||||
}
|
||||
|
||||
// Close file system
|
||||
if (fs != null) {
|
||||
fs.close()
|
||||
fs = null
|
||||
}
|
||||
}
|
||||
|
||||
@Test def testMetadataBootstrapCOWNonPartitioned(): Unit = {
|
||||
val timestamp = Instant.now.toEpochMilli
|
||||
val numRecords = 100
|
||||
val jsc = JavaSparkContext.fromSparkContext(spark.sparkContext)
|
||||
|
||||
val sourceDF = TestBootstrap.generateTestRawTripDataset(timestamp, 0, numRecords, Collections.emptyList(), jsc,
|
||||
spark.sqlContext)
|
||||
|
||||
// Write source data non-partitioned
|
||||
sourceDF.write
|
||||
.format("parquet")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(srcPath)
|
||||
|
||||
// Perform bootstrap
|
||||
val bootstrapDF = spark.emptyDataFrame
|
||||
bootstrapDF.write
|
||||
.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(HoodieWriteConfig.TABLE_NAME, "hoodie_test")
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key")
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP, srcPath)
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS, "org.apache.hudi.keygen.NonpartitionedKeyGenerator")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
val commitInstantTime1: String = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
assertEquals(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, commitInstantTime1)
|
||||
|
||||
// Read bootstrapped table and verify count
|
||||
var hoodieROViewDF1 = spark.read.format("hudi").load(basePath + "/*")
|
||||
assertEquals(numRecords, hoodieROViewDF1.count())
|
||||
|
||||
// Perform upsert
|
||||
val updateTimestamp = Instant.now.toEpochMilli
|
||||
val numRecordsUpdate = 10
|
||||
val updateDF = TestBootstrap.generateTestRawTripDataset(updateTimestamp, 0, numRecordsUpdate,
|
||||
Collections.emptyList(), jsc, spark.sqlContext)
|
||||
|
||||
updateDF.write
|
||||
.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(HoodieWriteConfig.TABLE_NAME, "hoodie_test")
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key")
|
||||
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, "timestamp")
|
||||
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY, "org.apache.hudi.keygen.NonpartitionedKeyGenerator")
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
val commitInstantTime2: String = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
assertEquals(1, HoodieDataSourceHelpers.listCommitsSince(fs, basePath, commitInstantTime1).size())
|
||||
|
||||
// Read table after upsert and verify count
|
||||
hoodieROViewDF1 = spark.read.format("hudi").load(basePath + "/*")
|
||||
assertEquals(numRecords, hoodieROViewDF1.count())
|
||||
assertEquals(numRecordsUpdate, hoodieROViewDF1.filter(s"timestamp == $updateTimestamp").count())
|
||||
|
||||
// incrementally pull only changes in the bootstrap commit, which would pull all the initial records written
|
||||
// during bootstrap
|
||||
val hoodieIncViewDF1 = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000")
|
||||
.option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.load(basePath)
|
||||
|
||||
assertEquals(numRecords, hoodieIncViewDF1.count())
|
||||
var countsPerCommit = hoodieIncViewDF1.groupBy("_hoodie_commit_time").count().collect();
|
||||
assertEquals(1, countsPerCommit.length)
|
||||
assertEquals(commitInstantTime1, countsPerCommit(0).get(0))
|
||||
|
||||
// incrementally pull only changes in the latest commit, which would pull only the updated records in the
|
||||
// latest commit
|
||||
val hoodieIncViewDF2 = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.load(basePath);
|
||||
|
||||
assertEquals(numRecordsUpdate, hoodieIncViewDF2.count())
|
||||
countsPerCommit = hoodieIncViewDF2.groupBy("_hoodie_commit_time").count().collect();
|
||||
assertEquals(1, countsPerCommit.length)
|
||||
assertEquals(commitInstantTime2, countsPerCommit(0).get(0))
|
||||
}
|
||||
|
||||
@Test def testMetadataBootstrapCOWHiveStylePartitioned(): Unit = {
|
||||
val timestamp = Instant.now.toEpochMilli
|
||||
val numRecords = 100
|
||||
val partitionPaths = List("2020-04-01", "2020-04-02", "2020-04-03")
|
||||
val jsc = JavaSparkContext.fromSparkContext(spark.sparkContext)
|
||||
|
||||
val sourceDF = TestBootstrap.generateTestRawTripDataset(timestamp, 0, numRecords, partitionPaths.asJava, jsc,
|
||||
spark.sqlContext)
|
||||
|
||||
// Write source data hive style partitioned
|
||||
sourceDF.write
|
||||
.partitionBy("datestr")
|
||||
.format("parquet")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(srcPath)
|
||||
|
||||
// Perform bootstrap
|
||||
val bootstrapDF = spark.emptyDataFrame
|
||||
bootstrapDF.write
|
||||
.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(HoodieWriteConfig.TABLE_NAME, "hoodie_test")
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key")
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP, srcPath)
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS, "org.apache.hudi.keygen.SimpleKeyGenerator")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
val commitInstantTime1: String = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
assertEquals(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, commitInstantTime1)
|
||||
|
||||
// Read bootstrapped table and verify count
|
||||
val hoodieROViewDF1 = spark.read.format("hudi").load(basePath + "/*")
|
||||
assertEquals(numRecords, hoodieROViewDF1.count())
|
||||
|
||||
// Perform upsert
|
||||
val updateTimestamp = Instant.now.toEpochMilli
|
||||
val numRecordsUpdate = 10
|
||||
val updateDF = TestBootstrap.generateTestRawTripDataset(updateTimestamp, 0, numRecordsUpdate, partitionPaths.asJava,
|
||||
jsc, spark.sqlContext)
|
||||
|
||||
updateDF.write
|
||||
.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(HoodieWriteConfig.TABLE_NAME, "hoodie_test")
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key")
|
||||
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, "timestamp")
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr")
|
||||
// Required because source data is hive style partitioned
|
||||
.option(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING_OPT_KEY, "true")
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
val commitInstantTime2: String = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
assertEquals(1, HoodieDataSourceHelpers.listCommitsSince(fs, basePath, commitInstantTime1).size())
|
||||
|
||||
// Read table after upsert and verify count
|
||||
val hoodieROViewDF2 = spark.read.format("hudi").load(basePath + "/*")
|
||||
assertEquals(numRecords, hoodieROViewDF2.count())
|
||||
assertEquals(numRecordsUpdate, hoodieROViewDF2.filter(s"timestamp == $updateTimestamp").count())
|
||||
|
||||
// incrementally pull only changes in the bootstrap commit, which would pull all the initial records written
|
||||
// during bootstrap
|
||||
val hoodieIncViewDF1 = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000")
|
||||
.option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.load(basePath)
|
||||
|
||||
assertEquals(numRecords, hoodieIncViewDF1.count())
|
||||
var countsPerCommit = hoodieIncViewDF1.groupBy("_hoodie_commit_time").count().collect();
|
||||
assertEquals(1, countsPerCommit.length)
|
||||
assertEquals(commitInstantTime1, countsPerCommit(0).get(0))
|
||||
|
||||
// incrementally pull only changes in the latest commit, which would pull only the updated records in the
|
||||
// latest commit
|
||||
val hoodieIncViewDF2 = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.load(basePath);
|
||||
|
||||
assertEquals(numRecordsUpdate, hoodieIncViewDF2.count())
|
||||
countsPerCommit = hoodieIncViewDF2.groupBy("_hoodie_commit_time").count().collect();
|
||||
assertEquals(1, countsPerCommit.length)
|
||||
assertEquals(commitInstantTime2, countsPerCommit(0).get(0))
|
||||
|
||||
// pull the latest commit within certain partitions
|
||||
val hoodieIncViewDF3 = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.option(DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY, "/datestr=2020-04-02/*")
|
||||
.load(basePath)
|
||||
|
||||
assertEquals(hoodieIncViewDF2.filter(col("_hoodie_partition_path").contains("2020-04-02")).count(),
|
||||
hoodieIncViewDF3.count())
|
||||
}
|
||||
|
||||
@Test def testMetadataBootstrapCOWPartitioned(): Unit = {
|
||||
val timestamp = Instant.now.toEpochMilli
|
||||
val numRecords = 100
|
||||
val partitionPaths = List("2020-04-01", "2020-04-02", "2020-04-03")
|
||||
val jsc = JavaSparkContext.fromSparkContext(spark.sparkContext)
|
||||
|
||||
var sourceDF = TestBootstrap.generateTestRawTripDataset(timestamp, 0, numRecords, partitionPaths.asJava, jsc,
|
||||
spark.sqlContext)
|
||||
|
||||
// Writing data for each partition instead of using partitionBy to avoid hive-style partitioning and hence
|
||||
// have partitioned columns stored in the data file
|
||||
partitionPaths.foreach(partitionPath => {
|
||||
sourceDF
|
||||
.filter(sourceDF("datestr").equalTo(lit(partitionPath)))
|
||||
.write
|
||||
.format("parquet")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(srcPath + "/" + partitionPath)
|
||||
})
|
||||
|
||||
// Perform bootstrap
|
||||
val bootstrapDF = spark.emptyDataFrame
|
||||
bootstrapDF.write
|
||||
.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(HoodieWriteConfig.TABLE_NAME, "hoodie_test")
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key")
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP, srcPath)
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS, "org.apache.hudi.keygen.SimpleKeyGenerator")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
val commitInstantTime1: String = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
assertEquals(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, commitInstantTime1)
|
||||
|
||||
// Read bootstrapped table and verify count
|
||||
val hoodieROViewDF1 = spark.read.format("hudi").load(basePath + "/*")
|
||||
assertEquals(numRecords, hoodieROViewDF1.count())
|
||||
|
||||
// Perform upsert
|
||||
val updateTimestamp = Instant.now.toEpochMilli
|
||||
val numRecordsUpdate = 10
|
||||
var updateDF = TestBootstrap.generateTestRawTripDataset(updateTimestamp, 0, numRecordsUpdate, partitionPaths.asJava,
|
||||
jsc, spark.sqlContext)
|
||||
|
||||
updateDF.write
|
||||
.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(HoodieWriteConfig.TABLE_NAME, "hoodie_test")
|
||||
.option("hoodie.upsert.shuffle.parallelism", "4")
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key")
|
||||
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, "timestamp")
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr")
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
val commitInstantTime2: String = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
assertEquals(1, HoodieDataSourceHelpers.listCommitsSince(fs, basePath, commitInstantTime1).size())
|
||||
|
||||
// Read table after upsert and verify count
|
||||
val hoodieROViewDF2 = spark.read.format("hudi").load(basePath + "/*")
|
||||
assertEquals(numRecords, hoodieROViewDF2.count())
|
||||
assertEquals(numRecordsUpdate, hoodieROViewDF2.filter(s"timestamp == $updateTimestamp").count())
|
||||
|
||||
// incrementally pull only changes in the bootstrap commit, which would pull all the initial records written
|
||||
// during bootstrap
|
||||
val hoodieIncViewDF1 = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000")
|
||||
.option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.load(basePath)
|
||||
|
||||
assertEquals(numRecords, hoodieIncViewDF1.count())
|
||||
var countsPerCommit = hoodieIncViewDF1.groupBy("_hoodie_commit_time").count().collect();
|
||||
assertEquals(1, countsPerCommit.length)
|
||||
assertEquals(commitInstantTime1, countsPerCommit(0).get(0))
|
||||
|
||||
// incrementally pull only changes in the latest commit, which would pull only the updated records in the
|
||||
// latest commit
|
||||
val hoodieIncViewDF2 = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.load(basePath);
|
||||
|
||||
assertEquals(numRecordsUpdate, hoodieIncViewDF2.count())
|
||||
countsPerCommit = hoodieIncViewDF2.groupBy("_hoodie_commit_time").count().collect();
|
||||
assertEquals(1, countsPerCommit.length)
|
||||
assertEquals(commitInstantTime2, countsPerCommit(0).get(0))
|
||||
|
||||
// pull the latest commit within certain partitions
|
||||
val hoodieIncViewDF3 = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.option(DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY, "/2020-04-02/*")
|
||||
.load(basePath)
|
||||
|
||||
assertEquals(hoodieIncViewDF2.filter(col("_hoodie_partition_path").contains("2020-04-02")).count(),
|
||||
hoodieIncViewDF3.count())
|
||||
}
|
||||
|
||||
@Test def testMetadataBootstrapMORPartitionedInlineCompactionOn(): Unit = {
|
||||
val timestamp = Instant.now.toEpochMilli
|
||||
val numRecords = 100
|
||||
val partitionPaths = List("2020-04-01", "2020-04-02", "2020-04-03")
|
||||
val jsc = JavaSparkContext.fromSparkContext(spark.sparkContext)
|
||||
|
||||
val sourceDF = TestBootstrap.generateTestRawTripDataset(timestamp, 0, numRecords, partitionPaths.asJava, jsc,
|
||||
spark.sqlContext)
|
||||
|
||||
// Writing data for each partition instead of using partitionBy to avoid hive-style partitioning and hence
|
||||
// have partitioned columns stored in the data file
|
||||
partitionPaths.foreach(partitionPath => {
|
||||
sourceDF
|
||||
.filter(sourceDF("datestr").equalTo(lit(partitionPath)))
|
||||
.write
|
||||
.format("parquet")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(srcPath + "/" + partitionPath)
|
||||
})
|
||||
|
||||
// Perform bootstrap
|
||||
val bootstrapDF = spark.emptyDataFrame
|
||||
bootstrapDF.write
|
||||
.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(HoodieWriteConfig.TABLE_NAME, "hoodie_test")
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key")
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP, srcPath)
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS, "org.apache.hudi.keygen.SimpleKeyGenerator")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
val commitInstantTime1: String = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
assertEquals(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, commitInstantTime1)
|
||||
|
||||
// Read bootstrapped table and verify count
|
||||
val hoodieROViewDF1 = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY,
|
||||
DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL)
|
||||
.load(basePath + "/*")
|
||||
assertEquals(numRecords, hoodieROViewDF1.count())
|
||||
|
||||
// Perform upsert
|
||||
val updateTimestamp = Instant.now.toEpochMilli
|
||||
val numRecordsUpdate = 10
|
||||
val updateDF = TestBootstrap.generateTestRawTripDataset(updateTimestamp, 0, numRecordsUpdate, partitionPaths.asJava,
|
||||
jsc, spark.sqlContext)
|
||||
|
||||
updateDF.write
|
||||
.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(HoodieWriteConfig.TABLE_NAME, "hoodie_test")
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key")
|
||||
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, "timestamp")
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr")
|
||||
.option(HoodieCompactionConfig.INLINE_COMPACT_PROP, "true")
|
||||
.option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, "1")
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
// Expect 2 new commits since meta bootstrap - delta commit and compaction commit (due to inline compaction)
|
||||
assertEquals(2, HoodieDataSourceHelpers.listCommitsSince(fs, basePath, commitInstantTime1).size())
|
||||
|
||||
// Read table after upsert and verify count. Since we have inline compaction enabled the RO view will have
|
||||
// the updated rows.
|
||||
val hoodieROViewDF2 = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY,
|
||||
DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL)
|
||||
.load(basePath + "/*")
|
||||
assertEquals(numRecords, hoodieROViewDF2.count())
|
||||
assertEquals(numRecordsUpdate, hoodieROViewDF2.filter(s"timestamp == $updateTimestamp").count())
|
||||
}
|
||||
|
||||
@Test def testMetadataBootstrapMORPartitioned(): Unit = {
|
||||
val timestamp = Instant.now.toEpochMilli
|
||||
val numRecords = 100
|
||||
val partitionPaths = List("2020-04-01", "2020-04-02", "2020-04-03")
|
||||
val jsc = JavaSparkContext.fromSparkContext(spark.sparkContext)
|
||||
|
||||
val sourceDF = TestBootstrap.generateTestRawTripDataset(timestamp, 0, numRecords, partitionPaths.asJava, jsc,
|
||||
spark.sqlContext)
|
||||
|
||||
// Writing data for each partition instead of using partitionBy to avoid hive-style partitioning and hence
|
||||
// have partitioned columns stored in the data file
|
||||
partitionPaths.foreach(partitionPath => {
|
||||
sourceDF
|
||||
.filter(sourceDF("datestr").equalTo(lit(partitionPath)))
|
||||
.write
|
||||
.format("parquet")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(srcPath + "/" + partitionPath)
|
||||
})
|
||||
|
||||
// Perform bootstrap
|
||||
val bootstrapDF = spark.emptyDataFrame
|
||||
bootstrapDF.write
|
||||
.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(HoodieWriteConfig.TABLE_NAME, "hoodie_test")
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key")
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP, srcPath)
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS, "org.apache.hudi.keygen.SimpleKeyGenerator")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
val commitInstantTime1: String = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
assertEquals(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, commitInstantTime1)
|
||||
|
||||
// Read bootstrapped table and verify count
|
||||
val hoodieROViewDF1 = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY,
|
||||
DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL)
|
||||
.load(basePath + "/*")
|
||||
assertEquals(numRecords, hoodieROViewDF1.count())
|
||||
|
||||
// Perform upsert
|
||||
val updateTimestamp = Instant.now.toEpochMilli
|
||||
val numRecordsUpdate = 10
|
||||
val updateDF = TestBootstrap.generateTestRawTripDataset(updateTimestamp, 0, numRecordsUpdate,
|
||||
partitionPaths.asJava, jsc, spark.sqlContext)
|
||||
|
||||
updateDF.write
|
||||
.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(HoodieWriteConfig.TABLE_NAME, "hoodie_test")
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key")
|
||||
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, "timestamp")
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr")
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
// Expect 1 new commit since meta bootstrap - delta commit (because inline compaction is off)
|
||||
assertEquals(1, HoodieDataSourceHelpers.listCommitsSince(fs, basePath, commitInstantTime1).size())
|
||||
|
||||
// Read table after upsert and verify count. Since we have inline compaction off the RO view will have
|
||||
// no updated rows.
|
||||
val hoodieROViewDF2 = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY,
|
||||
DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL)
|
||||
.load(basePath + "/*")
|
||||
assertEquals(numRecords, hoodieROViewDF2.count())
|
||||
assertEquals(0, hoodieROViewDF2.filter(s"timestamp == $updateTimestamp").count())
|
||||
}
|
||||
|
||||
@Test def testFullBootstrapCOWPartitioned(): Unit = {
|
||||
val timestamp = Instant.now.toEpochMilli
|
||||
val numRecords = 100
|
||||
val partitionPaths = List("2020-04-01", "2020-04-02", "2020-04-03")
|
||||
val jsc = JavaSparkContext.fromSparkContext(spark.sparkContext)
|
||||
|
||||
val sourceDF = TestBootstrap.generateTestRawTripDataset(timestamp, 0, numRecords, partitionPaths.asJava, jsc,
|
||||
spark.sqlContext)
|
||||
|
||||
// Writing data for each partition instead of using partitionBy to avoid hive-style partitioning and hence
|
||||
// have partitioned columns stored in the data file
|
||||
partitionPaths.foreach(partitionPath => {
|
||||
sourceDF
|
||||
.filter(sourceDF("datestr").equalTo(lit(partitionPath)))
|
||||
.write
|
||||
.format("parquet")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(srcPath + "/" + partitionPath)
|
||||
})
|
||||
|
||||
// Perform bootstrap
|
||||
val bootstrapDF = spark.emptyDataFrame
|
||||
bootstrapDF.write
|
||||
.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(HoodieWriteConfig.TABLE_NAME, "hoodie_test")
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key")
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr")
|
||||
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, "timestamp")
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP, srcPath)
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS, classOf[SimpleKeyGenerator].getName)
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_MODE_SELECTOR, classOf[FullRecordBootstrapModeSelector].getName)
|
||||
.option(HoodieBootstrapConfig.FULL_BOOTSTRAP_INPUT_PROVIDER, classOf[SparkParquetBootstrapDataProvider].getName)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
val commitInstantTime1: String = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
assertEquals(HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS, commitInstantTime1)
|
||||
|
||||
// Read bootstrapped table and verify count
|
||||
val hoodieROViewDF1 = spark.read.format("hudi").load(basePath + "/*")
|
||||
assertEquals(numRecords, hoodieROViewDF1.count())
|
||||
|
||||
// Perform upsert
|
||||
val updateTimestamp = Instant.now.toEpochMilli
|
||||
val numRecordsUpdate = 10
|
||||
val updateDF = TestBootstrap.generateTestRawTripDataset(updateTimestamp, 0, numRecordsUpdate, partitionPaths.asJava,
|
||||
jsc, spark.sqlContext)
|
||||
|
||||
updateDF.write
|
||||
.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(HoodieWriteConfig.TABLE_NAME, "hoodie_test")
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key")
|
||||
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, "timestamp")
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr")
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
val commitInstantTime2: String = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
assertEquals(1, HoodieDataSourceHelpers.listCommitsSince(fs, basePath, commitInstantTime1).size())
|
||||
|
||||
// Read table after upsert and verify count
|
||||
val hoodieROViewDF2 = spark.read.format("hudi").load(basePath + "/*")
|
||||
assertEquals(numRecords, hoodieROViewDF2.count())
|
||||
assertEquals(numRecordsUpdate, hoodieROViewDF2.filter(s"timestamp == $updateTimestamp").count())
|
||||
|
||||
// incrementally pull only changes in the bootstrap commit, which would pull all the initial records written
|
||||
// during bootstrap
|
||||
val hoodieIncViewDF1 = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000")
|
||||
.option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.load(basePath)
|
||||
|
||||
assertEquals(numRecords, hoodieIncViewDF1.count())
|
||||
var countsPerCommit = hoodieIncViewDF1.groupBy("_hoodie_commit_time").count().collect();
|
||||
assertEquals(1, countsPerCommit.length)
|
||||
assertEquals(commitInstantTime1, countsPerCommit(0).get(0))
|
||||
|
||||
// incrementally pull only changes in the latest commit, which would pull only the updated records in the
|
||||
// latest commit
|
||||
val hoodieIncViewDF2 = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.load(basePath);
|
||||
|
||||
assertEquals(numRecordsUpdate, hoodieIncViewDF2.count())
|
||||
countsPerCommit = hoodieIncViewDF2.groupBy("_hoodie_commit_time").count().collect();
|
||||
assertEquals(1, countsPerCommit.length)
|
||||
assertEquals(commitInstantTime2, countsPerCommit(0).get(0))
|
||||
|
||||
// pull the latest commit within certain partitions
|
||||
val hoodieIncViewDF3 = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.option(DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY, "/2020-04-02/*")
|
||||
.load(basePath)
|
||||
|
||||
assertEquals(hoodieIncViewDF2.filter(col("_hoodie_partition_path").contains("2020-04-02")).count(),
|
||||
hoodieIncViewDF3.count())
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,391 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.functional
|
||||
|
||||
import org.apache.hudi.common.fs.FSUtils
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers}
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase
|
||||
import org.apache.log4j.LogManager
|
||||
import org.apache.spark.sql._
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
|
||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
|
||||
/**
|
||||
* Tests on Spark DataSource for MOR table.
|
||||
*/
|
||||
class TestMORDataSource extends HoodieClientTestBase {
|
||||
|
||||
var spark: SparkSession = null
|
||||
private val log = LogManager.getLogger(classOf[TestMORDataSource])
|
||||
val commonOpts = Map(
|
||||
"hoodie.insert.shuffle.parallelism" -> "4",
|
||||
"hoodie.upsert.shuffle.parallelism" -> "4",
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
|
||||
DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY -> "timestamp",
|
||||
HoodieWriteConfig.TABLE_NAME -> "hoodie_test"
|
||||
)
|
||||
|
||||
@BeforeEach override def setUp() {
|
||||
initPath()
|
||||
initSparkContexts()
|
||||
spark = sqlContext.sparkSession
|
||||
initTestDataGenerator()
|
||||
initFileSystem()
|
||||
}
|
||||
|
||||
@AfterEach override def tearDown() = {
|
||||
cleanupSparkContexts()
|
||||
cleanupTestDataGenerator()
|
||||
cleanupFileSystem()
|
||||
}
|
||||
|
||||
@Test def testMergeOnReadStorage() {
|
||||
|
||||
val fs = FSUtils.getFs(basePath, spark.sparkContext.hadoopConfiguration)
|
||||
// Bulk Insert Operation
|
||||
val records1 = recordsToStrings(dataGen.generateInserts("001", 100)).toList
|
||||
val inputDF1: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option("hoodie.compact.inline", "false") // else fails due to compaction & deltacommit instant times being same
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000"))
|
||||
|
||||
// Read RO View
|
||||
val hudiRODF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, hudiRODF1.count()) // still 100, since we only updated
|
||||
val insertCommitTime = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
val insertCommitTimes = hudiRODF1.select("_hoodie_commit_time").distinct().collectAsList().map(r => r.getString(0)).toList
|
||||
assertEquals(List(insertCommitTime), insertCommitTimes)
|
||||
|
||||
// Upsert operation
|
||||
val records2 = recordsToStrings(dataGen.generateUniqueUpdates("002", 100)).toList
|
||||
val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
// Read Snapshot query
|
||||
val updateCommitTime = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
val hudiSnapshotDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
val updateCommitTimes = hudiSnapshotDF2.select("_hoodie_commit_time").distinct().collectAsList().map(r => r.getString(0)).toList
|
||||
assertEquals(List(updateCommitTime), updateCommitTimes)
|
||||
}
|
||||
|
||||
@Test def testCount() {
|
||||
// First Operation:
|
||||
// Producing parquet files to three default partitions.
|
||||
// SNAPSHOT view on MOR table with parquet files only.
|
||||
val records1 = recordsToStrings(dataGen.generateInserts("001", 100)).toList
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option("hoodie.compact.inline", "false") // else fails due to compaction & deltacommit instant times being same
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000"))
|
||||
val hudiSnapshotDF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, hudiSnapshotDF1.count()) // still 100, since we only updated
|
||||
|
||||
// Second Operation:
|
||||
// Upsert the update to the default partitions with duplicate records. Produced a log file for each parquet.
|
||||
// SNAPSHOT view should read the log files only with the latest commit time.
|
||||
val records2 = recordsToStrings(dataGen.generateUniqueUpdates("002", 100)).toList
|
||||
val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, hudiSnapshotDF2.count()) // still 100, since we only updated
|
||||
val commit1Time = hudiSnapshotDF1.select("_hoodie_commit_time").head().get(0).toString
|
||||
val commit2Time = hudiSnapshotDF2.select("_hoodie_commit_time").head().get(0).toString
|
||||
assertEquals(hudiSnapshotDF2.select("_hoodie_commit_time").distinct().count(), 1)
|
||||
assertTrue(commit2Time > commit1Time)
|
||||
assertEquals(100, hudiSnapshotDF2.join(hudiSnapshotDF1, Seq("_hoodie_record_key"), "left").count())
|
||||
|
||||
// Unmerge
|
||||
val hudiSnapshotSkipMergeDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.option(DataSourceReadOptions.REALTIME_MERGE_OPT_KEY, DataSourceReadOptions.REALTIME_SKIP_MERGE_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(200, hudiSnapshotSkipMergeDF2.count())
|
||||
assertEquals(100, hudiSnapshotSkipMergeDF2.select("_hoodie_record_key").distinct().count())
|
||||
assertEquals(200, hudiSnapshotSkipMergeDF2.join(hudiSnapshotDF2, Seq("_hoodie_record_key"), "left").count())
|
||||
|
||||
// Test Read Optimized Query on MOR table
|
||||
val hudiRODF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, hudiRODF2.count())
|
||||
|
||||
// Third Operation:
|
||||
// Upsert another update to the default partitions with 50 duplicate records. Produced the second log file for each parquet.
|
||||
// SNAPSHOT view should read the latest log files.
|
||||
val records3 = recordsToStrings(dataGen.generateUniqueUpdates("003", 50)).toList
|
||||
val inputDF3: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records3, 2))
|
||||
inputDF3.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF3 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
// still 100, because we only updated the existing records
|
||||
assertEquals(100, hudiSnapshotDF3.count())
|
||||
|
||||
// 50 from commit2, 50 from commit3
|
||||
assertEquals(hudiSnapshotDF3.select("_hoodie_commit_time").distinct().count(), 2)
|
||||
assertEquals(50, hudiSnapshotDF3.filter(col("_hoodie_commit_time") > commit2Time).count())
|
||||
assertEquals(50,
|
||||
hudiSnapshotDF3.join(hudiSnapshotDF2, Seq("_hoodie_record_key", "_hoodie_commit_time"), "inner").count())
|
||||
|
||||
// Fourth Operation:
|
||||
// Insert records to a new partition. Produced a new parquet file.
|
||||
// SNAPSHOT view should read the latest log files from the default partition and parquet from the new partition.
|
||||
val partitionPaths = new Array[String](1)
|
||||
partitionPaths.update(0, "2020/01/10")
|
||||
val newDataGen = new HoodieTestDataGenerator(partitionPaths)
|
||||
val records4 = recordsToStrings(newDataGen.generateInserts("004", 100)).toList
|
||||
val inputDF4: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records4, 2))
|
||||
inputDF4.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF4 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
// 200, because we insert 100 records to a new partition
|
||||
assertEquals(200, hudiSnapshotDF4.count())
|
||||
assertEquals(100,
|
||||
hudiSnapshotDF1.join(hudiSnapshotDF4, Seq("_hoodie_record_key"), "inner").count())
|
||||
|
||||
// Fifth Operation:
|
||||
// Upsert records to the new partition. Produced a newer version of parquet file.
|
||||
// SNAPSHOT view should read the latest log files from the default partition
|
||||
// and the latest parquet from the new partition.
|
||||
val records5 = recordsToStrings(newDataGen.generateUpdates("005", 100)).toList
|
||||
val inputDF5: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records5, 2))
|
||||
inputDF5.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF5 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(200, hudiSnapshotDF5.count())
|
||||
}
|
||||
|
||||
@Test
|
||||
def testPayloadDelete() {
|
||||
// First Operation:
|
||||
// Producing parquet files to three default partitions.
|
||||
// SNAPSHOT view on MOR table with parquet files only.
|
||||
val records1 = recordsToStrings(dataGen.generateInserts("001", 100)).toList
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option("hoodie.compact.inline", "false") // else fails due to compaction & deltacommit instant times being same
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000"))
|
||||
val hudiSnapshotDF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, hudiSnapshotDF1.count()) // still 100, since we only updated
|
||||
|
||||
// Second Operation:
|
||||
// Upsert 50 delete records
|
||||
// Snopshot view should only read 50 records
|
||||
val records2 = recordsToStrings(dataGen.generateUniqueDeleteRecords("002", 50)).toList
|
||||
val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(50, hudiSnapshotDF2.count()) // 50 records were deleted
|
||||
assertEquals(hudiSnapshotDF2.select("_hoodie_commit_time").distinct().count(), 1)
|
||||
val commit1Time = hudiSnapshotDF1.select("_hoodie_commit_time").head().get(0).toString
|
||||
val commit2Time = hudiSnapshotDF2.select("_hoodie_commit_time").head().get(0).toString
|
||||
assertTrue(commit1Time.equals(commit2Time))
|
||||
assertEquals(50, hudiSnapshotDF2.join(hudiSnapshotDF1, Seq("_hoodie_record_key"), "left").count())
|
||||
|
||||
// unmerge query, skip the delete records
|
||||
val hudiSnapshotDF2Unmerge = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.option(DataSourceReadOptions.REALTIME_MERGE_OPT_KEY, DataSourceReadOptions.REALTIME_SKIP_MERGE_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, hudiSnapshotDF2Unmerge.count())
|
||||
|
||||
// Third Operation:
|
||||
// Upsert 50 delete records to delete the reset
|
||||
// Snopshot view should read 0 record
|
||||
val records3 = recordsToStrings(dataGen.generateUniqueDeleteRecords("003", 50)).toList
|
||||
val inputDF3: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records3, 2))
|
||||
inputDF3.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF3 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(0, hudiSnapshotDF3.count()) // 100 records were deleted, 0 record to load
|
||||
}
|
||||
|
||||
@Test
|
||||
def testPrunedFiltered() {
|
||||
// First Operation:
|
||||
// Producing parquet files to three default partitions.
|
||||
// SNAPSHOT view on MOR table with parquet files only.
|
||||
val records1 = recordsToStrings(dataGen.generateInserts("001", 100)).toList
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option("hoodie.compact.inline", "false") // else fails due to compaction & deltacommit instant times being same
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, hudiSnapshotDF1.count())
|
||||
// select nested columns with order different from the actual schema
|
||||
assertEquals("amount,currency,tip_history,_hoodie_commit_seqno",
|
||||
hudiSnapshotDF1
|
||||
.select("fare.amount", "fare.currency", "tip_history", "_hoodie_commit_seqno")
|
||||
.orderBy(desc("_hoodie_commit_seqno"))
|
||||
.columns.mkString(","))
|
||||
|
||||
// Second Operation:
|
||||
// Upsert 50 update records
|
||||
// Snopshot view should read 100 records
|
||||
val records2 = recordsToStrings(dataGen.generateUniqueUpdates("002", 50))
|
||||
.toList
|
||||
val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
|
||||
val commit1Time = hudiSnapshotDF1.select("_hoodie_commit_time").head().get(0).toString
|
||||
|
||||
// filter first commit and only read log records
|
||||
assertEquals(50, hudiSnapshotDF2.select("_hoodie_commit_seqno", "fare.amount", "fare.currency", "tip_history")
|
||||
.filter(col("_hoodie_commit_time") > commit1Time).count())
|
||||
|
||||
// select nested columns with order different from the actual schema
|
||||
assertEquals("amount,currency,tip_history,_hoodie_commit_seqno",
|
||||
hudiSnapshotDF2
|
||||
.select("fare.amount", "fare.currency", "tip_history", "_hoodie_commit_seqno")
|
||||
.orderBy(desc("_hoodie_commit_seqno"))
|
||||
.columns.mkString(","))
|
||||
|
||||
// Correctly loading type
|
||||
val sampleRow = hudiSnapshotDF2
|
||||
.select("begin_lat", "current_date", "fare.currency", "tip_history", "nation")
|
||||
.orderBy(desc("_hoodie_commit_time"))
|
||||
.head()
|
||||
assertEquals(sampleRow.getDouble(0), sampleRow.get(0))
|
||||
assertEquals(sampleRow.getLong(1), sampleRow.get(1))
|
||||
assertEquals(sampleRow.getString(2), sampleRow.get(2))
|
||||
assertEquals(sampleRow.getSeq(3), sampleRow.get(3))
|
||||
assertEquals(sampleRow.getStruct(4), sampleRow.get(4))
|
||||
|
||||
// make sure show() work
|
||||
hudiSnapshotDF1.show(1)
|
||||
hudiSnapshotDF2.show(1)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testVectorizedReader() {
|
||||
spark.conf.set("spark.sql.parquet.enableVectorizedReader", true)
|
||||
assertTrue(spark.conf.get("spark.sql.parquet.enableVectorizedReader").toBoolean)
|
||||
// Vectorized Reader will only be triggered with AtomicType schema,
|
||||
// which is not null, UDTs, arrays, structs, and maps.
|
||||
val schema = HoodieTestDataGenerator.SHORT_TRIP_SCHEMA
|
||||
val records1 = recordsToStrings(dataGen.generateInsertsAsPerSchema("001", 100, schema)).toList
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option("hoodie.compact.inline", "false") // else fails due to compaction & deltacommit instant times being same
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, hudiSnapshotDF1.count())
|
||||
|
||||
val records2 = recordsToStrings(dataGen.generateUniqueUpdatesAsPerSchema("002", 50, schema))
|
||||
.toList
|
||||
val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, hudiSnapshotDF2.count())
|
||||
|
||||
// loading correct type
|
||||
val sampleRow = hudiSnapshotDF2
|
||||
.select("fare", "driver", "_hoodie_is_deleted")
|
||||
.head()
|
||||
assertEquals(sampleRow.getDouble(0), sampleRow.get(0))
|
||||
assertEquals(sampleRow.getString(1), sampleRow.get(1))
|
||||
assertEquals(sampleRow.getBoolean(2), sampleRow.get(2))
|
||||
|
||||
// test show()
|
||||
hudiSnapshotDF1.show(1)
|
||||
hudiSnapshotDF2.show(1)
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,180 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.functional
|
||||
|
||||
import org.apache.hadoop.fs.{FileSystem, Path}
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.exception.TableNotFoundException
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase
|
||||
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers}
|
||||
import org.apache.log4j.LogManager
|
||||
import org.apache.spark.sql._
|
||||
import org.apache.spark.sql.streaming.{OutputMode, Trigger}
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
|
||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
import scala.concurrent.ExecutionContext.Implicits.global
|
||||
import scala.concurrent.duration.Duration
|
||||
import scala.concurrent.{Await, Future}
|
||||
|
||||
/**
|
||||
* Basic tests on the spark datasource for structured streaming sink
|
||||
*/
|
||||
class TestStructuredStreaming extends HoodieClientTestBase {
|
||||
private val log = LogManager.getLogger(getClass)
|
||||
var spark: SparkSession = null
|
||||
val commonOpts = Map(
|
||||
"hoodie.insert.shuffle.parallelism" -> "4",
|
||||
"hoodie.upsert.shuffle.parallelism" -> "4",
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
|
||||
DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY -> "timestamp",
|
||||
HoodieWriteConfig.TABLE_NAME -> "hoodie_test"
|
||||
)
|
||||
|
||||
@BeforeEach override def setUp() {
|
||||
initPath()
|
||||
initSparkContexts()
|
||||
spark = sqlContext.sparkSession
|
||||
initTestDataGenerator()
|
||||
initFileSystem()
|
||||
}
|
||||
|
||||
@AfterEach override def tearDown() = {
|
||||
cleanupSparkContexts()
|
||||
cleanupTestDataGenerator()
|
||||
cleanupFileSystem()
|
||||
}
|
||||
|
||||
@Test
|
||||
def testStructuredStreaming(): Unit = {
|
||||
fs.delete(new Path(basePath), true)
|
||||
val sourcePath = basePath + "/source"
|
||||
val destPath = basePath + "/dest"
|
||||
fs.mkdirs(new Path(sourcePath))
|
||||
|
||||
// First chunk of data
|
||||
val records1 = recordsToStrings(dataGen.generateInserts("000", 100)).toList
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
|
||||
// Second chunk of data
|
||||
val records2 = recordsToStrings(dataGen.generateUpdates("001", 100)).toList
|
||||
val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
val uniqueKeyCnt = inputDF2.select("_row_key").distinct().count()
|
||||
|
||||
// define the source of streaming
|
||||
val streamingInput =
|
||||
spark.readStream
|
||||
.schema(inputDF1.schema)
|
||||
.json(sourcePath)
|
||||
|
||||
val f1 = Future {
|
||||
println("streaming starting")
|
||||
//'writeStream' can be called only on streaming Dataset/DataFrame
|
||||
streamingInput
|
||||
.writeStream
|
||||
.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.trigger(Trigger.ProcessingTime(100))
|
||||
.option("checkpointLocation", basePath + "/checkpoint")
|
||||
.outputMode(OutputMode.Append)
|
||||
.start(destPath)
|
||||
.awaitTermination(10000)
|
||||
println("streaming ends")
|
||||
}
|
||||
|
||||
val f2 = Future {
|
||||
inputDF1.coalesce(1).write.mode(SaveMode.Append).json(sourcePath)
|
||||
// wait for spark streaming to process one microbatch
|
||||
val currNumCommits = waitTillAtleastNCommits(fs, destPath, 1, 120, 5)
|
||||
assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, destPath, "000"))
|
||||
val commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, destPath)
|
||||
// Read RO View
|
||||
val hoodieROViewDF1 = spark.read.format("org.apache.hudi")
|
||||
.load(destPath + "/*/*/*/*")
|
||||
assert(hoodieROViewDF1.count() == 100)
|
||||
|
||||
inputDF2.coalesce(1).write.mode(SaveMode.Append).json(sourcePath)
|
||||
// wait for spark streaming to process one microbatch
|
||||
waitTillAtleastNCommits(fs, destPath, currNumCommits + 1, 120, 5)
|
||||
val commitInstantTime2 = HoodieDataSourceHelpers.latestCommit(fs, destPath)
|
||||
assertEquals(2, HoodieDataSourceHelpers.listCommitsSince(fs, destPath, "000").size())
|
||||
// Read RO View
|
||||
val hoodieROViewDF2 = spark.read.format("org.apache.hudi")
|
||||
.load(destPath + "/*/*/*/*")
|
||||
assertEquals(100, hoodieROViewDF2.count()) // still 100, since we only updated
|
||||
|
||||
|
||||
// Read Incremental View
|
||||
// we have 2 commits, try pulling the first commit (which is not the latest)
|
||||
val firstCommit = HoodieDataSourceHelpers.listCommitsSince(fs, destPath, "000").get(0)
|
||||
val hoodieIncViewDF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000")
|
||||
.option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, firstCommit)
|
||||
.load(destPath)
|
||||
assertEquals(100, hoodieIncViewDF1.count())
|
||||
// 100 initial inserts must be pulled
|
||||
var countsPerCommit = hoodieIncViewDF1.groupBy("_hoodie_commit_time").count().collect()
|
||||
assertEquals(1, countsPerCommit.length)
|
||||
assertEquals(firstCommit, countsPerCommit(0).get(0))
|
||||
|
||||
// pull the latest commit
|
||||
val hoodieIncViewDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.load(destPath)
|
||||
|
||||
assertEquals(uniqueKeyCnt, hoodieIncViewDF2.count()) // 100 records must be pulled
|
||||
countsPerCommit = hoodieIncViewDF2.groupBy("_hoodie_commit_time").count().collect()
|
||||
assertEquals(1, countsPerCommit.length)
|
||||
assertEquals(commitInstantTime2, countsPerCommit(0).get(0))
|
||||
}
|
||||
Await.result(Future.sequence(Seq(f1, f2)), Duration.Inf)
|
||||
}
|
||||
|
||||
@throws[InterruptedException]
|
||||
private def waitTillAtleastNCommits(fs: FileSystem, tablePath: String,
|
||||
numCommits: Int, timeoutSecs: Int, sleepSecsAfterEachRun: Int) = {
|
||||
val beginTime = System.currentTimeMillis
|
||||
var currTime = beginTime
|
||||
val timeoutMsecs = timeoutSecs * 1000
|
||||
var numInstants = 0
|
||||
var success = false
|
||||
while ({!success && (currTime - beginTime) < timeoutMsecs}) try {
|
||||
val timeline = HoodieDataSourceHelpers.allCompletedCommitsCompactions(fs, tablePath)
|
||||
log.info("Timeline :" + timeline.getInstants.toArray)
|
||||
if (timeline.countInstants >= numCommits) {
|
||||
numInstants = timeline.countInstants
|
||||
success = true
|
||||
}
|
||||
val metaClient = new HoodieTableMetaClient(fs.getConf, tablePath, true)
|
||||
} catch {
|
||||
case te: TableNotFoundException =>
|
||||
log.info("Got table not found exception. Retrying")
|
||||
} finally {
|
||||
Thread.sleep(sleepSecsAfterEachRun * 1000)
|
||||
currTime = System.currentTimeMillis
|
||||
}
|
||||
if (!success) throw new IllegalStateException("Timed-out waiting for " + numCommits + " commits to appear in " + tablePath)
|
||||
numInstants
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user