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
@@ -17,6 +17,7 @@
|
||||
package com.uber.hoodie.cli;
|
||||
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import java.io.IOException;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
@@ -36,7 +37,7 @@ public class HoodieCLI {
|
||||
|
||||
public static boolean initConf() {
|
||||
if (HoodieCLI.conf == null) {
|
||||
HoodieCLI.conf = new Configuration();
|
||||
HoodieCLI.conf = FSUtils.prepareHadoopConf(new Configuration());
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
|
||||
@@ -54,12 +54,14 @@ public class ArchivedCommitsCommand implements CommandMarker {
|
||||
|
||||
System.out
|
||||
.println("===============> Showing only " + limit + " archived commits <===============");
|
||||
FileStatus[] fsStatuses = FSUtils.getFs().globStatus(
|
||||
new Path(HoodieCLI.tableMetadata.getBasePath() + "/.hoodie/.commits_.archive*"));
|
||||
String basePath = HoodieCLI.tableMetadata.getBasePath();
|
||||
FileStatus[] fsStatuses = FSUtils.getFs(basePath, HoodieCLI.conf)
|
||||
.globStatus(new Path(basePath + "/.hoodie/.commits_.archive*"));
|
||||
List<String[]> allCommits = new ArrayList<>();
|
||||
for (FileStatus fs : fsStatuses) {
|
||||
//read the archived file
|
||||
HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(FSUtils.getFs(),
|
||||
HoodieLogFormat.Reader reader = HoodieLogFormat
|
||||
.newReader(FSUtils.getFs(basePath, HoodieCLI.conf),
|
||||
new HoodieLogFile(fs.getPath()), HoodieArchivedMetaEntry.getClassSchema(), false);
|
||||
|
||||
List<IndexedRecord> readRecords = new ArrayList<>();
|
||||
|
||||
@@ -77,7 +77,7 @@ public class CleansCommand implements CommandMarker {
|
||||
@CliCommand(value = "cleans refresh", help = "Refresh the commits")
|
||||
public String refreshCleans() throws IOException {
|
||||
HoodieTableMetaClient metadata =
|
||||
new HoodieTableMetaClient(HoodieCLI.fs, HoodieCLI.tableMetadata.getBasePath());
|
||||
new HoodieTableMetaClient(HoodieCLI.conf, HoodieCLI.tableMetadata.getBasePath());
|
||||
HoodieCLI.setTableMetadata(metadata);
|
||||
return "Metadata for table " + metadata.getTableConfig().getTableName() + " refreshed.";
|
||||
}
|
||||
|
||||
@@ -96,7 +96,7 @@ public class CommitsCommand implements CommandMarker {
|
||||
@CliCommand(value = "commits refresh", help = "Refresh the commits")
|
||||
public String refreshCommits() throws IOException {
|
||||
HoodieTableMetaClient metadata =
|
||||
new HoodieTableMetaClient(HoodieCLI.fs, HoodieCLI.tableMetadata.getBasePath());
|
||||
new HoodieTableMetaClient(HoodieCLI.conf, HoodieCLI.tableMetadata.getBasePath());
|
||||
HoodieCLI.setTableMetadata(metadata);
|
||||
return "Metadata for table " + metadata.getTableConfig().getTableName() + " refreshed.";
|
||||
}
|
||||
@@ -224,14 +224,13 @@ public class CommitsCommand implements CommandMarker {
|
||||
public String compareCommits(
|
||||
@CliOption(key = {"path"}, help = "Path of the dataset to compare to")
|
||||
final String path) throws Exception {
|
||||
HoodieTableMetaClient target = new HoodieTableMetaClient(HoodieCLI.fs, path);
|
||||
|
||||
HoodieTableMetaClient target = new HoodieTableMetaClient(HoodieCLI.conf, path);
|
||||
HoodieTimeline targetTimeline = target.getActiveTimeline().getCommitsTimeline()
|
||||
.filterCompletedInstants();
|
||||
;
|
||||
HoodieTableMetaClient source = HoodieCLI.tableMetadata;
|
||||
HoodieTimeline sourceTimeline = source.getActiveTimeline().getCommitsTimeline()
|
||||
.filterCompletedInstants();
|
||||
;
|
||||
String targetLatestCommit =
|
||||
targetTimeline.getInstants().iterator().hasNext() ? "0"
|
||||
: targetTimeline.lastInstant().get().getTimestamp();
|
||||
@@ -266,7 +265,7 @@ public class CommitsCommand implements CommandMarker {
|
||||
public String syncCommits(
|
||||
@CliOption(key = {"path"}, help = "Path of the dataset to compare to")
|
||||
final String path) throws Exception {
|
||||
HoodieCLI.syncTableMetadata = new HoodieTableMetaClient(HoodieCLI.fs, path);
|
||||
HoodieCLI.syncTableMetadata = new HoodieTableMetaClient(HoodieCLI.conf, path);
|
||||
HoodieCLI.state = HoodieCLI.CLIState.SYNC;
|
||||
return "Load sync state between " + HoodieCLI.tableMetadata.getTableConfig().getTableName()
|
||||
+ " and " + HoodieCLI.syncTableMetadata.getTableConfig().getTableName();
|
||||
|
||||
@@ -33,7 +33,7 @@ public class DatasetsCommand implements CommandMarker {
|
||||
final String path) throws IOException {
|
||||
boolean initialized = HoodieCLI.initConf();
|
||||
HoodieCLI.initFS(initialized);
|
||||
HoodieCLI.setTableMetadata(new HoodieTableMetaClient(HoodieCLI.fs, path));
|
||||
HoodieCLI.setTableMetadata(new HoodieTableMetaClient(HoodieCLI.conf, path));
|
||||
HoodieCLI.state = HoodieCLI.CLIState.DATASET;
|
||||
return "Metadata for table " + HoodieCLI.tableMetadata.getTableConfig().getTableName()
|
||||
+ " loaded";
|
||||
|
||||
@@ -138,7 +138,7 @@ public class SavepointsCommand implements CommandMarker {
|
||||
@CliCommand(value = "savepoints refresh", help = "Refresh the savepoints")
|
||||
public String refreshMetaClient() throws IOException {
|
||||
HoodieTableMetaClient metadata =
|
||||
new HoodieTableMetaClient(HoodieCLI.fs, HoodieCLI.tableMetadata.getBasePath());
|
||||
new HoodieTableMetaClient(HoodieCLI.conf, HoodieCLI.tableMetadata.getBasePath());
|
||||
HoodieCLI.setTableMetadata(metadata);
|
||||
return "Metadata for table " + metadata.getTableConfig().getTableName() + " refreshed.";
|
||||
}
|
||||
|
||||
@@ -97,8 +97,8 @@ public class SparkMain {
|
||||
String repairedOutputPath,
|
||||
String basePath)
|
||||
throws Exception {
|
||||
DedupeSparkJob job = new DedupeSparkJob(basePath,
|
||||
duplicatedPartitionPath, repairedOutputPath, new SQLContext(jsc), FSUtils.getFs());
|
||||
DedupeSparkJob job = new DedupeSparkJob(basePath, duplicatedPartitionPath, repairedOutputPath,
|
||||
new SQLContext(jsc), FSUtils.getFs(basePath, jsc.hadoopConfiguration()));
|
||||
job.fixDuplicates(true);
|
||||
return 0;
|
||||
}
|
||||
|
||||
@@ -18,6 +18,7 @@ package com.uber.hoodie.cli.utils;
|
||||
|
||||
import com.uber.hoodie.HoodieWriteClient;
|
||||
import com.uber.hoodie.cli.commands.SparkMain;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import java.io.File;
|
||||
import java.net.URISyntaxException;
|
||||
import org.apache.log4j.Logger;
|
||||
@@ -66,6 +67,7 @@ public class SparkUtil {
|
||||
sparkConf = HoodieWriteClient.registerClasses(sparkConf);
|
||||
JavaSparkContext jsc = new JavaSparkContext(sparkConf);
|
||||
jsc.hadoopConfiguration().setBoolean("parquet.enable.summary-metadata", false);
|
||||
FSUtils.prepareHadoopConf(jsc.hadoopConfiguration());
|
||||
return jsc;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -75,7 +75,7 @@ class DedupeSparkJob(basePath: String,
|
||||
val tmpTableName = s"htbl_${System.currentTimeMillis()}"
|
||||
val dedupeTblName = s"${tmpTableName}_dupeKeys"
|
||||
|
||||
val metadata = new HoodieTableMetaClient(fs, basePath)
|
||||
val metadata = new HoodieTableMetaClient(fs.getConf, basePath)
|
||||
|
||||
val allFiles = fs.listStatus(new org.apache.hadoop.fs.Path(s"${basePath}/${duplicatedPartitionPath}"))
|
||||
val fsView = new HoodieTableFileSystemView(metadata, metadata.getActiveTimeline.getCommitTimeline.filterCompletedInstants(), allFiles)
|
||||
@@ -127,7 +127,7 @@ class DedupeSparkJob(basePath: String,
|
||||
|
||||
|
||||
def fixDuplicates(dryRun: Boolean = true) = {
|
||||
val metadata = new HoodieTableMetaClient(fs, basePath)
|
||||
val metadata = new HoodieTableMetaClient(fs.getConf, basePath)
|
||||
|
||||
val allFiles = fs.listStatus(new Path(s"${basePath}/${duplicatedPartitionPath}"))
|
||||
val fsView = new HoodieTableFileSystemView(metadata, metadata.getActiveTimeline.getCommitTimeline.filterCompletedInstants(), allFiles)
|
||||
|
||||
@@ -38,7 +38,7 @@ import scala.collection.mutable._
|
||||
object SparkHelpers {
|
||||
@throws[Exception]
|
||||
def skipKeysAndWriteNewFile(commitTime: String, fs: FileSystem, sourceFile: Path, destinationFile: Path, keysToSkip: Set[String]) {
|
||||
val sourceRecords = ParquetUtils.readAvroRecords(sourceFile)
|
||||
val sourceRecords = ParquetUtils.readAvroRecords(fs.getConf, sourceFile)
|
||||
val schema: Schema = sourceRecords.get(0).getSchema
|
||||
val filter: BloomFilter = new BloomFilter(HoodieIndexConfig.DEFAULT_BLOOM_FILTER_NUM_ENTRIES.toInt, HoodieIndexConfig.DEFAULT_BLOOM_FILTER_FPP.toDouble)
|
||||
val writeSupport: HoodieAvroWriteSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter)
|
||||
@@ -47,6 +47,7 @@ object SparkHelpers {
|
||||
for (rec <- sourceRecords) {
|
||||
val key: String = rec.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString
|
||||
if (!keysToSkip.contains(key)) {
|
||||
|
||||
writer.writeAvro(key, rec)
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user