1
0

HUDI-138 - Meta Files handling also need to support consistency guard

This commit is contained in:
Balaji Varadarajan
2019-06-20 18:05:01 -07:00
committed by Balaji Varadarajan
parent 621c246fa9
commit 5823c1ebd7
21 changed files with 482 additions and 167 deletions

View File

@@ -22,6 +22,7 @@ import com.uber.hoodie.common.storage.StorageSchemes;
import com.uber.hoodie.common.util.ConsistencyGuard;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.NoOpConsistencyGuard;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieIOException;
import java.io.IOException;
import java.net.URI;
@@ -236,7 +237,28 @@ public class HoodieWrapperFileSystem extends FileSystem {
@Override
public boolean rename(Path src, Path dst) throws IOException {
return fileSystem.rename(convertToDefaultPath(src), convertToDefaultPath(dst));
try {
consistencyGuard.waitTillFileAppears(convertToDefaultPath(src));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for " + src + " to appear", e);
}
boolean success = fileSystem.rename(convertToDefaultPath(src), convertToDefaultPath(dst));
if (success) {
try {
consistencyGuard.waitTillFileAppears(convertToDefaultPath(dst));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for " + dst + " to appear", e);
}
try {
consistencyGuard.waitTillFileDisappears(convertToDefaultPath(src));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for " + src + " to disappear", e);
}
}
return success;
}
@Override
@@ -247,7 +269,7 @@ public class HoodieWrapperFileSystem extends FileSystem {
try {
consistencyGuard.waitTillFileDisappears(f);
} catch (TimeoutException e) {
return false;
throw new HoodieException("Timed out waiting for " + f + " to disappear", e);
}
}
return success;
@@ -270,7 +292,15 @@ public class HoodieWrapperFileSystem extends FileSystem {
@Override
public boolean mkdirs(Path f, FsPermission permission) throws IOException {
return fileSystem.mkdirs(convertToDefaultPath(f), permission);
boolean success = fileSystem.mkdirs(convertToDefaultPath(f), permission);
if (success) {
try {
consistencyGuard.waitTillFileAppears(convertToDefaultPath(f));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for directory " + f + " to appear", e);
}
}
return success;
}
@Override
@@ -353,31 +383,39 @@ public class HoodieWrapperFileSystem extends FileSystem {
@Override
public FSDataOutputStream createNonRecursive(Path f, boolean overwrite, int bufferSize,
short replication, long blockSize, Progressable progress) throws IOException {
return fileSystem
.createNonRecursive(convertToDefaultPath(f), overwrite, bufferSize, replication, blockSize,
progress);
Path p = convertToDefaultPath(f);
return wrapOutputStream(p, fileSystem.createNonRecursive(p, overwrite, bufferSize, replication, blockSize,
progress));
}
@Override
public FSDataOutputStream createNonRecursive(Path f, FsPermission permission, boolean overwrite,
int bufferSize, short replication, long blockSize, Progressable progress) throws IOException {
return fileSystem
.createNonRecursive(convertToDefaultPath(f), permission, overwrite, bufferSize, replication,
blockSize, progress);
Path p = convertToDefaultPath(f);
return wrapOutputStream(p, fileSystem.createNonRecursive(p, permission, overwrite, bufferSize, replication,
blockSize, progress));
}
@Override
public FSDataOutputStream createNonRecursive(Path f, FsPermission permission,
EnumSet<CreateFlag> flags, int bufferSize, short replication, long blockSize,
Progressable progress) throws IOException {
return fileSystem
.createNonRecursive(convertToDefaultPath(f), permission, flags, bufferSize, replication,
blockSize, progress);
Path p = convertToDefaultPath(f);
return wrapOutputStream(p, fileSystem.createNonRecursive(p, permission, flags, bufferSize, replication,
blockSize, progress));
}
@Override
public boolean createNewFile(Path f) throws IOException {
return fileSystem.createNewFile(convertToDefaultPath(f));
boolean newFile = fileSystem.createNewFile(convertToDefaultPath(f));
if (newFile) {
try {
consistencyGuard.waitTillFileAppears(convertToDefaultPath(f));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for " + f + " to appear", e);
}
}
return newFile;
}
@Override
@@ -394,6 +432,11 @@ public class HoodieWrapperFileSystem extends FileSystem {
public void concat(Path trg, Path[] psrcs) throws IOException {
Path[] psrcsNew = convertDefaults(psrcs);
fileSystem.concat(convertToDefaultPath(trg), psrcsNew);
try {
consistencyGuard.waitTillFileAppears(convertToDefaultPath(trg));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for " + trg + " to appear", e);
}
}
@Override
@@ -408,7 +451,7 @@ public class HoodieWrapperFileSystem extends FileSystem {
@Override
public boolean delete(Path f) throws IOException {
return fileSystem.delete(convertToDefaultPath(f));
return delete(f, true);
}
@Override
@@ -493,62 +536,100 @@ public class HoodieWrapperFileSystem extends FileSystem {
@Override
public boolean mkdirs(Path f) throws IOException {
return fileSystem.mkdirs(convertToDefaultPath(f));
boolean success = fileSystem.mkdirs(convertToDefaultPath(f));
if (success) {
try {
consistencyGuard.waitTillFileAppears(convertToDefaultPath(f));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for directory " + f + " to appear", e);
}
}
return success;
}
@Override
public void copyFromLocalFile(Path src, Path dst) throws IOException {
fileSystem.copyFromLocalFile(convertToDefaultPath(src), convertToDefaultPath(dst));
fileSystem.copyFromLocalFile(convertToLocalPath(src), convertToDefaultPath(dst));
try {
consistencyGuard.waitTillFileAppears(convertToDefaultPath(dst));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for destination " + dst + " to appear", e);
}
}
@Override
public void moveFromLocalFile(Path[] srcs, Path dst) throws IOException {
fileSystem.moveFromLocalFile(convertDefaults(srcs), convertToDefaultPath(dst));
fileSystem.moveFromLocalFile(convertLocalPaths(srcs), convertToDefaultPath(dst));
try {
consistencyGuard.waitTillFileAppears(convertToDefaultPath(dst));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for destination " + dst + " to appear", e);
}
}
@Override
public void moveFromLocalFile(Path src, Path dst) throws IOException {
fileSystem.moveFromLocalFile(convertToDefaultPath(src), convertToDefaultPath(dst));
fileSystem.moveFromLocalFile(convertToLocalPath(src), convertToDefaultPath(dst));
try {
consistencyGuard.waitTillFileAppears(convertToDefaultPath(dst));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for destination " + dst + " to appear", e);
}
}
@Override
public void copyFromLocalFile(boolean delSrc, Path src, Path dst) throws IOException {
fileSystem.copyFromLocalFile(delSrc, convertToDefaultPath(src), convertToDefaultPath(dst));
fileSystem.copyFromLocalFile(delSrc, convertToLocalPath(src), convertToDefaultPath(dst));
try {
consistencyGuard.waitTillFileAppears(convertToDefaultPath(dst));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for destination " + dst + " to appear", e);
}
}
@Override
public void copyFromLocalFile(boolean delSrc, boolean overwrite, Path[] srcs, Path dst)
throws IOException {
fileSystem
.copyFromLocalFile(delSrc, overwrite, convertDefaults(srcs), convertToDefaultPath(dst));
.copyFromLocalFile(delSrc, overwrite, convertLocalPaths(srcs), convertToDefaultPath(dst));
try {
consistencyGuard.waitTillFileAppears(convertToDefaultPath(dst));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for destination " + dst + " to appear", e);
}
}
@Override
public void copyFromLocalFile(boolean delSrc, boolean overwrite, Path src, Path dst)
throws IOException {
fileSystem
.copyFromLocalFile(delSrc, overwrite, convertToDefaultPath(src), convertToDefaultPath(dst));
.copyFromLocalFile(delSrc, overwrite, convertToLocalPath(src), convertToDefaultPath(dst));
try {
consistencyGuard.waitTillFileAppears(convertToDefaultPath(dst));
} catch (TimeoutException e) {
throw new HoodieException("Timed out waiting for destination " + dst + " to appear", e);
}
}
@Override
public void copyToLocalFile(Path src, Path dst) throws IOException {
fileSystem.copyToLocalFile(convertToDefaultPath(src), convertToDefaultPath(dst));
fileSystem.copyToLocalFile(convertToDefaultPath(src), convertToLocalPath(dst));
}
@Override
public void moveToLocalFile(Path src, Path dst) throws IOException {
fileSystem.moveToLocalFile(convertToDefaultPath(src), convertToDefaultPath(dst));
fileSystem.moveToLocalFile(convertToDefaultPath(src), convertToLocalPath(dst));
}
@Override
public void copyToLocalFile(boolean delSrc, Path src, Path dst) throws IOException {
fileSystem.copyToLocalFile(delSrc, convertToDefaultPath(src), convertToDefaultPath(dst));
fileSystem.copyToLocalFile(delSrc, convertToDefaultPath(src), convertToLocalPath(dst));
}
@Override
public void copyToLocalFile(boolean delSrc, Path src, Path dst, boolean useRawLocalFileSystem)
throws IOException {
fileSystem.copyToLocalFile(delSrc, convertToDefaultPath(src), convertToDefaultPath(dst),
fileSystem.copyToLocalFile(delSrc, convertToDefaultPath(src), convertToLocalPath(dst),
useRawLocalFileSystem);
}
@@ -787,6 +868,22 @@ public class HoodieWrapperFileSystem extends FileSystem {
return convertPathWithScheme(oldPath, fileSystem.getScheme());
}
private Path convertToLocalPath(Path oldPath) {
try {
return convertPathWithScheme(oldPath, FileSystem.getLocal(getConf()).getScheme());
} catch (IOException e) {
throw new HoodieIOException(e.getMessage(), e);
}
}
private Path[] convertLocalPaths(Path[] psrcs) {
Path[] psrcsNew = new Path[psrcs.length];
for (int i = 0; i < psrcs.length; i++) {
psrcsNew[i] = convertToLocalPath(psrcs[i]);
}
return psrcsNew;
}
private Path[] convertDefaults(Path[] psrcs) {
Path[] psrcsNew = new Path[psrcs.length];
for (int i = 0; i < psrcs.length; i++) {
@@ -803,4 +900,8 @@ public class HoodieWrapperFileSystem extends FileSystem {
throw new IllegalArgumentException(file.toString()
+ " does not have a open stream. Cannot get the bytes written on the stream");
}
public FileSystem getFileSystem() {
return fileSystem;
}
}

View File

@@ -20,12 +20,17 @@ package com.uber.hoodie.common.table;
import static com.uber.hoodie.common.model.HoodieTableType.MERGE_ON_READ;
import com.google.common.base.Preconditions;
import com.uber.hoodie.common.SerializableConfiguration;
import com.uber.hoodie.common.io.storage.HoodieWrapperFileSystem;
import com.uber.hoodie.common.model.HoodieTableType;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieArchivedTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.ConsistencyGuardConfig;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.FailSafeConsistencyGuard;
import com.uber.hoodie.common.util.NoOpConsistencyGuard;
import com.uber.hoodie.exception.DatasetNotFoundException;
import com.uber.hoodie.exception.HoodieException;
import java.io.File;
@@ -66,13 +71,14 @@ public class HoodieTableMetaClient implements Serializable {
public static final String MARKER_EXTN = ".marker";
private String basePath;
private transient FileSystem fs;
private transient HoodieWrapperFileSystem fs;
private String metaPath;
private SerializableConfiguration hadoopConf;
private HoodieTableType tableType;
private HoodieTableConfig tableConfig;
private HoodieActiveTimeline activeTimeline;
private HoodieArchivedTimeline archivedTimeline;
private ConsistencyGuardConfig consistencyGuardConfig = ConsistencyGuardConfig.newBuilder().build();
public HoodieTableMetaClient(Configuration conf, String basePath)
throws DatasetNotFoundException {
@@ -81,13 +87,19 @@ public class HoodieTableMetaClient implements Serializable {
}
public HoodieTableMetaClient(Configuration conf, String basePath,
boolean loadActiveTimelineOnLoad)
boolean loadActiveTimelineOnLoad) {
this(conf, basePath, loadActiveTimelineOnLoad, ConsistencyGuardConfig.newBuilder().build());
}
public HoodieTableMetaClient(Configuration conf, String basePath,
boolean loadActiveTimelineOnLoad, ConsistencyGuardConfig consistencyGuardConfig)
throws DatasetNotFoundException {
log.info("Loading HoodieTableMetaClient from " + basePath);
this.basePath = basePath;
this.consistencyGuardConfig = consistencyGuardConfig;
this.hadoopConf = new SerializableConfiguration(conf);
Path basePathDir = new Path(this.basePath);
this.metaPath = basePath + File.separator + METAFOLDER_NAME;
this.metaPath = new Path(basePath, METAFOLDER_NAME).toString();
Path metaPathDir = new Path(this.metaPath);
this.fs = getFs();
DatasetNotFoundException.checkValidDataset(fs, basePathDir, metaPathDir);
@@ -190,13 +202,25 @@ public class HoodieTableMetaClient implements Serializable {
/**
* Get the FS implementation for this table
*/
public FileSystem getFs() {
public HoodieWrapperFileSystem getFs() {
if (fs == null) {
fs = FSUtils.getFs(metaPath, hadoopConf.get());
FileSystem fileSystem = FSUtils.getFs(metaPath, hadoopConf.get());
Preconditions.checkArgument(!(fileSystem instanceof HoodieWrapperFileSystem),
"File System not expected to be that of HoodieWrapperFileSystem");
fs = new HoodieWrapperFileSystem(fileSystem, consistencyGuardConfig.isConsistencyCheckEnabled()
? new FailSafeConsistencyGuard(fileSystem, consistencyGuardConfig) : new NoOpConsistencyGuard());
}
return fs;
}
/**
* Return raw file-system
* @return
*/
public FileSystem getRawFs() {
return getFs().getFileSystem();
}
public Configuration getHadoopConf() {
return hadoopConf.get();
}
@@ -223,6 +247,10 @@ public class HoodieTableMetaClient implements Serializable {
return activeTimeline;
}
public ConsistencyGuardConfig getConsistencyGuardConfig() {
return consistencyGuardConfig;
}
/**
* Get the archived commits as a timeline. This is costly operation, as all data from the archived
* files are read. This should not be used, unless for historical debugging purposes

View File

@@ -0,0 +1,121 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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.util;
import com.uber.hoodie.config.DefaultHoodieConfig;
import java.io.File;
import java.io.FileReader;
import java.io.IOException;
import java.util.Properties;
public class ConsistencyGuardConfig extends DefaultHoodieConfig {
private static final String CONSISTENCY_CHECK_ENABLED_PROP = "hoodie.consistency.check.enabled";
private static final String DEFAULT_CONSISTENCY_CHECK_ENABLED = "false";
// time between successive attempts to ensure written data's metadata is consistent on storage
private static final String INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP =
"hoodie.consistency.check.initial_interval_ms";
private static long DEFAULT_INITIAL_CONSISTENCY_CHECK_INTERVAL_MS = 2000L;
// max interval time
private static final String MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP = "hoodie.consistency.check.max_interval_ms";
private static long DEFAULT_MAX_CONSISTENCY_CHECK_INTERVAL_MS = 300000L;
// maximum number of checks, for consistency of written data. Will wait upto 256 Secs
private static final String MAX_CONSISTENCY_CHECKS_PROP = "hoodie.consistency.check.max_checks";
private static int DEFAULT_MAX_CONSISTENCY_CHECKS = 7;
public ConsistencyGuardConfig(Properties props) {
super(props);
}
public static ConsistencyGuardConfig.Builder newBuilder() {
return new Builder();
}
public boolean isConsistencyCheckEnabled() {
return Boolean.parseBoolean(props.getProperty(CONSISTENCY_CHECK_ENABLED_PROP));
}
public int getMaxConsistencyChecks() {
return Integer.parseInt(props.getProperty(MAX_CONSISTENCY_CHECKS_PROP));
}
public int getInitialConsistencyCheckIntervalMs() {
return Integer.parseInt(props.getProperty(INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP));
}
public int getMaxConsistencyCheckIntervalMs() {
return Integer.parseInt(props.getProperty(MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP));
}
public static class Builder {
private final Properties props = new Properties();
public Builder fromFile(File propertiesFile) throws IOException {
FileReader reader = new FileReader(propertiesFile);
try {
props.load(reader);
return this;
} finally {
reader.close();
}
}
public Builder fromProperties(Properties props) {
this.props.putAll(props);
return this;
}
public Builder withConsistencyCheckEnabled(boolean enabled) {
props.setProperty(CONSISTENCY_CHECK_ENABLED_PROP, String.valueOf(enabled));
return this;
}
public Builder withInitialConsistencyCheckIntervalMs(int initialIntevalMs) {
props.setProperty(INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP, String.valueOf(initialIntevalMs));
return this;
}
public Builder withMaxConsistencyCheckIntervalMs(int maxIntervalMs) {
props.setProperty(MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP, String.valueOf(maxIntervalMs));
return this;
}
public Builder withMaxConsistencyChecks(int maxConsistencyChecks) {
props.setProperty(MAX_CONSISTENCY_CHECKS_PROP, String.valueOf(maxConsistencyChecks));
return this;
}
public ConsistencyGuardConfig build() {
setDefaultOnCondition(props, !props.containsKey(CONSISTENCY_CHECK_ENABLED_PROP),
CONSISTENCY_CHECK_ENABLED_PROP, DEFAULT_CONSISTENCY_CHECK_ENABLED);
setDefaultOnCondition(props, !props.containsKey(INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP),
INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP, String.valueOf(DEFAULT_INITIAL_CONSISTENCY_CHECK_INTERVAL_MS));
setDefaultOnCondition(props, !props.containsKey(MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP),
MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP, String.valueOf(DEFAULT_MAX_CONSISTENCY_CHECK_INTERVAL_MS));
setDefaultOnCondition(props, !props.containsKey(MAX_CONSISTENCY_CHECKS_PROP),
MAX_CONSISTENCY_CHECKS_PROP, String.valueOf(DEFAULT_MAX_CONSISTENCY_CHECKS));
return new ConsistencyGuardConfig(props);
}
}
}

View File

@@ -18,6 +18,7 @@
package com.uber.hoodie.common.util;
import com.google.common.base.Preconditions;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.ArrayList;
@@ -40,15 +41,12 @@ public class FailSafeConsistencyGuard implements ConsistencyGuard {
private static final transient Logger log = LogManager.getLogger(FailSafeConsistencyGuard.class);
private final FileSystem fs;
private final int maxAttempts;
private final long initialDelayMs;
private final long maxDelayMs;
private final ConsistencyGuardConfig consistencyGuardConfig;
public FailSafeConsistencyGuard(FileSystem fs, int maxAttempts, long initalDelayMs, long maxDelayMs) {
public FailSafeConsistencyGuard(FileSystem fs, ConsistencyGuardConfig consistencyGuardConfig) {
this.fs = fs;
this.maxAttempts = maxAttempts;
this.initialDelayMs = initalDelayMs;
this.maxDelayMs = maxDelayMs;
this.consistencyGuardConfig = consistencyGuardConfig;
Preconditions.checkArgument(consistencyGuardConfig.isConsistencyCheckEnabled());
}
@Override
@@ -121,13 +119,13 @@ public class FailSafeConsistencyGuard implements ConsistencyGuard {
*/
private boolean checkFileVisibility(Path filePath, FileVisibility visibility) throws IOException {
try {
FileStatus[] status = fs.listStatus(filePath);
FileStatus status = fs.getFileStatus(filePath);
switch (visibility) {
case APPEAR:
return status.length != 0;
return status != null;
case DISAPPEAR:
default:
return status.length == 0;
return status == null;
}
} catch (FileNotFoundException nfe) {
switch (visibility) {
@@ -147,9 +145,9 @@ public class FailSafeConsistencyGuard implements ConsistencyGuard {
* @throws TimeoutException
*/
private void waitForFileVisibility(Path filePath, FileVisibility visibility) throws TimeoutException {
long waitMs = initialDelayMs;
long waitMs = consistencyGuardConfig.getInitialConsistencyCheckIntervalMs();
int attempt = 0;
while (attempt < maxAttempts) {
while (attempt < consistencyGuardConfig.getMaxConsistencyChecks()) {
try {
if (checkFileVisibility(filePath, visibility)) {
return;
@@ -160,7 +158,7 @@ public class FailSafeConsistencyGuard implements ConsistencyGuard {
sleepSafe(waitMs);
waitMs = waitMs * 2; // double check interval every attempt
waitMs = waitMs > maxDelayMs ? maxDelayMs : waitMs;
waitMs = Math.min(waitMs, consistencyGuardConfig.getMaxConsistencyCheckIntervalMs());
attempt++;
}
throw new TimeoutException("Timed-out waiting for the file to " + visibility.name());
@@ -173,17 +171,17 @@ public class FailSafeConsistencyGuard implements ConsistencyGuard {
* @throws TimeoutException when retries are exhausted
*/
private void retryTillSuccess(Function<Integer, Boolean> predicate, String timedOutMessage) throws TimeoutException {
long waitMs = initialDelayMs;
long waitMs = consistencyGuardConfig.getInitialConsistencyCheckIntervalMs();
int attempt = 0;
log.warn("Max Attempts=" + maxAttempts);
while (attempt < maxAttempts) {
log.info("Max Attempts=" + consistencyGuardConfig.getMaxConsistencyChecks());
while (attempt < consistencyGuardConfig.getMaxConsistencyChecks()) {
boolean success = predicate.apply(attempt);
if (success) {
return;
}
sleepSafe(waitMs);
waitMs = waitMs * 2; // double check interval every attempt
waitMs = waitMs > maxDelayMs ? maxDelayMs : waitMs;
waitMs = Math.min(waitMs, consistencyGuardConfig.getMaxConsistencyCheckIntervalMs());
attempt++;
}
throw new TimeoutException(timedOutMessage);