1
0

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:
Vinoth Chandar
2017-12-10 23:31:54 -08:00
committed by vinoth chandar
parent 44839b88c6
commit 0cd186c899
78 changed files with 851 additions and 535 deletions

View File

@@ -33,7 +33,6 @@ import java.util.Iterator;
import java.util.List;
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;
@@ -51,14 +50,12 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
private final HoodieTimeline commitTimeline;
private HoodieTable<T> hoodieTable;
private HoodieWriteConfig config;
private FileSystem fs;
public HoodieCleanHelper(HoodieTable<T> hoodieTable, HoodieWriteConfig config) {
this.hoodieTable = hoodieTable;
this.fileSystemView = hoodieTable.getCompletedFileSystemView();
this.commitTimeline = hoodieTable.getCompletedCommitTimeline();
this.config = config;
this.fs = hoodieTable.getFs();
}

View File

@@ -126,7 +126,8 @@ public class HoodieCommitArchiveLog {
int minCommitsToKeep = config.getMinCommitsToKeep();
HoodieTable table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
.getHoodieTable(new HoodieTableMetaClient(fs.getConf(), config.getBasePath(), true),
config);
// GroupBy each action and limit each action timeline to maxCommitsToKeep
HoodieTimeline cleanAndRollbackTimeline = table.getActiveTimeline()
@@ -165,7 +166,7 @@ public class HoodieCommitArchiveLog {
private boolean deleteArchivedInstants(List<HoodieInstant> archivedInstants) {
log.info("Deleting instants " + archivedInstants);
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, config.getBasePath(), true);
new HoodieTableMetaClient(fs.getConf(), config.getBasePath(), true);
boolean success = true;
for (HoodieInstant archivedInstant : archivedInstants) {
@@ -188,7 +189,7 @@ public class HoodieCommitArchiveLog {
try {
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, config.getBasePath(), true);
new HoodieTableMetaClient(fs.getConf(), config.getBasePath(), true);
HoodieTimeline commitTimeline =
metaClient.getActiveTimeline().getAllCommitsTimeline().filterCompletedInstants();

View File

@@ -47,7 +47,7 @@ public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
HoodieTable<T> hoodieTable) {
this.commitTime = commitTime;
this.config = config;
this.fs = FSUtils.getFs();
this.fs = hoodieTable.getMetaClient().getFs();
this.hoodieTable = hoodieTable;
this.hoodieTimeline = hoodieTable.getCompletedCommitTimeline();
this.fileSystemView = hoodieTable.getROFileSystemView();
@@ -73,8 +73,9 @@ public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
public static void cleanupTmpFilesFromCurrentCommit(HoodieWriteConfig config,
String commitTime,
String partitionPath,
int taskPartitionId) {
FileSystem fs = FSUtils.getFs();
int taskPartitionId,
HoodieTable hoodieTable) {
FileSystem fs = hoodieTable.getMetaClient().getFs();
try {
FileStatus[] prevFailedFiles = fs.globStatus(new Path(String
.format("%s/%s/%s", config.getBasePath(), partitionPath,

View File

@@ -16,6 +16,8 @@
package com.uber.hoodie.io.compact;
import static java.util.stream.Collectors.toList;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
@@ -34,13 +36,6 @@ import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieCompactionException;
import com.uber.hoodie.table.HoodieCopyOnWriteTable;
import com.uber.hoodie.table.HoodieTable;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.FileSystem;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.FlatMapFunction;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.Collection;
@@ -49,8 +44,12 @@ import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;
import java.util.stream.StreamSupport;
import static java.util.stream.Collectors.toList;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.FileSystem;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.FlatMapFunction;
/**
* HoodieRealtimeTableCompactor compacts a hoodie table with merge on read storage. Computes all
@@ -73,7 +72,6 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
.getTableType().name());
// TODO - rollback any compactions in flight
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
log.info("Compacting " + metaClient.getBasePath() + " with commit " + compactionCommitTime);
List<String> partitionPaths =
@@ -102,15 +100,9 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
log.info("After filtering, Compacting " + operations + " files");
List<HoodieWriteStat> updateStatusMap =
jsc.parallelize(operations, operations.size())
.map(s -> executeCompaction(metaClient, config, s, compactionCommitTime))
.flatMap(new FlatMapFunction<List<HoodieWriteStat>, HoodieWriteStat>() {
@Override
public Iterator<HoodieWriteStat> call(
List<HoodieWriteStat> hoodieWriteStats)
throws Exception {
return hoodieWriteStats.iterator();
}
}).collect();
.map(s -> executeCompaction(hoodieTable, config, s, compactionCommitTime))
.flatMap(writeStatList -> writeStatList.iterator())
.collect();
HoodieCommitMetadata metadata = new HoodieCommitMetadata(true);
for (HoodieWriteStat stat : updateStatusMap) {
@@ -134,10 +126,11 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
return true;
}
private List<HoodieWriteStat> executeCompaction(HoodieTableMetaClient metaClient,
HoodieWriteConfig config, CompactionOperation operation, String commitTime)
private List<HoodieWriteStat> executeCompaction(HoodieTable hoodieTable,
HoodieWriteConfig config, CompactionOperation operation, String commitTime)
throws IOException {
FileSystem fs = FSUtils.getFs();
FileSystem fs = hoodieTable.getMetaClient().getFs();
Schema readerSchema =
HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
@@ -147,7 +140,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
// Reads the entire avro file. Always only specific blocks should be read from the avro file (failure recover).
// Load all the delta commits since the last compaction commit and get all the blocks to be loaded and load it using CompositeAvroLogReader
// Since a DeltaCommit is not defined yet, reading all the records. revisit this soon.
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
String maxInstantTime = metaClient.getActiveTimeline()
.getTimelineOfActions(
Sets.newHashSet(HoodieTimeline.COMMIT_ACTION,
@@ -162,8 +155,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
}
// Compacting is very similar to applying updates to existing file
HoodieCopyOnWriteTable table =
new HoodieCopyOnWriteTable(config, metaClient);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metaClient);
Iterator<List<WriteStatus>> result = table
.handleUpdate(commitTime, operation.getFileId(), scanner.iterator());
Iterable<List<WriteStatus>> resultIterable = () -> result;

View File

@@ -19,6 +19,7 @@ package com.uber.hoodie.io.storage;
import com.uber.hoodie.avro.HoodieAvroWriteSupport;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import java.io.IOException;
import java.util.concurrent.atomic.AtomicLong;
@@ -26,7 +27,6 @@ import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.parquet.hadoop.ParquetFileWriter;
import org.apache.parquet.hadoop.ParquetWriter;
@@ -51,9 +51,9 @@ public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends Indexe
private final Schema schema;
private static Configuration registerFileSystem(Configuration conf) {
private static Configuration registerFileSystem(Path file, Configuration conf) {
Configuration returnConf = new Configuration(conf);
String scheme = FileSystem.getDefaultUri(conf).getScheme();
String scheme = FSUtils.getFs(file.toString(), conf).getScheme();
returnConf.set("fs." + HoodieWrapperFileSystem.getHoodieScheme(scheme) + ".impl",
HoodieWrapperFileSystem.class.getName());
return returnConf;
@@ -67,11 +67,11 @@ public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends Indexe
parquetConfig.getPageSize(), parquetConfig.getPageSize(),
ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED,
ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED, ParquetWriter.DEFAULT_WRITER_VERSION,
registerFileSystem(parquetConfig.getHadoopConf()));
registerFileSystem(file, parquetConfig.getHadoopConf()));
this.file =
HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf());
this.fs = (HoodieWrapperFileSystem) this.file
.getFileSystem(registerFileSystem(parquetConfig.getHadoopConf()));
.getFileSystem(registerFileSystem(file, parquetConfig.getHadoopConf()));
// We cannot accurately measure the snappy compressed output file size. We are choosing a conservative 10%
// TODO - compute this compression ratio dynamically by looking at the bytes written to the stream and the actual file size reported by HDFS
this.maxFileSize = parquetConfig.getMaxFileSize() + Math

View File

@@ -19,7 +19,6 @@ package com.uber.hoodie.io.storage;
import com.uber.hoodie.avro.HoodieAvroWriteSupport;
import com.uber.hoodie.common.BloomFilter;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.table.HoodieTable;
import java.io.IOException;
@@ -37,11 +36,12 @@ public class HoodieStorageWriterFactory {
throws IOException {
//TODO - based on the metadata choose the implementation of HoodieStorageWriter
// Currently only parquet is supported
return newParquetStorageWriter(commitTime, path, config, schema);
return newParquetStorageWriter(commitTime, path, config, schema, hoodieTable);
}
private static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieStorageWriter<R> newParquetStorageWriter(
String commitTime, Path path, HoodieWriteConfig config, Schema schema) throws IOException {
String commitTime, Path path, HoodieWriteConfig config, Schema schema,
HoodieTable hoodieTable) throws IOException {
BloomFilter filter =
new BloomFilter(config.getBloomFilterNumEntries(), config.getBloomFilterFPP());
HoodieAvroWriteSupport writeSupport =
@@ -50,7 +50,7 @@ public class HoodieStorageWriterFactory {
HoodieParquetConfig parquetConfig =
new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP,
config.getParquetBlockSize(), config.getParquetPageSize(),
config.getParquetMaxFileSize(), FSUtils.getFs().getConf());
config.getParquetMaxFileSize(), hoodieTable.getHadoopConf());
return new HoodieParquetWriter<>(commitTime, path, parquetConfig, schema);
}

View File

@@ -16,7 +16,8 @@
package com.uber.hoodie.io.storage;
import java.io.FileNotFoundException;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.HoodieIOException;
import java.io.IOException;
import java.net.URI;
import java.net.URISyntaxException;
@@ -33,7 +34,6 @@ import org.apache.hadoop.fs.ContentSummary;
import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.FileChecksum;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@@ -41,17 +41,14 @@ import org.apache.hadoop.fs.FsServerDefaults;
import org.apache.hadoop.fs.FsStatus;
import org.apache.hadoop.fs.LocatedFileStatus;
import org.apache.hadoop.fs.Options;
import org.apache.hadoop.fs.ParentNotDirectoryException;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.fs.UnsupportedFileSystemException;
import org.apache.hadoop.fs.XAttrSetFlag;
import org.apache.hadoop.fs.permission.AclEntry;
import org.apache.hadoop.fs.permission.AclStatus;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.Progressable;
@@ -70,6 +67,8 @@ public class HoodieWrapperFileSystem extends FileSystem {
SUPPORT_SCHEMES.add("file");
SUPPORT_SCHEMES.add("hdfs");
SUPPORT_SCHEMES.add("s3");
SUPPORT_SCHEMES.add("s3a");
// Hoodie currently relies on underlying object store being fully
// consistent so only regional buckets should be used.
@@ -85,7 +84,12 @@ public class HoodieWrapperFileSystem extends FileSystem {
@Override
public void initialize(URI uri, Configuration conf) throws IOException {
// Get the default filesystem to decorate
fileSystem = FileSystem.get(conf);
Path path = new Path(uri);
// Remove 'hoodie-' prefix from path
if (path.toString().startsWith(HOODIE_SCHEME_PREFIX)) {
path = new Path(path.toString().replace(HOODIE_SCHEME_PREFIX, ""));
}
this.fileSystem = FSUtils.getFs(path.toString(), conf);
// Do not need to explicitly initialize the default filesystem, its done already in the above FileSystem.get
// fileSystem.initialize(FileSystem.getDefaultUri(conf), conf);
// fileSystem.setConf(conf);
@@ -219,7 +223,7 @@ public class HoodieWrapperFileSystem extends FileSystem {
}
@Override
public FileStatus[] listStatus(Path f) throws FileNotFoundException, IOException {
public FileStatus[] listStatus(Path f) throws IOException {
return fileSystem.listStatus(convertToDefaultPath(f));
}
@@ -415,19 +419,19 @@ public class HoodieWrapperFileSystem extends FileSystem {
@Override
public FileStatus[] listStatus(Path f, PathFilter filter)
throws FileNotFoundException, IOException {
throws IOException {
return fileSystem.listStatus(convertToDefaultPath(f), filter);
}
@Override
public FileStatus[] listStatus(Path[] files)
throws FileNotFoundException, IOException {
throws IOException {
return fileSystem.listStatus(convertDefaults(files));
}
@Override
public FileStatus[] listStatus(Path[] files, PathFilter filter)
throws FileNotFoundException, IOException {
throws IOException {
return fileSystem.listStatus(convertDefaults(files), filter);
}
@@ -444,13 +448,13 @@ public class HoodieWrapperFileSystem extends FileSystem {
@Override
public RemoteIterator<LocatedFileStatus> listLocatedStatus(Path f)
throws FileNotFoundException, IOException {
throws IOException {
return fileSystem.listLocatedStatus(convertToDefaultPath(f));
}
@Override
public RemoteIterator<LocatedFileStatus> listFiles(Path f, boolean recursive)
throws FileNotFoundException, IOException {
throws IOException {
return fileSystem.listFiles(convertToDefaultPath(f), recursive);
}
@@ -571,21 +575,21 @@ public class HoodieWrapperFileSystem extends FileSystem {
@Override
public void access(Path path, FsAction mode)
throws AccessControlException, FileNotFoundException, IOException {
throws IOException {
fileSystem.access(convertToDefaultPath(path), mode);
}
@Override
public void createSymlink(Path target, Path link, boolean createParent)
throws AccessControlException, FileAlreadyExistsException, FileNotFoundException,
ParentNotDirectoryException, UnsupportedFileSystemException, IOException {
throws
IOException {
fileSystem
.createSymlink(convertToDefaultPath(target), convertToDefaultPath(link), createParent);
}
@Override
public FileStatus getFileLinkStatus(Path f)
throws AccessControlException, FileNotFoundException, UnsupportedFileSystemException,
throws
IOException {
return fileSystem.getFileLinkStatus(convertToDefaultPath(f));
}
@@ -759,8 +763,12 @@ public class HoodieWrapperFileSystem extends FileSystem {
}
public static Path convertToHoodiePath(Path file, Configuration conf) {
String scheme = FileSystem.getDefaultUri(conf).getScheme();
return convertPathWithScheme(file, getHoodieScheme(scheme));
try {
String scheme = FSUtils.getFs(file.toString(), conf).getScheme();
return convertPathWithScheme(file, getHoodieScheme(scheme));
} catch (HoodieIOException e) {
throw e;
}
}
private Path convertToDefaultPath(Path oldPath) {