HUDI-70 : Making DeltaStreamer run in continuous mode with concurrent compaction
This commit is contained in:
committed by
Balaji Varadarajan
parent
3a210ef08e
commit
a0d7ab2384
@@ -18,6 +18,7 @@
|
||||
|
||||
package com.uber.hoodie;
|
||||
|
||||
import com.uber.hoodie.client.embedded.EmbeddedTimelineService;
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
@@ -38,6 +39,7 @@ import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.avro.Schema.Field;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
@@ -182,10 +184,10 @@ public class DataSourceUtils {
|
||||
@SuppressWarnings("unchecked")
|
||||
public static JavaRDD<HoodieRecord> dropDuplicates(JavaSparkContext jssc,
|
||||
JavaRDD<HoodieRecord> incomingHoodieRecords,
|
||||
HoodieWriteConfig writeConfig) throws Exception {
|
||||
HoodieWriteConfig writeConfig, Optional<EmbeddedTimelineService> timelineService) throws Exception {
|
||||
HoodieReadClient client = null;
|
||||
try {
|
||||
client = new HoodieReadClient<>(jssc, writeConfig);
|
||||
client = new HoodieReadClient<>(jssc, writeConfig, timelineService);
|
||||
return client.tagLocation(incomingHoodieRecords)
|
||||
.filter(r -> !((HoodieRecord<HoodieRecordPayload>) r).isCurrentLocationKnown());
|
||||
} catch (DatasetNotFoundException e) {
|
||||
@@ -202,12 +204,14 @@ public class DataSourceUtils {
|
||||
@SuppressWarnings("unchecked")
|
||||
public static JavaRDD<HoodieRecord> dropDuplicates(JavaSparkContext jssc,
|
||||
JavaRDD<HoodieRecord> incomingHoodieRecords,
|
||||
Map<String, String> parameters) throws Exception {
|
||||
Map<String, String> parameters,
|
||||
Optional<EmbeddedTimelineService> timelineService)
|
||||
throws Exception {
|
||||
HoodieWriteConfig writeConfig = HoodieWriteConfig
|
||||
.newBuilder()
|
||||
.withPath(parameters.get("path"))
|
||||
.withProps(parameters).build();
|
||||
return dropDuplicates(jssc, incomingHoodieRecords, writeConfig);
|
||||
return dropDuplicates(jssc, incomingHoodieRecords, writeConfig, timelineService);
|
||||
}
|
||||
|
||||
public static HiveSyncConfig buildHiveSyncConfig(TypedProperties props, String basePath) {
|
||||
|
||||
@@ -32,7 +32,7 @@ import org.apache.hadoop.hive.conf.HiveConf
|
||||
import org.apache.log4j.LogManager
|
||||
import org.apache.spark.api.java.JavaSparkContext
|
||||
import org.apache.spark.rdd.RDD
|
||||
import org.apache.spark.sql.{DataFrame, SaveMode, SQLContext}
|
||||
import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode}
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
import scala.collection.mutable.ListBuffer
|
||||
@@ -98,21 +98,6 @@ private[hoodie] object HoodieSparkSqlWriter {
|
||||
|
||||
val jsc = new JavaSparkContext(sparkContext)
|
||||
|
||||
val hoodieRecords =
|
||||
if (parameters(INSERT_DROP_DUPS_OPT_KEY).toBoolean) {
|
||||
DataSourceUtils.dropDuplicates(
|
||||
jsc,
|
||||
hoodieAllIncomingRecords,
|
||||
mapAsJavaMap(parameters))
|
||||
} else {
|
||||
hoodieAllIncomingRecords
|
||||
}
|
||||
|
||||
if (hoodieRecords.isEmpty()) {
|
||||
log.info("new batch has no new records, skipping...")
|
||||
return (true, None)
|
||||
}
|
||||
|
||||
val basePath = new Path(parameters("path"))
|
||||
val fs = basePath.getFileSystem(sparkContext.hadoopConfiguration)
|
||||
var exists = fs.exists(basePath)
|
||||
@@ -141,6 +126,22 @@ private[hoodie] object HoodieSparkSqlWriter {
|
||||
val client = DataSourceUtils.createHoodieClient(jsc, schema.toString, path.get, tblName.get,
|
||||
mapAsJavaMap(parameters)
|
||||
)
|
||||
|
||||
val hoodieRecords =
|
||||
if (parameters(INSERT_DROP_DUPS_OPT_KEY).toBoolean) {
|
||||
DataSourceUtils.dropDuplicates(
|
||||
jsc,
|
||||
hoodieAllIncomingRecords,
|
||||
mapAsJavaMap(parameters), client.getTimelineServer)
|
||||
} else {
|
||||
hoodieAllIncomingRecords
|
||||
}
|
||||
|
||||
if (hoodieRecords.isEmpty()) {
|
||||
log.info("new batch has no new records, skipping...")
|
||||
return (true, None)
|
||||
}
|
||||
|
||||
val commitTime = client.startCommit()
|
||||
|
||||
val writeStatuses = DataSourceUtils.doWriteOperation(client, hoodieRecords, commitTime, operation)
|
||||
|
||||
@@ -236,8 +236,9 @@ class DataSourceTest extends AssertionsForJUnit {
|
||||
|
||||
inputDF2.write.mode(SaveMode.Append).json(sourcePath)
|
||||
// wait for spark streaming to process one microbatch
|
||||
Thread.sleep(3000)
|
||||
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("com.uber.hoodie")
|
||||
|
||||
Reference in New Issue
Block a user