Multi FS Support
- Reviving PR 191, to make FileSystem creation off actual path - Streamline all filesystem access to HoodieTableMetaClient - Hadoop Conf from Spark Context serialized & passed to executor code too - Pick up env vars prefixed with HOODIE_ENV_ into Configuration object - Cleanup usage of FSUtils.getFS, piggybacking off HoodieTableMetaClient.getFS - Adding s3a to supported schemes & support escaping "." in env vars - Tests use HoodieTestUtils.getDefaultHadoopConf
This commit is contained in:
committed by
vinoth chandar
parent
44839b88c6
commit
0cd186c899
@@ -53,18 +53,6 @@ import com.uber.hoodie.table.HoodieTable;
|
||||
import com.uber.hoodie.table.UserDefinedBulkInsertPartitioner;
|
||||
import com.uber.hoodie.table.WorkloadProfile;
|
||||
import com.uber.hoodie.table.WorkloadStat;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.Partitioner;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.PairFunction;
|
||||
import org.apache.spark.storage.StorageLevel;
|
||||
import scala.Option;
|
||||
import scala.Tuple2;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
@@ -76,6 +64,17 @@ import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.Partitioner;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.PairFunction;
|
||||
import org.apache.spark.storage.StorageLevel;
|
||||
import scala.Option;
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* Hoodie Write Client helps you build datasets on HDFS [insert()] and then perform efficient
|
||||
@@ -112,7 +111,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
*/
|
||||
public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig,
|
||||
boolean rollbackInFlight) {
|
||||
this.fs = FSUtils.getFs();
|
||||
this.fs = FSUtils.getFs(clientConfig.getBasePath(), jsc.hadoopConfiguration());
|
||||
this.jsc = jsc;
|
||||
this.config = clientConfig;
|
||||
this.index = HoodieIndex.createIndex(config, jsc);
|
||||
@@ -133,8 +132,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
*/
|
||||
public JavaRDD<HoodieRecord<T>> filterExists(JavaRDD<HoodieRecord<T>> hoodieRecords) {
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable<T> table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
|
||||
JavaRDD<HoodieRecord<T>> recordsWithLocation = index.tagLocation(hoodieRecords, table);
|
||||
return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown());
|
||||
@@ -146,8 +146,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
public JavaRDD<WriteStatus> upsert(JavaRDD<HoodieRecord<T>> records, final String commitTime) {
|
||||
writeContext = metrics.getCommitCtx();
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable<T> table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
|
||||
try {
|
||||
// De-dupe/merge if needed
|
||||
@@ -180,8 +181,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
public JavaRDD<WriteStatus> insert(JavaRDD<HoodieRecord<T>> records, final String commitTime) {
|
||||
writeContext = metrics.getCommitCtx();
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable<T> table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
try {
|
||||
// De-dupe/merge if needed
|
||||
JavaRDD<HoodieRecord<T>> dedupedRecords =
|
||||
@@ -237,8 +239,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
Option<UserDefinedBulkInsertPartitioner> bulkInsertPartitioner) {
|
||||
writeContext = metrics.getCommitCtx();
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable<T> table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
|
||||
try {
|
||||
// De-dupe/merge if needed
|
||||
@@ -306,8 +309,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
* file instead of using HoodieCommitMetadata
|
||||
*/
|
||||
private void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile,
|
||||
HoodieTable<T> table,
|
||||
String commitTime) throws HoodieCommitException {
|
||||
HoodieTable<T> table, String commitTime) throws HoodieCommitException {
|
||||
try {
|
||||
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
|
||||
profile.getPartitionPaths().stream().forEach(path -> {
|
||||
@@ -409,8 +411,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
|
||||
logger.info("Commiting " + commitTime);
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable<T> table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
|
||||
@@ -485,8 +488,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
* @return true if the savepoint was created successfully
|
||||
*/
|
||||
public boolean savepoint(String user, String comment) {
|
||||
HoodieTable<T> table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
if (table.getCompletedCommitTimeline().empty()) {
|
||||
throw new HoodieSavepointException("Could not savepoint. Commit timeline is empty");
|
||||
}
|
||||
@@ -512,8 +516,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
* @return true if the savepoint was created successfully
|
||||
*/
|
||||
public boolean savepoint(String commitTime, String user, String comment) {
|
||||
HoodieTable<T> table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
Optional<HoodieInstant> cleanInstant = table.getCompletedCleanTimeline().lastInstant();
|
||||
|
||||
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION,
|
||||
@@ -575,8 +580,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
* @return true if the savepoint was deleted successfully
|
||||
*/
|
||||
public void deleteSavepoint(String savepointTime) {
|
||||
HoodieTable<T> table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
|
||||
HoodieInstant savePoint =
|
||||
@@ -602,8 +608,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
* @return true if the savepoint was rollecback to successfully
|
||||
*/
|
||||
public boolean rollbackToSavepoint(String savepointTime) {
|
||||
HoodieTable<T> table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
HoodieTimeline commitTimeline = table.getCommitsTimeline();
|
||||
|
||||
@@ -653,8 +660,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
String startRollbackTime = HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date());
|
||||
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable<T> table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
HoodieTimeline inflightTimeline = table.getInflightCommitTimeline();
|
||||
HoodieTimeline commitTimeline = table.getCompletedCommitTimeline();
|
||||
@@ -761,8 +769,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
final Timer.Context context = metrics.getCleanCtx();
|
||||
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable<T> table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(),
|
||||
true), config);
|
||||
|
||||
List<HoodieCleanStat> cleanStats = table.clean(jsc);
|
||||
if (cleanStats.isEmpty()) {
|
||||
@@ -810,8 +819,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
|
||||
public void startCommitWithTime(String commitTime) {
|
||||
logger.info("Generate a new commit time " + commitTime);
|
||||
HoodieTable<T> table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
String commitActionType = table.getCommitActionType();
|
||||
activeTimeline.createInflight(
|
||||
@@ -827,8 +837,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
*/
|
||||
private void compact(String compactionCommitTime) throws IOException {
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable<T> table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
Optional<HoodieCommitMetadata> compactionMetadata = table.compact(jsc, compactionCommitTime);
|
||||
if (compactionMetadata.isPresent()) {
|
||||
logger.info("Compacted successfully on commit " + compactionCommitTime);
|
||||
@@ -876,8 +887,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
* Cleanup all inflight commits
|
||||
*/
|
||||
private void rollbackInflightCommits() {
|
||||
HoodieTable<T> table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
HoodieTimeline inflightTimeline = table.getCommitsTimeline().filterInflights();
|
||||
List<String> commits = inflightTimeline.getInstants().map(HoodieInstant::getTimestamp)
|
||||
.collect(Collectors.toList());
|
||||
|
||||
Reference in New Issue
Block a user