Adding compaction to HoodieClient example
This commit is contained in:
committed by
vinoth chandar
parent
169e3f66bb
commit
68723764ed
@@ -17,6 +17,7 @@
|
||||
import com.beust.jcommander.JCommander;
|
||||
import com.beust.jcommander.Parameter;
|
||||
import com.uber.hoodie.HoodieWriteClient;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.HoodieTestDataGenerator;
|
||||
import com.uber.hoodie.common.model.HoodieAvroPayload;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
@@ -28,6 +29,7 @@ import com.uber.hoodie.config.HoodieIndexConfig;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.index.HoodieIndex.IndexType;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
@@ -110,5 +112,15 @@ public class HoodieClientExample {
|
||||
records.addAll(dataGen.generateUpdates(newCommitTime, 100));
|
||||
writeRecords = jsc.<HoodieRecord>parallelize(records, 1);
|
||||
client.upsert(writeRecords, newCommitTime);
|
||||
|
||||
/**
|
||||
* Schedule a compaction and also perform compaction on a MOR dataset
|
||||
*/
|
||||
if (HoodieTableType.valueOf(tableType) == HoodieTableType.MERGE_ON_READ) {
|
||||
Optional<String> instant = client.scheduleCompaction(Optional.empty());
|
||||
JavaRDD<WriteStatus> writeStatues = client.compact(instant.get());
|
||||
client.commitCompaction(instant.get(), writeStatues, Optional.empty());
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user