1
0

Ensure Hoodie works for non-partitioned Hive table

This commit is contained in:
Balaji Varadarajan
2018-12-05 11:42:38 -08:00
committed by vinoth chandar
parent 466ff73ffb
commit 30c5f8b7bd
13 changed files with 157 additions and 26 deletions

View File

@@ -21,10 +21,13 @@ import com.beust.jcommander.Parameter;
import com.uber.hoodie.DataSourceReadOptions;
import com.uber.hoodie.DataSourceWriteOptions;
import com.uber.hoodie.HoodieDataSourceHelpers;
import com.uber.hoodie.NonpartitionedKeyGenerator;
import com.uber.hoodie.SimpleKeyGenerator;
import com.uber.hoodie.common.HoodieTestDataGenerator;
import com.uber.hoodie.common.model.HoodieTableType;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.hive.MultiPartKeysValueExtractor;
import com.uber.hoodie.hive.NonPartitionedExtractor;
import java.util.List;
import org.apache.hadoop.fs.FileSystem;
import org.apache.log4j.LogManager;
@@ -68,6 +71,9 @@ public class HoodieJavaApp {
@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;
@@ -98,7 +104,13 @@ public class HoodieJavaApp {
FileSystem fs = FileSystem.get(jssc.hadoopConfiguration());
// Generator of some records to be loaded in.
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
HoodieTestDataGenerator dataGen = null;
if (nonPartitionedTable) {
// All data goes to base-path
dataGen = new HoodieTestDataGenerator(new String[]{""});
} else {
dataGen = new HoodieTestDataGenerator();
}
/**
* Commit with only inserts
@@ -124,6 +136,9 @@ public class HoodieJavaApp {
.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
@@ -145,7 +160,11 @@ public class HoodieJavaApp {
.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);
@@ -157,7 +176,7 @@ public class HoodieJavaApp {
Dataset<Row> hoodieROViewDF = spark.read().format("com.uber.hoodie")
// pass any path glob, can include hoodie & non-hoodie
// datasets
.load(tablePath + "/*/*/*/*");
.load(tablePath + (nonPartitionedTable ? "/*" : "/*/*/*/*"));
hoodieROViewDF.registerTempTable("hoodie_ro");
spark.sql("describe hoodie_ro").show();
// all trips whose fare was greater than 2.
@@ -195,7 +214,11 @@ public class HoodieJavaApp {
.option(DataSourceWriteOptions.HIVE_USER_OPT_KEY(), hiveUser)
.option(DataSourceWriteOptions.HIVE_PASS_OPT_KEY(), hivePass)
.option(DataSourceWriteOptions.HIVE_SYNC_ENABLED_OPT_KEY(), "true");
if (useMultiPartitionKeys) {
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());