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
@@ -77,7 +77,7 @@ public class MercifulJsonConverter {
|
||||
switch (schema.getType()) {
|
||||
case BOOLEAN:
|
||||
if (value instanceof Boolean) {
|
||||
return (Boolean) value;
|
||||
return value;
|
||||
}
|
||||
break;
|
||||
case DOUBLE:
|
||||
|
||||
@@ -0,0 +1,56 @@
|
||||
/*
|
||||
* Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*
|
||||
*
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.common;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.ObjectInputStream;
|
||||
import java.io.ObjectOutputStream;
|
||||
import java.io.Serializable;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
||||
public class SerializableConfiguration implements Serializable {
|
||||
|
||||
private transient Configuration configuration;
|
||||
|
||||
public SerializableConfiguration(Configuration configuration) {
|
||||
this.configuration = configuration;
|
||||
}
|
||||
|
||||
public Configuration get() {
|
||||
return configuration;
|
||||
}
|
||||
|
||||
private void writeObject(ObjectOutputStream out) throws IOException {
|
||||
out.defaultWriteObject();
|
||||
configuration.write(out);
|
||||
}
|
||||
|
||||
private void readObject(ObjectInputStream in) throws IOException {
|
||||
configuration = new Configuration(false);
|
||||
configuration.readFields(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder str = new StringBuilder();
|
||||
configuration.iterator().forEachRemaining(e ->
|
||||
str.append(String.format("%s => %s \n", e.getKey(), e.getValue())));
|
||||
return configuration.toString();
|
||||
}
|
||||
}
|
||||
@@ -17,5 +17,5 @@
|
||||
package com.uber.hoodie.common.model;
|
||||
|
||||
public enum ActionType {
|
||||
commit, savepoint, compaction, clean, rollback;
|
||||
commit, savepoint, compaction, clean, rollback
|
||||
}
|
||||
|
||||
@@ -16,6 +16,7 @@
|
||||
|
||||
package com.uber.hoodie.common.table;
|
||||
|
||||
import com.uber.hoodie.common.SerializableConfiguration;
|
||||
import com.uber.hoodie.common.model.HoodieTableType;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieArchivedTimeline;
|
||||
@@ -26,6 +27,7 @@ import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.Objects;
|
||||
import java.util.Properties;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
@@ -52,24 +54,28 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
private String basePath;
|
||||
private transient FileSystem fs;
|
||||
private String metaPath;
|
||||
private SerializableConfiguration hadoopConf;
|
||||
private HoodieTableType tableType;
|
||||
private HoodieTableConfig tableConfig;
|
||||
private HoodieActiveTimeline activeTimeline;
|
||||
private HoodieArchivedTimeline archivedTimeline;
|
||||
|
||||
public HoodieTableMetaClient(FileSystem fs, String basePath) throws DatasetNotFoundException {
|
||||
public HoodieTableMetaClient(Configuration conf, String basePath)
|
||||
throws DatasetNotFoundException {
|
||||
// Do not load any timeline by default
|
||||
this(fs, basePath, false);
|
||||
this(conf, basePath, false);
|
||||
}
|
||||
|
||||
public HoodieTableMetaClient(FileSystem fs, String basePath, boolean loadActiveTimelineOnLoad)
|
||||
public HoodieTableMetaClient(Configuration conf, String basePath,
|
||||
boolean loadActiveTimelineOnLoad)
|
||||
throws DatasetNotFoundException {
|
||||
log.info("Loading HoodieTableMetaClient from " + basePath);
|
||||
this.basePath = basePath;
|
||||
this.fs = fs;
|
||||
this.hadoopConf = new SerializableConfiguration(conf);
|
||||
Path basePathDir = new Path(this.basePath);
|
||||
this.metaPath = basePath + File.separator + METAFOLDER_NAME;
|
||||
Path metaPathDir = new Path(this.metaPath);
|
||||
this.fs = getFs();
|
||||
DatasetNotFoundException.checkValidDataset(fs, basePathDir, metaPathDir);
|
||||
this.tableConfig = new HoodieTableConfig(fs, metaPath);
|
||||
this.tableType = tableConfig.getTableType();
|
||||
@@ -96,7 +102,7 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
private void readObject(java.io.ObjectInputStream in)
|
||||
throws IOException, ClassNotFoundException {
|
||||
in.defaultReadObject();
|
||||
this.fs = FSUtils.getFs();
|
||||
fs = null; // will be lazily inited
|
||||
}
|
||||
|
||||
private void writeObject(java.io.ObjectOutputStream out)
|
||||
@@ -136,9 +142,16 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
* Get the FS implementation for this table
|
||||
*/
|
||||
public FileSystem getFs() {
|
||||
if (fs == null) {
|
||||
fs = FSUtils.getFs(metaPath, hadoopConf.get());
|
||||
}
|
||||
return fs;
|
||||
}
|
||||
|
||||
public Configuration getHadoopConf() {
|
||||
return hadoopConf.get();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the active instants as a timeline
|
||||
*
|
||||
@@ -146,7 +159,7 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
*/
|
||||
public synchronized HoodieActiveTimeline getActiveTimeline() {
|
||||
if (activeTimeline == null) {
|
||||
activeTimeline = new HoodieActiveTimeline(fs, metaPath);
|
||||
activeTimeline = new HoodieActiveTimeline(this);
|
||||
}
|
||||
return activeTimeline;
|
||||
}
|
||||
@@ -159,7 +172,7 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
*/
|
||||
public synchronized HoodieArchivedTimeline getArchivedTimeline() {
|
||||
if (archivedTimeline == null) {
|
||||
archivedTimeline = new HoodieArchivedTimeline(fs, metaPath);
|
||||
archivedTimeline = new HoodieArchivedTimeline(this);
|
||||
}
|
||||
return archivedTimeline;
|
||||
}
|
||||
@@ -196,7 +209,7 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
fs.mkdirs(metaPathDir);
|
||||
}
|
||||
HoodieTableConfig.createHoodieProperties(fs, metaPathDir, props);
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath);
|
||||
log.info("Finished initializing Table of type " + metaClient.getTableConfig().getTableType()
|
||||
+ " from " + basePath);
|
||||
return metaClient;
|
||||
|
||||
@@ -83,7 +83,7 @@ public class HoodieCompactedLogRecordScanner implements
|
||||
Schema readerSchema, String latestInstantTime) {
|
||||
this.readerSchema = readerSchema;
|
||||
this.latestInstantTime = latestInstantTime;
|
||||
this.hoodieTableMetaClient = new HoodieTableMetaClient(fs, basePath);
|
||||
this.hoodieTableMetaClient = new HoodieTableMetaClient(fs.getConf(), basePath);
|
||||
// load class from the payload fully qualified class name
|
||||
this.payloadClassFQN = this.hoodieTableMetaClient.getTableConfig().getPayloadClass();
|
||||
|
||||
|
||||
@@ -150,7 +150,7 @@ public interface HoodieLogFormat {
|
||||
public Writer build() throws IOException, InterruptedException {
|
||||
log.info("Building HoodieLogFormat Writer");
|
||||
if (fs == null) {
|
||||
fs = FSUtils.getFs();
|
||||
throw new IllegalArgumentException("fs is not specified");
|
||||
}
|
||||
if (logFileId == null) {
|
||||
throw new IllegalArgumentException("FileID is not specified");
|
||||
|
||||
@@ -58,7 +58,7 @@ public abstract class HoodieLogBlock {
|
||||
*/
|
||||
public enum LogMetadataType {
|
||||
INSTANT_TIME,
|
||||
TARGET_INSTANT_TIME;
|
||||
TARGET_INSTANT_TIME
|
||||
}
|
||||
|
||||
public HoodieLogBlock(Map<LogMetadataType, String> logMetadata) {
|
||||
|
||||
@@ -37,7 +37,6 @@ import java.util.stream.Stream;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
@@ -54,11 +53,8 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
|
||||
public static final SimpleDateFormat COMMIT_FORMATTER = new SimpleDateFormat("yyyyMMddHHmmss");
|
||||
|
||||
|
||||
private final transient static Logger log = LogManager.getLogger(HoodieActiveTimeline.class);
|
||||
private String metaPath;
|
||||
private transient FileSystem fs;
|
||||
|
||||
private HoodieTableMetaClient metaClient;
|
||||
|
||||
/**
|
||||
* Returns next commit time in the {@link #COMMIT_FORMATTER} format.
|
||||
@@ -67,12 +63,14 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
return HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date());
|
||||
}
|
||||
|
||||
protected HoodieActiveTimeline(FileSystem fs, String metaPath, String[] includedExtensions) {
|
||||
protected HoodieActiveTimeline(HoodieTableMetaClient metaClient, String[] includedExtensions) {
|
||||
// Filter all the filter in the metapath and include only the extensions passed and
|
||||
// convert them into HoodieInstant
|
||||
try {
|
||||
this.instants =
|
||||
Arrays.stream(HoodieTableMetaClient.scanFiles(fs, new Path(metaPath), path -> {
|
||||
Arrays.stream(
|
||||
HoodieTableMetaClient
|
||||
.scanFiles(metaClient.getFs(), new Path(metaClient.getMetaPath()), path -> {
|
||||
// Include only the meta files with extensions that needs to be included
|
||||
String extension = FSUtils.getFileExtension(path.getName());
|
||||
return Arrays.stream(includedExtensions).anyMatch(Predicate.isEqual(extension));
|
||||
@@ -85,14 +83,13 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to scan metadata", e);
|
||||
}
|
||||
this.fs = fs;
|
||||
this.metaPath = metaPath;
|
||||
this.metaClient = metaClient;
|
||||
// multiple casts will make this lambda serializable - http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16
|
||||
this.details = (Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails;
|
||||
}
|
||||
|
||||
public HoodieActiveTimeline(FileSystem fs, String metaPath) {
|
||||
this(fs, metaPath,
|
||||
public HoodieActiveTimeline(HoodieTableMetaClient metaClient) {
|
||||
this(metaClient,
|
||||
new String[]{COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, DELTA_COMMIT_EXTENSION,
|
||||
INFLIGHT_DELTA_COMMIT_EXTENSION, SAVEPOINT_EXTENSION, INFLIGHT_SAVEPOINT_EXTENSION,
|
||||
CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION});
|
||||
@@ -114,7 +111,6 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
private void readObject(java.io.ObjectInputStream in)
|
||||
throws IOException, ClassNotFoundException {
|
||||
in.defaultReadObject();
|
||||
this.fs = FSUtils.getFs();
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -214,9 +210,9 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
|
||||
public void deleteInflight(HoodieInstant instant) {
|
||||
log.info("Deleting in-flight " + instant);
|
||||
Path inFlightCommitFilePath = new Path(metaPath, instant.getFileName());
|
||||
Path inFlightCommitFilePath = new Path(metaClient.getMetaPath(), instant.getFileName());
|
||||
try {
|
||||
boolean result = fs.delete(inFlightCommitFilePath, false);
|
||||
boolean result = metaClient.getFs().delete(inFlightCommitFilePath, false);
|
||||
if (result) {
|
||||
log.info("Removed in-flight " + instant);
|
||||
} else {
|
||||
@@ -230,18 +226,18 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
|
||||
@Override
|
||||
public Optional<byte[]> getInstantDetails(HoodieInstant instant) {
|
||||
Path detailPath = new Path(metaPath, instant.getFileName());
|
||||
Path detailPath = new Path(metaClient.getMetaPath(), instant.getFileName());
|
||||
return readDataFromPath(detailPath);
|
||||
}
|
||||
|
||||
protected void moveInflightToComplete(HoodieInstant inflight, HoodieInstant completed,
|
||||
Optional<byte[]> data) {
|
||||
Path commitFilePath = new Path(metaPath, completed.getFileName());
|
||||
Path commitFilePath = new Path(metaClient.getMetaPath(), completed.getFileName());
|
||||
try {
|
||||
// open a new file and write the commit metadata in
|
||||
Path inflightCommitFile = new Path(metaPath, inflight.getFileName());
|
||||
Path inflightCommitFile = new Path(metaClient.getMetaPath(), inflight.getFileName());
|
||||
createFileInMetaPath(inflight.getFileName(), data);
|
||||
boolean success = fs.rename(inflightCommitFile, commitFilePath);
|
||||
boolean success = metaClient.getFs().rename(inflightCommitFile, commitFilePath);
|
||||
if (!success) {
|
||||
throw new HoodieIOException(
|
||||
"Could not rename " + inflightCommitFile + " to " + commitFilePath);
|
||||
@@ -252,11 +248,11 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
}
|
||||
|
||||
protected void moveCompleteToInflight(HoodieInstant completed, HoodieInstant inflight) {
|
||||
Path inFlightCommitFilePath = new Path(metaPath, inflight.getFileName());
|
||||
Path inFlightCommitFilePath = new Path(metaClient.getMetaPath(), inflight.getFileName());
|
||||
try {
|
||||
if (!fs.exists(inFlightCommitFilePath)) {
|
||||
Path commitFilePath = new Path(metaPath, completed.getFileName());
|
||||
boolean success = fs.rename(commitFilePath, inFlightCommitFilePath);
|
||||
if (!metaClient.getFs().exists(inFlightCommitFilePath)) {
|
||||
Path commitFilePath = new Path(metaClient.getMetaPath(), completed.getFileName());
|
||||
boolean success = metaClient.getFs().rename(commitFilePath, inFlightCommitFilePath);
|
||||
if (!success) {
|
||||
throw new HoodieIOException(
|
||||
"Could not rename " + commitFilePath + " to " + inFlightCommitFilePath);
|
||||
@@ -272,15 +268,15 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
}
|
||||
|
||||
protected void createFileInMetaPath(String filename, Optional<byte[]> content) {
|
||||
Path fullPath = new Path(metaPath, filename);
|
||||
Path fullPath = new Path(metaClient.getMetaPath(), filename);
|
||||
try {
|
||||
if (!content.isPresent()) {
|
||||
if (fs.createNewFile(fullPath)) {
|
||||
if (metaClient.getFs().createNewFile(fullPath)) {
|
||||
log.info("Created a new file in meta path: " + fullPath);
|
||||
return;
|
||||
}
|
||||
} else {
|
||||
FSDataOutputStream fsout = fs.create(fullPath, true);
|
||||
FSDataOutputStream fsout = metaClient.getFs().create(fullPath, true);
|
||||
fsout.write(content.get());
|
||||
fsout.close();
|
||||
return;
|
||||
@@ -292,7 +288,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
}
|
||||
|
||||
protected Optional<byte[]> readDataFromPath(Path detailPath) {
|
||||
try (FSDataInputStream is = fs.open(detailPath)) {
|
||||
try (FSDataInputStream is = metaClient.getFs().open(detailPath)) {
|
||||
return Optional.of(IOUtils.toByteArray(is));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Could not read commit details from " + detailPath, e);
|
||||
@@ -300,6 +296,6 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
}
|
||||
|
||||
public HoodieActiveTimeline reload() {
|
||||
return new HoodieActiveTimeline(fs, metaPath);
|
||||
return new HoodieActiveTimeline(metaClient);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -16,8 +16,8 @@
|
||||
|
||||
package com.uber.hoodie.common.table.timeline;
|
||||
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
@@ -27,7 +27,6 @@ import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.io.SequenceFile;
|
||||
import org.apache.hadoop.io.Text;
|
||||
@@ -44,17 +43,17 @@ import org.apache.log4j.Logger;
|
||||
public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
|
||||
|
||||
private static final String HOODIE_COMMIT_ARCHIVE_LOG_FILE = "commits";
|
||||
private transient FileSystem fs;
|
||||
private String metaPath;
|
||||
private HoodieTableMetaClient metaClient;
|
||||
private Map<String, byte[]> readCommits = new HashMap<>();
|
||||
|
||||
private final transient static Logger log = LogManager.getLogger(HoodieArchivedTimeline.class);
|
||||
|
||||
public HoodieArchivedTimeline(FileSystem fs, String metaPath) {
|
||||
public HoodieArchivedTimeline(HoodieTableMetaClient metaClient) {
|
||||
// Read back the commits to make sure
|
||||
Path archiveLogPath = getArchiveLogPath(metaPath);
|
||||
Path archiveLogPath = getArchiveLogPath(metaClient.getMetaPath());
|
||||
try (SequenceFile.Reader reader =
|
||||
new SequenceFile.Reader(fs.getConf(), SequenceFile.Reader.file(archiveLogPath))) {
|
||||
new SequenceFile.Reader(metaClient.getHadoopConf(),
|
||||
SequenceFile.Reader.file(archiveLogPath))) {
|
||||
Text key = new Text();
|
||||
Text val = new Text();
|
||||
while (reader.next(key, val)) {
|
||||
@@ -71,8 +70,7 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
|
||||
}
|
||||
// multiple casts will make this lambda serializable - http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16
|
||||
this.details = (Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails;
|
||||
this.fs = fs;
|
||||
this.metaPath = metaPath;
|
||||
this.metaClient = metaClient;
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -91,7 +89,6 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
|
||||
private void readObject(java.io.ObjectInputStream in)
|
||||
throws IOException, ClassNotFoundException {
|
||||
in.defaultReadObject();
|
||||
this.fs = FSUtils.getFs();
|
||||
}
|
||||
|
||||
|
||||
@@ -105,7 +102,7 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
|
||||
}
|
||||
|
||||
public HoodieArchivedTimeline reload() {
|
||||
return new HoodieArchivedTimeline(fs, metaPath);
|
||||
return new HoodieArchivedTimeline(metaClient);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -40,7 +40,6 @@ import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.commons.lang3.tuple.Pair;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
/**
|
||||
@@ -57,7 +56,6 @@ public class HoodieTableFileSystemView implements TableFileSystemView,
|
||||
TableFileSystemView.RealtimeView, Serializable {
|
||||
|
||||
protected HoodieTableMetaClient metaClient;
|
||||
protected transient FileSystem fs;
|
||||
// This is the commits that will be visible for all views extending this view
|
||||
protected HoodieTimeline visibleActiveTimeline;
|
||||
|
||||
@@ -72,7 +70,6 @@ public class HoodieTableFileSystemView implements TableFileSystemView,
|
||||
public HoodieTableFileSystemView(HoodieTableMetaClient metaClient,
|
||||
HoodieTimeline visibleActiveTimeline) {
|
||||
this.metaClient = metaClient;
|
||||
this.fs = metaClient.getFs();
|
||||
this.visibleActiveTimeline = visibleActiveTimeline;
|
||||
this.fileGroupMap = new HashMap<>();
|
||||
this.partitionToFileGroupsMap = new HashMap<>();
|
||||
@@ -98,7 +95,6 @@ public class HoodieTableFileSystemView implements TableFileSystemView,
|
||||
private void readObject(java.io.ObjectInputStream in)
|
||||
throws IOException, ClassNotFoundException {
|
||||
in.defaultReadObject();
|
||||
this.fs = FSUtils.getFs();
|
||||
}
|
||||
|
||||
private void writeObject(java.io.ObjectOutputStream out)
|
||||
@@ -255,8 +251,8 @@ public class HoodieTableFileSystemView implements TableFileSystemView,
|
||||
try {
|
||||
// Create the path if it does not exist already
|
||||
Path partitionPath = new Path(metaClient.getBasePath(), partitionPathStr);
|
||||
FSUtils.createPathIfNotExists(fs, partitionPath);
|
||||
FileStatus[] statuses = fs.listStatus(partitionPath);
|
||||
FSUtils.createPathIfNotExists(metaClient.getFs(), partitionPath);
|
||||
FileStatus[] statuses = metaClient.getFs().listStatus(partitionPath);
|
||||
List<HoodieFileGroup> fileGroups = addFilesToView(statuses);
|
||||
return fileGroups.stream();
|
||||
} catch (IOException e) {
|
||||
|
||||
@@ -94,7 +94,7 @@ public class AvroUtils {
|
||||
public static HoodieCleanMetadata convertCleanMetadata(String startCleanTime,
|
||||
Optional<Long> durationInMs, List<HoodieCleanStat> cleanStats) {
|
||||
ImmutableMap.Builder<String, HoodieCleanPartitionMetadata> partitionMetadataBuilder =
|
||||
ImmutableMap.<String, HoodieCleanPartitionMetadata>builder();
|
||||
ImmutableMap.builder();
|
||||
int totalDeleted = 0;
|
||||
String earliestCommitToRetain = null;
|
||||
for (HoodieCleanStat stat : cleanStats) {
|
||||
@@ -116,7 +116,7 @@ public class AvroUtils {
|
||||
public static HoodieRollbackMetadata convertRollbackMetadata(String startRollbackTime,
|
||||
Optional<Long> durationInMs, List<String> commits, List<HoodieRollbackStat> stats) {
|
||||
ImmutableMap.Builder<String, HoodieRollbackPartitionMetadata> partitionMetadataBuilder =
|
||||
ImmutableMap.<String, HoodieRollbackPartitionMetadata>builder();
|
||||
ImmutableMap.builder();
|
||||
int totalDeleted = 0;
|
||||
for (HoodieRollbackStat stat : stats) {
|
||||
HoodieRollbackPartitionMetadata metadata =
|
||||
@@ -132,7 +132,7 @@ public class AvroUtils {
|
||||
public static HoodieSavepointMetadata convertSavepointMetadata(String user, String comment,
|
||||
Map<String, List<String>> latestFiles) {
|
||||
ImmutableMap.Builder<String, HoodieSavepointPartitionMetadata> partitionMetadataBuilder =
|
||||
ImmutableMap.<String, HoodieSavepointPartitionMetadata>builder();
|
||||
ImmutableMap.builder();
|
||||
for (Map.Entry<String, List<String>> stat : latestFiles.entrySet()) {
|
||||
HoodieSavepointPartitionMetadata metadata =
|
||||
new HoodieSavepointPartitionMetadata(stat.getKey(), stat.getValue());
|
||||
|
||||
@@ -29,6 +29,7 @@ import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Optional;
|
||||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
@@ -55,6 +56,7 @@ public class FSUtils {
|
||||
private static final int MAX_ATTEMPTS_RECOVER_LEASE = 10;
|
||||
private static final long MIN_CLEAN_TO_KEEP = 10;
|
||||
private static final long MIN_ROLLBACK_TO_KEEP = 10;
|
||||
private static final String HOODIE_ENV_PROPS_PREFIX = "HOODIE_ENV_";
|
||||
private static FileSystem fs;
|
||||
|
||||
/**
|
||||
@@ -65,17 +67,32 @@ public class FSUtils {
|
||||
FSUtils.fs = fs;
|
||||
}
|
||||
|
||||
public static Configuration prepareHadoopConf(Configuration conf) {
|
||||
conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName());
|
||||
conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName());
|
||||
|
||||
public static FileSystem getFs() {
|
||||
// look for all properties, prefixed to be picked up
|
||||
for (Entry<String, String> prop : System.getenv().entrySet()) {
|
||||
if (prop.getKey().startsWith(HOODIE_ENV_PROPS_PREFIX)) {
|
||||
LOG.info("Picking up value for hoodie env var :" + prop.getKey());
|
||||
conf.set(prop.getKey()
|
||||
.replace(HOODIE_ENV_PROPS_PREFIX, "")
|
||||
.replaceAll("_DOT_", "."),
|
||||
prop.getValue());
|
||||
}
|
||||
}
|
||||
return conf;
|
||||
}
|
||||
|
||||
|
||||
public static FileSystem getFs(String path, Configuration conf) {
|
||||
if (fs != null) {
|
||||
return fs;
|
||||
}
|
||||
Configuration conf = new Configuration();
|
||||
conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName());
|
||||
conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName());
|
||||
FileSystem fs;
|
||||
conf = prepareHadoopConf(conf);
|
||||
try {
|
||||
fs = FileSystem.get(conf);
|
||||
fs = new Path(path).getFileSystem(conf);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to get instance of " + FileSystem.class.getName(),
|
||||
e);
|
||||
@@ -83,7 +100,6 @@ public class FSUtils {
|
||||
LOG.info(
|
||||
String.format("Hadoop Configuration: fs.defaultFS: [%s], Config:[%s], FileSystem: [%s]",
|
||||
conf.getRaw("fs.defaultFS"), conf.toString(), fs.toString()));
|
||||
|
||||
return fs;
|
||||
}
|
||||
|
||||
|
||||
@@ -51,10 +51,11 @@ public class ParquetUtils {
|
||||
* Read the rowKey list from the given parquet file.
|
||||
*
|
||||
* @param filePath The parquet file path.
|
||||
* @param configuration configuration to build fs object
|
||||
*/
|
||||
public static Set<String> readRowKeysFromParquet(Path filePath) {
|
||||
Configuration conf = new Configuration();
|
||||
conf.addResource(getFs().getConf());
|
||||
public static Set<String> readRowKeysFromParquet(Configuration configuration, Path filePath) {
|
||||
Configuration conf = new Configuration(configuration);
|
||||
conf.addResource(getFs(filePath.toString(), conf).getConf());
|
||||
Schema readSchema = HoodieAvroUtils.getRecordKeySchema();
|
||||
AvroReadSupport.setAvroReadSchema(conf, readSchema);
|
||||
AvroReadSupport.setRequestedProjection(conf, readSchema);
|
||||
@@ -84,19 +85,12 @@ public class ParquetUtils {
|
||||
return rowKeys;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Read the metadata from a parquet file
|
||||
*/
|
||||
public static ParquetMetadata readMetadata(Path parquetFilePath) {
|
||||
return readMetadata(new Configuration(), parquetFilePath);
|
||||
}
|
||||
|
||||
public static ParquetMetadata readMetadata(Configuration conf, Path parquetFilePath) {
|
||||
ParquetMetadata footer;
|
||||
try {
|
||||
// TODO(vc): Should we use the parallel reading version here?
|
||||
footer = ParquetFileReader.readFooter(getFs().getConf(), parquetFilePath);
|
||||
footer = ParquetFileReader
|
||||
.readFooter(getFs(parquetFilePath.toString(), conf).getConf(), parquetFilePath);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to read footer for parquet " + parquetFilePath,
|
||||
e);
|
||||
@@ -108,14 +102,15 @@ public class ParquetUtils {
|
||||
/**
|
||||
* Get the schema of the given parquet file.
|
||||
*/
|
||||
public static MessageType readSchema(Path parquetFilePath) {
|
||||
return readMetadata(parquetFilePath).getFileMetaData().getSchema();
|
||||
public static MessageType readSchema(Configuration configuration, Path parquetFilePath) {
|
||||
return readMetadata(configuration, parquetFilePath).getFileMetaData().getSchema();
|
||||
}
|
||||
|
||||
|
||||
private static List<String> readParquetFooter(Path parquetFilePath, String... footerNames) {
|
||||
private static List<String> readParquetFooter(Configuration configuration, Path parquetFilePath,
|
||||
String... footerNames) {
|
||||
List<String> footerVals = new ArrayList<>();
|
||||
ParquetMetadata footer = readMetadata(parquetFilePath);
|
||||
ParquetMetadata footer = readMetadata(configuration, parquetFilePath);
|
||||
Map<String, String> metadata = footer.getFileMetaData().getKeyValueMetaData();
|
||||
for (String footerName : footerNames) {
|
||||
if (metadata.containsKey(footerName)) {
|
||||
@@ -128,21 +123,22 @@ public class ParquetUtils {
|
||||
return footerVals;
|
||||
}
|
||||
|
||||
public static Schema readAvroSchema(Path parquetFilePath) {
|
||||
return new AvroSchemaConverter().convert(readSchema(parquetFilePath));
|
||||
public static Schema readAvroSchema(Configuration configuration, Path parquetFilePath) {
|
||||
return new AvroSchemaConverter().convert(readSchema(configuration, parquetFilePath));
|
||||
}
|
||||
|
||||
/**
|
||||
* Read out the bloom filter from the parquet file meta data.
|
||||
*/
|
||||
public static BloomFilter readBloomFilterFromParquetMetadata(Path parquetFilePath) {
|
||||
String footerVal = readParquetFooter(parquetFilePath,
|
||||
public static BloomFilter readBloomFilterFromParquetMetadata(Configuration configuration,
|
||||
Path parquetFilePath) {
|
||||
String footerVal = readParquetFooter(configuration, parquetFilePath,
|
||||
HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY).get(0);
|
||||
return new BloomFilter(footerVal);
|
||||
}
|
||||
|
||||
public static String[] readMinMaxRecordKeys(Path parquetFilePath) {
|
||||
List<String> minMaxKeys = readParquetFooter(parquetFilePath,
|
||||
public static String[] readMinMaxRecordKeys(Configuration configuration, Path parquetFilePath) {
|
||||
List<String> minMaxKeys = readParquetFooter(configuration, parquetFilePath,
|
||||
HoodieAvroWriteSupport.HOODIE_MIN_RECORD_KEY_FOOTER,
|
||||
HoodieAvroWriteSupport.HOODIE_MAX_RECORD_KEY_FOOTER);
|
||||
if (minMaxKeys.size() != 2) {
|
||||
@@ -156,11 +152,11 @@ public class ParquetUtils {
|
||||
/**
|
||||
* NOTE: This literally reads the entire file contents, thus should be used with caution.
|
||||
*/
|
||||
public static List<GenericRecord> readAvroRecords(Path filePath) {
|
||||
public static List<GenericRecord> readAvroRecords(Configuration configuration, Path filePath) {
|
||||
ParquetReader reader = null;
|
||||
List<GenericRecord> records = new ArrayList<>();
|
||||
try {
|
||||
reader = AvroParquetReader.builder(filePath).build();
|
||||
reader = AvroParquetReader.builder(filePath).withConf(configuration).build();
|
||||
Object obj = reader.read();
|
||||
while (obj != null) {
|
||||
if (obj instanceof GenericRecord) {
|
||||
|
||||
Reference in New Issue
Block a user