1
0

[HUDI-328] Adding delete api to HoodieWriteClient (#1004)

[HUDI-328]  Adding delete api to HoodieWriteClient and Spark DataSource
This commit is contained in:
Sivabalan Narayanan
2019-11-22 15:05:25 -08:00
committed by Balaji Varadarajan
parent 7bc08cbfdc
commit c3355109b1
18 changed files with 818 additions and 172 deletions

View File

@@ -18,6 +18,7 @@
import com.beust.jcommander.JCommander;
import com.beust.jcommander.Parameter;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hudi.DataSourceReadOptions;
@@ -25,7 +26,9 @@ 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.HoodieClientTestUtils;
import org.apache.hudi.common.HoodieTestDataGenerator;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.hive.MultiPartKeysValueExtractor;
@@ -105,16 +108,18 @@ public class HoodieJavaApp {
HoodieTestDataGenerator dataGen = null;
if (nonPartitionedTable) {
// All data goes to base-path
dataGen = new HoodieTestDataGenerator(new String[] {""});
dataGen = new HoodieTestDataGenerator(new String[]{""});
} else {
dataGen = new HoodieTestDataGenerator();
}
List<HoodieRecord> recordsSoFar = new ArrayList<>();
/**
* Commit with only inserts
*/
// Generate some input..
List<String> records1 = DataSourceTestUtils.convertToStringList(dataGen.generateInserts("001"/* ignore */, 100));
recordsSoFar.addAll(dataGen.generateInserts("001"/* ignore */, 100));
List<String> records1 = DataSourceTestUtils.convertToStringList(recordsSoFar);
Dataset<Row> inputDF1 = spark.read().json(jssc.parallelize(records1, 2));
// Save as hoodie dataset (copy on write)
@@ -152,7 +157,9 @@ public class HoodieJavaApp {
/**
* Commit that updates records
*/
List<String> records2 = DataSourceTestUtils.convertToStringList(dataGen.generateUpdates("002"/* ignore */, 100));
List<HoodieRecord> recordsToBeUpdated = dataGen.generateUpdates("002"/* ignore */, 100);
recordsSoFar.addAll(recordsToBeUpdated);
List<String> records2 = DataSourceTestUtils.convertToStringList(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")
@@ -168,7 +175,31 @@ public class HoodieJavaApp {
updateHiveSyncConfig(writer);
writer.save(tablePath);
String commitInstantTime2 = HoodieDataSourceHelpers.latestCommit(fs, tablePath);
logger.info("Second commit at instant time :" + commitInstantTime1);
logger.info("Second commit at instant time :" + commitInstantTime2);
/**
* Commit that Deletes some records
*/
List<String> deletes = DataSourceTestUtils.convertKeysToStringList(
HoodieClientTestUtils
.getKeysToDelete(HoodieClientTestUtils.getHoodieKeys(recordsSoFar), 20));
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(DataSourceWriteOptions.STORAGE_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(HoodieWriteConfig.TABLE_NAME, tableName).mode(SaveMode.Append);
updateHiveSyncConfig(writer);
writer.save(tablePath);
String commitInstantTime3 = HoodieDataSourceHelpers.latestCommit(fs, tablePath);
logger.info("Third commit at instant time :" + commitInstantTime3);
/**
* Read & do some queries
@@ -200,9 +231,6 @@ public class HoodieJavaApp {
/**
* Setup configs for syncing to hive
*
* @param writer
* @return
*/
private DataFrameWriter<Row> updateHiveSyncConfig(DataFrameWriter<Row> writer) {
if (enableHiveSync) {