HUDI-123 Rename code packages/constants to org.apache.hudi (#830)
- Rename com.uber.hoodie to org.apache.hudi - Flag to pass com.uber.hoodie Input formats for hoodie-sync - Works with HUDI demo. - Also tested for backwards compatibility with datasets built by com.uber.hoodie packages - Migration guide : https://cwiki.apache.org/confluence/display/HUDI/Migration+Guide+From+com.uber.hoodie+to+org.apache.hudi
This commit is contained in:
committed by
vinoth chandar
parent
722b6be04a
commit
a4f9d7575f
46
hudi-spark/src/test/java/DataSourceTestUtils.java
Normal file
46
hudi-spark/src/test/java/DataSourceTestUtils.java
Normal file
@@ -0,0 +1,46 @@
|
||||
/*
|
||||
* 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.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hudi.common.TestRawTripPayload;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
/**
|
||||
* Test utils for data source tests.
|
||||
*/
|
||||
public class DataSourceTestUtils {
|
||||
|
||||
public static Option<String> convertToString(HoodieRecord record) {
|
||||
try {
|
||||
String str = ((TestRawTripPayload) record.getData()).getJsonData();
|
||||
str = "{" + str.substring(str.indexOf("\"timestamp\":"));
|
||||
return Option.of(str.replaceAll("}",
|
||||
", \"partition\": \"" + record.getPartitionPath() + "\"}"));
|
||||
} catch (IOException e) {
|
||||
return Option.empty();
|
||||
}
|
||||
}
|
||||
|
||||
public static List<String> convertToStringList(List<HoodieRecord> records) {
|
||||
return records.stream().map(hr -> convertToString(hr)).filter(os -> os.isPresent())
|
||||
.map(os -> os.get()).collect(Collectors.toList());
|
||||
}
|
||||
}
|
||||
231
hudi-spark/src/test/java/HoodieJavaApp.java
Normal file
231
hudi-spark/src/test/java/HoodieJavaApp.java
Normal file
@@ -0,0 +1,231 @@
|
||||
/*
|
||||
* 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 com.beust.jcommander.JCommander;
|
||||
import com.beust.jcommander.Parameter;
|
||||
import java.util.List;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hudi.DataSourceReadOptions;
|
||||
import org.apache.hudi.DataSourceWriteOptions;
|
||||
import org.apache.hudi.HoodieDataSourceHelpers;
|
||||
import org.apache.hudi.NonpartitionedKeyGenerator;
|
||||
import org.apache.hudi.SimpleKeyGenerator;
|
||||
import org.apache.hudi.common.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.hive.MultiPartKeysValueExtractor;
|
||||
import org.apache.hudi.hive.NonPartitionedExtractor;
|
||||
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;
|
||||
|
||||
/**
|
||||
* Sample program that writes & reads hoodie datasets 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 Logger logger = LogManager.getLogger(HoodieJavaApp.class);
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
HoodieJavaApp cli = new HoodieJavaApp();
|
||||
JCommander cmd = new JCommander(cli, 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());
|
||||
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();
|
||||
}
|
||||
|
||||
/**
|
||||
* Commit with only inserts
|
||||
*/
|
||||
// Generate some input..
|
||||
List<String> records1 = DataSourceTestUtils.convertToStringList(
|
||||
dataGen.generateInserts("001"/* ignore */, 100));
|
||||
Dataset<Row> inputDF1 = spark.read().json(jssc.parallelize(records1, 2));
|
||||
|
||||
// Save as hoodie dataset (copy on write)
|
||||
DataFrameWriter<Row> writer = inputDF1.write().format("org.apache.hudi") // specify the hoodie source
|
||||
.option("hoodie.insert.shuffle.parallelism",
|
||||
"2") // any hoodie client config can be passed like this
|
||||
.option("hoodie.upsert.shuffle.parallelism",
|
||||
"2") // full list in HoodieWriteConfig & its package
|
||||
.option(DataSourceWriteOptions.STORAGE_TYPE_OPT_KEY(), tableType) // Hoodie Table Type
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY(),
|
||||
DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL()) // insert
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(),
|
||||
"_row_key") // This is the record key
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(),
|
||||
"partition") // this is the partition to place it into
|
||||
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY(),
|
||||
"timestamp") // use to combine duplicate records in input/with disk val
|
||||
.option(HoodieWriteConfig.TABLE_NAME, tableName) // Used by hive sync and queries
|
||||
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(),
|
||||
nonPartitionedTable ? NonpartitionedKeyGenerator.class.getCanonicalName() :
|
||||
SimpleKeyGenerator.class.getCanonicalName()) // Add Key Extractor
|
||||
.mode(
|
||||
SaveMode.Overwrite); // This will remove any existing data at path below, and create a
|
||||
|
||||
updateHiveSyncConfig(writer);
|
||||
// new dataset if needed
|
||||
writer.save(tablePath); // ultimately where the dataset will be placed
|
||||
String commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, tablePath);
|
||||
logger.info("First commit at instant time :" + commitInstantTime1);
|
||||
|
||||
/**
|
||||
* Commit that updates records
|
||||
*/
|
||||
List<String> records2 = DataSourceTestUtils.convertToStringList(
|
||||
dataGen.generateUpdates("002"/* ignore */, 100));
|
||||
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.STORAGE_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(HoodieWriteConfig.TABLE_NAME, tableName).mode(SaveMode.Append);
|
||||
|
||||
updateHiveSyncConfig(writer);
|
||||
writer.save(tablePath);
|
||||
String commitInstantTime2 = HoodieDataSourceHelpers.latestCommit(fs, tablePath);
|
||||
logger.info("Second commit at instant time :" + commitInstantTime1);
|
||||
|
||||
/**
|
||||
* Read & do some queries
|
||||
*/
|
||||
Dataset<Row> hoodieROViewDF = spark.read().format("org.apache.hudi")
|
||||
// pass any path glob, can include hoodie & non-hoodie
|
||||
// datasets
|
||||
.load(tablePath + (nonPartitionedTable ? "/*" : "/*/*/*/*"));
|
||||
hoodieROViewDF.registerTempTable("hoodie_ro");
|
||||
spark.sql("describe hoodie_ro").show();
|
||||
// all trips whose fare was greater than 2.
|
||||
spark.sql("select fare, begin_lon, begin_lat, timestamp from hoodie_ro where fare > 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> hoodieIncViewDF = spark.read().format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.VIEW_TYPE_OPT_KEY(),
|
||||
DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL())
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY(),
|
||||
commitInstantTime1) // Only changes in write 2 above
|
||||
.load(
|
||||
tablePath); // For incremental view, pass in the root/base path of dataset
|
||||
|
||||
logger.info("You will only see records from : " + commitInstantTime2);
|
||||
hoodieIncViewDF.groupBy(hoodieIncViewDF.col("_hoodie_commit_time")).count().show();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Setup configs for syncing to hive
|
||||
* @param writer
|
||||
* @return
|
||||
*/
|
||||
private DataFrameWriter<Row> updateHiveSyncConfig(DataFrameWriter<Row> writer) {
|
||||
if (enableHiveSync) {
|
||||
logger.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;
|
||||
}
|
||||
}
|
||||
280
hudi-spark/src/test/java/HoodieJavaStreamingApp.java
Normal file
280
hudi-spark/src/test/java/HoodieJavaStreamingApp.java
Normal file
@@ -0,0 +1,280 @@
|
||||
/*
|
||||
* 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 com.beust.jcommander.JCommander;
|
||||
import com.beust.jcommander.Parameter;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.Future;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
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.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.hive.MultiPartKeysValueExtractor;
|
||||
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.ProcessingTime;
|
||||
|
||||
/**
|
||||
* Sample program that writes & reads hoodie datasets via the Spark datasource streaming
|
||||
*/
|
||||
public class HoodieJavaStreamingApp {
|
||||
|
||||
@Parameter(names = {"--table-path", "-p"}, description = "path for Hoodie sample table")
|
||||
private String tablePath = "file:///tmp/hoodie/streaming/sample-table";
|
||||
|
||||
@Parameter(names = {"--streaming-source-path", "-ssp"}, description = "path for streaming source file folder")
|
||||
private String streamingSourcePath = "file:///tmp/hoodie/streaming/source";
|
||||
|
||||
@Parameter(names = {"--streaming-checkpointing-path", "-scp"},
|
||||
description = "path for streaming checking pointing folder")
|
||||
private String streamingCheckpointingPath = "file:///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 Logger logger = LogManager.getLogger(HoodieJavaStreamingApp.class);
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
HoodieJavaStreamingApp cli = new HoodieJavaStreamingApp();
|
||||
JCommander cmd = new JCommander(cli, args);
|
||||
|
||||
if (cli.help) {
|
||||
cmd.usage();
|
||||
System.exit(1);
|
||||
}
|
||||
cli.run();
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @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 = DataSourceTestUtils.convertToStringList(
|
||||
dataGen.generateInserts("001", 100));
|
||||
Dataset<Row> inputDF1 = spark.read().json(jssc.parallelize(records1, 2));
|
||||
|
||||
List<String> records2 = DataSourceTestUtils.convertToStringList(
|
||||
dataGen.generateUpdates("002", 100));
|
||||
|
||||
Dataset<Row> inputDF2 = spark.read().json(jssc.parallelize(records2, 2));
|
||||
|
||||
// setup the input for streaming
|
||||
Dataset<Row> streamingInput = spark.readStream().schema(inputDF1.schema())
|
||||
.json(streamingSourcePath);
|
||||
|
||||
|
||||
// start streaming and showing
|
||||
ExecutorService executor = Executors.newFixedThreadPool(2);
|
||||
|
||||
// thread for spark strucutured streaming
|
||||
Future<Void> streamFuture = executor.submit(new Callable<Void>() {
|
||||
public Void call() throws Exception {
|
||||
logger.info("===== Streaming Starting =====");
|
||||
stream(streamingInput);
|
||||
logger.info("===== Streaming Ends =====");
|
||||
return null;
|
||||
}
|
||||
});
|
||||
|
||||
// thread for adding data to the streaming source and showing results over time
|
||||
Future<Void> showFuture = executor.submit(new Callable<Void>() {
|
||||
public Void call() throws Exception {
|
||||
logger.info("===== Showing Starting =====");
|
||||
show(spark, fs, inputDF1, inputDF2);
|
||||
logger.info("===== Showing Ends =====");
|
||||
return null;
|
||||
}
|
||||
});
|
||||
|
||||
// let the threads run
|
||||
streamFuture.get();
|
||||
showFuture.get();
|
||||
|
||||
executor.shutdown();
|
||||
}
|
||||
|
||||
/**
|
||||
* Adding data to the streaming source and showing results over time
|
||||
* @param spark
|
||||
* @param fs
|
||||
* @param inputDF1
|
||||
* @param inputDF2
|
||||
* @throws Exception
|
||||
*/
|
||||
public void show(SparkSession spark,
|
||||
FileSystem fs,
|
||||
Dataset<Row> inputDF1,
|
||||
Dataset<Row> inputDF2) throws Exception {
|
||||
inputDF1.write().mode(SaveMode.Append).json(streamingSourcePath);
|
||||
// wait for spark streaming to process one microbatch
|
||||
Thread.sleep(3000);
|
||||
String commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, tablePath);
|
||||
logger.info("First commit at instant time :" + commitInstantTime1);
|
||||
|
||||
inputDF2.write().mode(SaveMode.Append).json(streamingSourcePath);
|
||||
// wait for spark streaming to process one microbatch
|
||||
Thread.sleep(3000);
|
||||
String commitInstantTime2 = HoodieDataSourceHelpers.latestCommit(fs, tablePath);
|
||||
logger.info("Second commit at instant time :" + commitInstantTime1);
|
||||
|
||||
/**
|
||||
* Read & do some queries
|
||||
*/
|
||||
Dataset<Row> hoodieROViewDF = spark.read().format("org.apache.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 was greater than 2.
|
||||
spark.sql("select fare, begin_lon, begin_lat, timestamp from hoodie_ro where fare > 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> hoodieIncViewDF = spark.read().format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.VIEW_TYPE_OPT_KEY(),
|
||||
DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL())
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY(),
|
||||
commitInstantTime1) // Only changes in write 2 above
|
||||
.load(
|
||||
tablePath); // For incremental view, pass in the root/base path of dataset
|
||||
|
||||
logger.info("You will only see records from : " + commitInstantTime2);
|
||||
hoodieIncViewDF.groupBy(hoodieIncViewDF.col("_hoodie_commit_time")).count().show();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Hoodie spark streaming job
|
||||
* @param streamingInput
|
||||
* @throws Exception
|
||||
*/
|
||||
public void stream(Dataset<Row> streamingInput) throws Exception {
|
||||
|
||||
DataStreamWriter<Row> writer = streamingInput
|
||||
.writeStream()
|
||||
.format("org.apache.hudi")
|
||||
.option("hoodie.insert.shuffle.parallelism", "2")
|
||||
.option("hoodie.upsert.shuffle.parallelism", "2")
|
||||
.option(DataSourceWriteOptions.STORAGE_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(HoodieWriteConfig.TABLE_NAME, tableName)
|
||||
.option("checkpointLocation", streamingCheckpointingPath)
|
||||
.outputMode(OutputMode.Append());
|
||||
|
||||
updateHiveSyncConfig(writer);
|
||||
writer
|
||||
.trigger(new ProcessingTime(500))
|
||||
.start(tablePath)
|
||||
.awaitTermination(streamingDurationInMs);
|
||||
}
|
||||
|
||||
/**
|
||||
* Setup configs for syncing to hive
|
||||
* @param writer
|
||||
* @return
|
||||
*/
|
||||
private DataStreamWriter<Row> updateHiveSyncConfig(DataStreamWriter<Row> writer) {
|
||||
if (enableHiveSync) {
|
||||
logger.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,23 @@
|
||||
###
|
||||
# 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, A1
|
||||
# A1 is set to be a ConsoleAppender.
|
||||
log4j.appender.A1=org.apache.log4j.ConsoleAppender
|
||||
# A1 uses PatternLayout.
|
||||
log4j.appender.A1.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.A1.layout.ConversionPattern=[%-5p] %d %c %x - %m%n
|
||||
29
hudi-spark/src/test/resources/log4j-surefire.properties
Normal file
29
hudi-spark/src/test/resources/log4j-surefire.properties
Normal file
@@ -0,0 +1,29 @@
|
||||
###
|
||||
# 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, A1
|
||||
log4j.category.com.uber=INFO
|
||||
log4j.category.com.uber.hoodie.common.utils=WARN
|
||||
log4j.category.com.uber.hoodie.io=WARN
|
||||
log4j.category.com.uber.hoodie.common=WARN
|
||||
log4j.category.com.uber.hoodie.table.log=WARN
|
||||
log4j.category.org.apache.parquet.hadoop=WARN
|
||||
# A1 is set to be a ConsoleAppender.
|
||||
log4j.appender.A1=org.apache.log4j.ConsoleAppender
|
||||
# A1 uses PatternLayout.
|
||||
log4j.appender.A1.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.A1.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
|
||||
181
hudi-spark/src/test/scala/DataSourceDefaultsTest.scala
Normal file
181
hudi-spark/src/test/scala/DataSourceDefaultsTest.scala
Normal file
@@ -0,0 +1,181 @@
|
||||
/*
|
||||
* 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.avro.generic.GenericRecord
|
||||
import org.apache.hudi.common.util.{Option, SchemaTestUtil, TypedProperties}
|
||||
import org.apache.hudi.exception.HoodieException
|
||||
import org.apache.hudi.{ComplexKeyGenerator, DataSourceWriteOptions, EmptyHoodieRecordPayload, OverwriteWithLatestAvroPayload, SimpleKeyGenerator}
|
||||
import org.junit.Assert._
|
||||
import org.junit.{Before, Test}
|
||||
import org.scalatest.junit.AssertionsForJUnit
|
||||
|
||||
/**
|
||||
* Tests on the default key generator, payload classes.
|
||||
*/
|
||||
class DataSourceDefaultsTest extends AssertionsForJUnit {
|
||||
|
||||
val schema = SchemaTestUtil.getComplexEvolvedSchema
|
||||
var baseRecord: GenericRecord = null
|
||||
|
||||
@Before def initialize(): Unit = {
|
||||
baseRecord = SchemaTestUtil
|
||||
.generateAvroRecordFromJson(schema, 1, "001", "f1")
|
||||
}
|
||||
|
||||
|
||||
private def getKeyConfig(recordKeyFieldName: String, partitionPathField: String): TypedProperties = {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, recordKeyFieldName)
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, partitionPathField)
|
||||
props
|
||||
}
|
||||
|
||||
@Test def testSimpleKeyGenerator() = {
|
||||
// top level, valid fields
|
||||
val hk1 = new SimpleKeyGenerator(getKeyConfig("field1", "name")).getKey(baseRecord)
|
||||
assertEquals("field1", hk1.getRecordKey)
|
||||
assertEquals("name1", hk1.getPartitionPath)
|
||||
|
||||
// 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
|
||||
}
|
||||
};
|
||||
|
||||
// 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
|
||||
}
|
||||
};
|
||||
|
||||
// nested field as record key and partition path
|
||||
val hk2 = new SimpleKeyGenerator(getKeyConfig("testNestedRecord.userId", "testNestedRecord.isAdmin"))
|
||||
.getKey(baseRecord)
|
||||
assertEquals("UserId1@001", hk2.getRecordKey)
|
||||
assertEquals("false", hk2.getPartitionPath)
|
||||
|
||||
// Nested record key not found
|
||||
try {
|
||||
new SimpleKeyGenerator(getKeyConfig("testNestedRecord.NotThere", "testNestedRecord.isAdmin"))
|
||||
.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"))
|
||||
.getKey(baseRecord);
|
||||
assertEquals("default", hk3.getPartitionPath)
|
||||
}
|
||||
|
||||
@Test def testComplexKeyGenerator() = {
|
||||
// top level, valid fields
|
||||
val hk1 = new ComplexKeyGenerator(getKeyConfig("field1,name", "field1,name")).getKey(baseRecord)
|
||||
assertEquals("field1:field1,name:name1", hk1.getRecordKey)
|
||||
assertEquals("field1/name1", hk1.getPartitionPath)
|
||||
|
||||
// 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
|
||||
}
|
||||
};
|
||||
|
||||
// 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
|
||||
}
|
||||
};
|
||||
|
||||
// nested field as record key and partition path
|
||||
val hk2 = new ComplexKeyGenerator(getKeyConfig("testNestedRecord.userId,testNestedRecord.isAdmin", "testNestedRecord.userId,testNestedRecord.isAdmin"))
|
||||
.getKey(baseRecord)
|
||||
assertEquals("testNestedRecord.userId:UserId1@001,testNestedRecord.isAdmin:false", hk2.getRecordKey)
|
||||
assertEquals("UserId1@001/false", hk2.getPartitionPath)
|
||||
|
||||
// Nested record key not found
|
||||
try {
|
||||
new ComplexKeyGenerator(getKeyConfig("testNestedRecord.NotThere", "testNestedRecord.isAdmin"))
|
||||
.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 ComplexKeyGenerator(getKeyConfig("testNestedRecord.userId", "testNestedRecord.notThere"))
|
||||
.getKey(baseRecord);
|
||||
assertEquals("default", hk3.getPartitionPath)
|
||||
}
|
||||
|
||||
@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)
|
||||
}
|
||||
}
|
||||
278
hudi-spark/src/test/scala/DataSourceTest.scala
Normal file
278
hudi-spark/src/test/scala/DataSourceTest.scala
Normal file
@@ -0,0 +1,278 @@
|
||||
/*
|
||||
* 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.{FileSystem, Path}
|
||||
import org.apache.hudi.common.HoodieTestDataGenerator
|
||||
import org.apache.hudi.common.util.FSUtils
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers}
|
||||
import org.apache.spark.sql._
|
||||
import org.apache.spark.sql.streaming.{OutputMode, ProcessingTime}
|
||||
import org.junit.Assert._
|
||||
import org.junit.rules.TemporaryFolder
|
||||
import org.junit.{Before, Test}
|
||||
import org.scalatest.junit.AssertionsForJUnit
|
||||
|
||||
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
|
||||
*/
|
||||
class DataSourceTest extends AssertionsForJUnit {
|
||||
|
||||
var spark: SparkSession = null
|
||||
var dataGen: HoodieTestDataGenerator = 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"
|
||||
)
|
||||
var basePath: String = null
|
||||
var fs: FileSystem = null
|
||||
|
||||
@Before def initialize() {
|
||||
spark = SparkSession.builder
|
||||
.appName("Hoodie Datasource test")
|
||||
.master("local[2]")
|
||||
.config("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
|
||||
.getOrCreate
|
||||
dataGen = new HoodieTestDataGenerator()
|
||||
val folder = new TemporaryFolder
|
||||
folder.create
|
||||
basePath = folder.getRoot.getAbsolutePath
|
||||
fs = FSUtils.getFs(basePath, spark.sparkContext.hadoopConfiguration)
|
||||
}
|
||||
|
||||
@Test def testCopyOnWriteStorage() {
|
||||
// Insert Operation
|
||||
val records1 = DataSourceTestUtils.convertToStringList(dataGen.generateInserts("000", 100)).toList
|
||||
val inputDF1: Dataset[Row] = 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: String = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
|
||||
// Read RO View
|
||||
val hoodieROViewDF1 = spark.read.format("org.apache.hudi")
|
||||
.load(basePath + "/*/*/*/*");
|
||||
assertEquals(100, hoodieROViewDF1.count())
|
||||
|
||||
val records2 = DataSourceTestUtils.convertToStringList(dataGen.generateUpdates("001", 100)).toList
|
||||
val inputDF2: Dataset[Row] = 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: String = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
assertEquals(2, HoodieDataSourceHelpers.listCommitsSince(fs, basePath, "000").size())
|
||||
|
||||
// Read RO View
|
||||
val hoodieROViewDF2 = spark.read.format("org.apache.hudi")
|
||||
.load(basePath + "/*/*/*/*");
|
||||
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, basePath, "000").get(0);
|
||||
val hoodieIncViewDF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.VIEW_TYPE_OPT_KEY, DataSourceReadOptions.VIEW_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))
|
||||
|
||||
// pull the latest commit
|
||||
val hoodieIncViewDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.VIEW_TYPE_OPT_KEY, DataSourceReadOptions.VIEW_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))
|
||||
}
|
||||
|
||||
@Test def testMergeOnReadStorage() {
|
||||
// Bulk Insert Operation
|
||||
val records1 = DataSourceTestUtils.convertToStringList(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.STORAGE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_STORAGE_TYPE_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000"))
|
||||
|
||||
// Read RO View
|
||||
val hoodieROViewDF1 = spark.read.format("org.apache.hudi").load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, hoodieROViewDF1.count()) // still 100, since we only updated
|
||||
}
|
||||
|
||||
@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 = DataSourceTestUtils.convertToStringList(inserts1).toList
|
||||
val inputDF1: Dataset[Row] = 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 = DataSourceTestUtils
|
||||
.convertToStringList(inserts2Dup ++ inserts2New)
|
||||
.toList
|
||||
val inputDF2: Dataset[Row] = 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.VIEW_TYPE_OPT_KEY, DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.load(basePath)
|
||||
assertEquals(hoodieIncViewDF2.count(), insert2NewKeyCnt)
|
||||
}
|
||||
|
||||
//@Test (TODO: re-enable after fixing noisyness)
|
||||
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 = DataSourceTestUtils.convertToStringList(dataGen.generateInserts("000", 100)).toList
|
||||
val inputDF1: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
|
||||
// Second chunk of data
|
||||
val records2 = DataSourceTestUtils.convertToStringList(dataGen.generateUpdates("001", 100)).toList
|
||||
val inputDF2: Dataset[Row] = 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(new ProcessingTime(100))
|
||||
.option("checkpointLocation", basePath + "/checkpoint")
|
||||
.outputMode(OutputMode.Append)
|
||||
.start(destPath)
|
||||
.awaitTermination(10000)
|
||||
println("streaming ends")
|
||||
}
|
||||
|
||||
val f2 = Future {
|
||||
inputDF1.write.mode(SaveMode.Append).json(sourcePath)
|
||||
// wait for spark streaming to process one microbatch
|
||||
Thread.sleep(3000)
|
||||
assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, destPath, "000"))
|
||||
val commitInstantTime1: String = HoodieDataSourceHelpers.latestCommit(fs, destPath)
|
||||
// Read RO View
|
||||
val hoodieROViewDF1 = spark.read.format("org.apache.hudi")
|
||||
.load(destPath + "/*/*/*/*")
|
||||
assert(hoodieROViewDF1.count() == 100)
|
||||
|
||||
inputDF2.write.mode(SaveMode.Append).json(sourcePath)
|
||||
// wait for spark streaming to process one microbatch
|
||||
Thread.sleep(10000)
|
||||
val commitInstantTime2: String = 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.VIEW_TYPE_OPT_KEY, DataSourceReadOptions.VIEW_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.VIEW_TYPE_OPT_KEY, DataSourceReadOptions.VIEW_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)
|
||||
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,52 @@
|
||||
/*
|
||||
* 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.DataSourceWriteOptions._
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.exception.HoodieException
|
||||
import org.apache.spark.sql.{SaveMode, SparkSession}
|
||||
import org.scalatest.{FunSuite, Matchers}
|
||||
|
||||
class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
|
||||
test("Parameters With Write Defaults") {
|
||||
val originals = HoodieSparkSqlWriter.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, STORAGE_TYPE_OPT_KEY -> MOR_STORAGE_TYPE_OPT_VAL, rhsKey -> rhsVal)
|
||||
val modified = HoodieSparkSqlWriter.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 (STORAGE_TYPE_OPT_KEY, _) => matcher(STORAGE_TYPE_OPT_KEY, MOR_STORAGE_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()
|
||||
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"))
|
||||
}
|
||||
|
||||
}
|
||||
Reference in New Issue
Block a user