Spark Stage retry handling
This commit is contained in:
committed by
vinoth chandar
parent
3fd2fd6e9d
commit
145034c5fa
@@ -0,0 +1,804 @@
|
||||
/*
|
||||
* Copyright (c) 2016 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.io.storage;
|
||||
|
||||
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.HoodieIOException;
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.net.URISyntaxException;
|
||||
import java.util.EnumSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.BlockLocation;
|
||||
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.FileChecksum;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
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.Path;
|
||||
import org.apache.hadoop.fs.PathFilter;
|
||||
import org.apache.hadoop.fs.RemoteIterator;
|
||||
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.Credentials;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.util.Progressable;
|
||||
|
||||
/**
|
||||
* HoodieWrapperFileSystem wraps the default file system. It holds state about the open streams in
|
||||
* the file system to support getting the written size to each of the open streams.
|
||||
*/
|
||||
public class HoodieWrapperFileSystem extends FileSystem {
|
||||
|
||||
public static final String HOODIE_SCHEME_PREFIX = "hoodie-";
|
||||
|
||||
private ConcurrentMap<String, SizeAwareFSDataOutputStream> openStreams = new
|
||||
ConcurrentHashMap<>();
|
||||
private FileSystem fileSystem;
|
||||
private URI uri;
|
||||
private ConsistencyGuard consistencyGuard = new NoOpConsistencyGuard();
|
||||
|
||||
public HoodieWrapperFileSystem() {
|
||||
}
|
||||
|
||||
public HoodieWrapperFileSystem(FileSystem fileSystem, ConsistencyGuard consistencyGuard) {
|
||||
this.fileSystem = fileSystem;
|
||||
this.uri = fileSystem.getUri();
|
||||
this.consistencyGuard = consistencyGuard;
|
||||
}
|
||||
|
||||
public static Path convertToHoodiePath(Path file, Configuration conf) {
|
||||
try {
|
||||
String scheme = FSUtils.getFs(file.toString(), conf).getScheme();
|
||||
return convertPathWithScheme(file, getHoodieScheme(scheme));
|
||||
} catch (HoodieIOException e) {
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
private static Path convertPathWithScheme(Path oldPath, String newScheme) {
|
||||
URI oldURI = oldPath.toUri();
|
||||
URI newURI;
|
||||
try {
|
||||
newURI = new URI(newScheme, oldURI.getUserInfo(), oldURI.getHost(), oldURI.getPort(),
|
||||
oldURI.getPath(), oldURI.getQuery(), oldURI.getFragment());
|
||||
return new Path(newURI);
|
||||
} catch (URISyntaxException e) {
|
||||
// TODO - Better Exception handling
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
public static String getHoodieScheme(String scheme) {
|
||||
String newScheme;
|
||||
if (StorageSchemes.isSchemeSupported(scheme)) {
|
||||
newScheme = HOODIE_SCHEME_PREFIX + scheme;
|
||||
} else {
|
||||
throw new IllegalArgumentException(
|
||||
"BlockAlignedAvroParquetWriter does not support scheme " + scheme);
|
||||
}
|
||||
return newScheme;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void initialize(URI uri, Configuration conf) throws IOException {
|
||||
// Get the default filesystem to decorate
|
||||
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);
|
||||
this.uri = uri;
|
||||
}
|
||||
|
||||
@Override
|
||||
public URI getUri() {
|
||||
return uri;
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataInputStream open(Path f, int bufferSize) throws IOException {
|
||||
return fileSystem.open(convertToDefaultPath(f), bufferSize);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f, FsPermission permission, boolean overwrite,
|
||||
int bufferSize, short replication, long blockSize, Progressable progress) throws IOException {
|
||||
final Path translatedPath = convertToDefaultPath(f);
|
||||
return wrapOutputStream(f, fileSystem
|
||||
.create(translatedPath, permission, overwrite, bufferSize, replication, blockSize,
|
||||
progress));
|
||||
}
|
||||
|
||||
private FSDataOutputStream wrapOutputStream(final Path path,
|
||||
FSDataOutputStream fsDataOutputStream) throws IOException {
|
||||
if (fsDataOutputStream instanceof SizeAwareFSDataOutputStream) {
|
||||
return fsDataOutputStream;
|
||||
}
|
||||
|
||||
SizeAwareFSDataOutputStream os = new SizeAwareFSDataOutputStream(path,
|
||||
fsDataOutputStream, consistencyGuard, () -> openStreams.remove(path.getName()));
|
||||
openStreams.put(path.getName(), os);
|
||||
return os;
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f, boolean overwrite) throws IOException {
|
||||
return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), overwrite));
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f) throws IOException {
|
||||
return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f)));
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f, Progressable progress) throws IOException {
|
||||
return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), progress));
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f, short replication) throws IOException {
|
||||
return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), replication));
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f, short replication, Progressable progress)
|
||||
throws IOException {
|
||||
return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), replication, progress));
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize) throws IOException {
|
||||
return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), overwrite, bufferSize));
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, Progressable progress)
|
||||
throws IOException {
|
||||
return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), overwrite, bufferSize, progress));
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, short replication,
|
||||
long blockSize, Progressable progress) throws IOException {
|
||||
return wrapOutputStream(f, fileSystem
|
||||
.create(convertToDefaultPath(f), overwrite, bufferSize, replication, blockSize, progress));
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f, FsPermission permission, EnumSet<CreateFlag> flags,
|
||||
int bufferSize, short replication, long blockSize, Progressable progress) throws IOException {
|
||||
return wrapOutputStream(f, fileSystem
|
||||
.create(convertToDefaultPath(f), permission, flags, bufferSize, replication, blockSize,
|
||||
progress));
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f, FsPermission permission, EnumSet<CreateFlag> flags,
|
||||
int bufferSize, short replication, long blockSize, Progressable progress,
|
||||
Options.ChecksumOpt checksumOpt) throws IOException {
|
||||
return wrapOutputStream(f, fileSystem
|
||||
.create(convertToDefaultPath(f), permission, flags, bufferSize, replication, blockSize,
|
||||
progress, checksumOpt));
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, short replication,
|
||||
long blockSize) throws IOException {
|
||||
return wrapOutputStream(f, fileSystem
|
||||
.create(convertToDefaultPath(f), overwrite, bufferSize, replication, blockSize));
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream append(Path f, int bufferSize, Progressable progress)
|
||||
throws IOException {
|
||||
return wrapOutputStream(f, fileSystem.append(convertToDefaultPath(f), bufferSize, progress));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean rename(Path src, Path dst) throws IOException {
|
||||
return fileSystem.rename(convertToDefaultPath(src), convertToDefaultPath(dst));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean delete(Path f, boolean recursive) throws IOException {
|
||||
boolean success = fileSystem.delete(convertToDefaultPath(f), recursive);
|
||||
|
||||
if (success) {
|
||||
try {
|
||||
consistencyGuard.waitTillFileDisappears(f);
|
||||
} catch (TimeoutException e) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return success;
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileStatus[] listStatus(Path f) throws IOException {
|
||||
return fileSystem.listStatus(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Path getWorkingDirectory() {
|
||||
return convertToHoodiePath(fileSystem.getWorkingDirectory());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setWorkingDirectory(Path newDir) {
|
||||
fileSystem.setWorkingDirectory(convertToDefaultPath(newDir));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean mkdirs(Path f, FsPermission permission) throws IOException {
|
||||
return fileSystem.mkdirs(convertToDefaultPath(f), permission);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileStatus getFileStatus(Path f) throws IOException {
|
||||
try {
|
||||
consistencyGuard.waitTillFileAppears(convertToDefaultPath(f));
|
||||
} catch (TimeoutException e) {
|
||||
// pass
|
||||
}
|
||||
return fileSystem.getFileStatus(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getScheme() {
|
||||
return uri.getScheme();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getCanonicalServiceName() {
|
||||
return fileSystem.getCanonicalServiceName();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getName() {
|
||||
return fileSystem.getName();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Path makeQualified(Path path) {
|
||||
return convertToHoodiePath(fileSystem.makeQualified(convertToDefaultPath(path)));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Token<?> getDelegationToken(String renewer) throws IOException {
|
||||
return fileSystem.getDelegationToken(renewer);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Token<?>[] addDelegationTokens(String renewer, Credentials credentials)
|
||||
throws IOException {
|
||||
return fileSystem.addDelegationTokens(renewer, credentials);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileSystem[] getChildFileSystems() {
|
||||
return fileSystem.getChildFileSystems();
|
||||
}
|
||||
|
||||
@Override
|
||||
public BlockLocation[] getFileBlockLocations(FileStatus file, long start, long len)
|
||||
throws IOException {
|
||||
return fileSystem.getFileBlockLocations(file, start, len);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BlockLocation[] getFileBlockLocations(Path p, long start, long len) throws IOException {
|
||||
return fileSystem.getFileBlockLocations(convertToDefaultPath(p), start, len);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FsServerDefaults getServerDefaults() throws IOException {
|
||||
return fileSystem.getServerDefaults();
|
||||
}
|
||||
|
||||
@Override
|
||||
public FsServerDefaults getServerDefaults(Path p) throws IOException {
|
||||
return fileSystem.getServerDefaults(convertToDefaultPath(p));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Path resolvePath(Path p) throws IOException {
|
||||
return convertToHoodiePath(fileSystem.resolvePath(convertToDefaultPath(p)));
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataInputStream open(Path f) throws IOException {
|
||||
return fileSystem.open(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@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);
|
||||
}
|
||||
|
||||
@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);
|
||||
}
|
||||
|
||||
@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);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean createNewFile(Path f) throws IOException {
|
||||
return fileSystem.createNewFile(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream append(Path f) throws IOException {
|
||||
return wrapOutputStream(f, fileSystem.append(convertToDefaultPath(f)));
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream append(Path f, int bufferSize) throws IOException {
|
||||
return wrapOutputStream(f, fileSystem.append(convertToDefaultPath(f), bufferSize));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void concat(Path trg, Path[] psrcs) throws IOException {
|
||||
Path[] psrcsNew = convertDefaults(psrcs);
|
||||
fileSystem.concat(convertToDefaultPath(trg), psrcsNew);
|
||||
}
|
||||
|
||||
@Override
|
||||
public short getReplication(Path src) throws IOException {
|
||||
return fileSystem.getReplication(convertToDefaultPath(src));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean setReplication(Path src, short replication) throws IOException {
|
||||
return fileSystem.setReplication(convertToDefaultPath(src), replication);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean delete(Path f) throws IOException {
|
||||
return fileSystem.delete(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean deleteOnExit(Path f) throws IOException {
|
||||
return fileSystem.deleteOnExit(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean cancelDeleteOnExit(Path f) {
|
||||
return fileSystem.cancelDeleteOnExit(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean exists(Path f) throws IOException {
|
||||
return fileSystem.exists(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isDirectory(Path f) throws IOException {
|
||||
return fileSystem.isDirectory(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isFile(Path f) throws IOException {
|
||||
return fileSystem.isFile(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLength(Path f) throws IOException {
|
||||
return fileSystem.getLength(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override
|
||||
public ContentSummary getContentSummary(Path f) throws IOException {
|
||||
return fileSystem.getContentSummary(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override
|
||||
public RemoteIterator<Path> listCorruptFileBlocks(Path path) throws IOException {
|
||||
return fileSystem.listCorruptFileBlocks(convertToDefaultPath(path));
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileStatus[] listStatus(Path f, PathFilter filter) throws IOException {
|
||||
return fileSystem.listStatus(convertToDefaultPath(f), filter);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileStatus[] listStatus(Path[] files) throws IOException {
|
||||
return fileSystem.listStatus(convertDefaults(files));
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileStatus[] listStatus(Path[] files, PathFilter filter) throws IOException {
|
||||
return fileSystem.listStatus(convertDefaults(files), filter);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileStatus[] globStatus(Path pathPattern) throws IOException {
|
||||
return fileSystem.globStatus(convertToDefaultPath(pathPattern));
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileStatus[] globStatus(Path pathPattern, PathFilter filter) throws IOException {
|
||||
return fileSystem.globStatus(convertToDefaultPath(pathPattern), filter);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RemoteIterator<LocatedFileStatus> listLocatedStatus(Path f) throws IOException {
|
||||
return fileSystem.listLocatedStatus(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override
|
||||
public RemoteIterator<LocatedFileStatus> listFiles(Path f, boolean recursive) throws IOException {
|
||||
return fileSystem.listFiles(convertToDefaultPath(f), recursive);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Path getHomeDirectory() {
|
||||
return convertToHoodiePath(fileSystem.getHomeDirectory());
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean mkdirs(Path f) throws IOException {
|
||||
return fileSystem.mkdirs(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void copyFromLocalFile(Path src, Path dst) throws IOException {
|
||||
fileSystem.copyFromLocalFile(convertToDefaultPath(src), convertToDefaultPath(dst));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void moveFromLocalFile(Path[] srcs, Path dst) throws IOException {
|
||||
fileSystem.moveFromLocalFile(convertDefaults(srcs), convertToDefaultPath(dst));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void moveFromLocalFile(Path src, Path dst) throws IOException {
|
||||
fileSystem.moveFromLocalFile(convertToDefaultPath(src), convertToDefaultPath(dst));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void copyFromLocalFile(boolean delSrc, Path src, Path dst) throws IOException {
|
||||
fileSystem.copyFromLocalFile(delSrc, convertToDefaultPath(src), convertToDefaultPath(dst));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void copyFromLocalFile(boolean delSrc, boolean overwrite, Path[] srcs, Path dst)
|
||||
throws IOException {
|
||||
fileSystem
|
||||
.copyFromLocalFile(delSrc, overwrite, convertDefaults(srcs), convertToDefaultPath(dst));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void copyFromLocalFile(boolean delSrc, boolean overwrite, Path src, Path dst)
|
||||
throws IOException {
|
||||
fileSystem
|
||||
.copyFromLocalFile(delSrc, overwrite, convertToDefaultPath(src), convertToDefaultPath(dst));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void copyToLocalFile(Path src, Path dst) throws IOException {
|
||||
fileSystem.copyToLocalFile(convertToDefaultPath(src), convertToDefaultPath(dst));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void moveToLocalFile(Path src, Path dst) throws IOException {
|
||||
fileSystem.moveToLocalFile(convertToDefaultPath(src), convertToDefaultPath(dst));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void copyToLocalFile(boolean delSrc, Path src, Path dst) throws IOException {
|
||||
fileSystem.copyToLocalFile(delSrc, convertToDefaultPath(src), convertToDefaultPath(dst));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void copyToLocalFile(boolean delSrc, Path src, Path dst, boolean useRawLocalFileSystem)
|
||||
throws IOException {
|
||||
fileSystem.copyToLocalFile(delSrc, convertToDefaultPath(src), convertToDefaultPath(dst),
|
||||
useRawLocalFileSystem);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Path startLocalOutput(Path fsOutputFile, Path tmpLocalFile) throws IOException {
|
||||
return convertToHoodiePath(fileSystem
|
||||
.startLocalOutput(convertToDefaultPath(fsOutputFile), convertToDefaultPath(tmpLocalFile)));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void completeLocalOutput(Path fsOutputFile, Path tmpLocalFile) throws IOException {
|
||||
fileSystem.completeLocalOutput(convertToDefaultPath(fsOutputFile),
|
||||
convertToDefaultPath(tmpLocalFile));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
// Don't close the wrapped `fileSystem` object. This will end up closing it for every thread since it
|
||||
// could be cached across jvm. We don't own that object anyway.
|
||||
super.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getUsed() throws IOException {
|
||||
return fileSystem.getUsed();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getBlockSize(Path f) throws IOException {
|
||||
return fileSystem.getBlockSize(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getDefaultBlockSize() {
|
||||
return fileSystem.getDefaultBlockSize();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getDefaultBlockSize(Path f) {
|
||||
return fileSystem.getDefaultBlockSize(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override
|
||||
public short getDefaultReplication() {
|
||||
return fileSystem.getDefaultReplication();
|
||||
}
|
||||
|
||||
@Override
|
||||
public short getDefaultReplication(Path path) {
|
||||
return fileSystem.getDefaultReplication(convertToDefaultPath(path));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void access(Path path, FsAction mode) throws IOException {
|
||||
fileSystem.access(convertToDefaultPath(path), mode);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void createSymlink(Path target, Path link, boolean createParent) throws IOException {
|
||||
fileSystem
|
||||
.createSymlink(convertToDefaultPath(target), convertToDefaultPath(link), createParent);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileStatus getFileLinkStatus(Path f) throws IOException {
|
||||
return fileSystem.getFileLinkStatus(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean supportsSymlinks() {
|
||||
return fileSystem.supportsSymlinks();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Path getLinkTarget(Path f) throws IOException {
|
||||
return convertToHoodiePath(fileSystem.getLinkTarget(convertToDefaultPath(f)));
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileChecksum getFileChecksum(Path f) throws IOException {
|
||||
return fileSystem.getFileChecksum(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileChecksum getFileChecksum(Path f, long length) throws IOException {
|
||||
return fileSystem.getFileChecksum(convertToDefaultPath(f), length);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setVerifyChecksum(boolean verifyChecksum) {
|
||||
fileSystem.setVerifyChecksum(verifyChecksum);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setWriteChecksum(boolean writeChecksum) {
|
||||
fileSystem.setWriteChecksum(writeChecksum);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FsStatus getStatus() throws IOException {
|
||||
return fileSystem.getStatus();
|
||||
}
|
||||
|
||||
@Override
|
||||
public FsStatus getStatus(Path p) throws IOException {
|
||||
return fileSystem.getStatus(convertToDefaultPath(p));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setPermission(Path p, FsPermission permission) throws IOException {
|
||||
fileSystem.setPermission(convertToDefaultPath(p), permission);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setOwner(Path p, String username, String groupname) throws IOException {
|
||||
fileSystem.setOwner(convertToDefaultPath(p), username, groupname);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setTimes(Path p, long mtime, long atime) throws IOException {
|
||||
fileSystem.setTimes(convertToDefaultPath(p), mtime, atime);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Path createSnapshot(Path path, String snapshotName) throws IOException {
|
||||
return convertToHoodiePath(fileSystem.createSnapshot(convertToDefaultPath(path), snapshotName));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void renameSnapshot(Path path, String snapshotOldName, String snapshotNewName)
|
||||
throws IOException {
|
||||
fileSystem.renameSnapshot(convertToDefaultPath(path), snapshotOldName, snapshotNewName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void deleteSnapshot(Path path, String snapshotName) throws IOException {
|
||||
fileSystem.deleteSnapshot(convertToDefaultPath(path), snapshotName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void modifyAclEntries(Path path, List<AclEntry> aclSpec) throws IOException {
|
||||
fileSystem.modifyAclEntries(convertToDefaultPath(path), aclSpec);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removeAclEntries(Path path, List<AclEntry> aclSpec) throws IOException {
|
||||
fileSystem.removeAclEntries(convertToDefaultPath(path), aclSpec);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removeDefaultAcl(Path path) throws IOException {
|
||||
fileSystem.removeDefaultAcl(convertToDefaultPath(path));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removeAcl(Path path) throws IOException {
|
||||
fileSystem.removeAcl(convertToDefaultPath(path));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setAcl(Path path, List<AclEntry> aclSpec) throws IOException {
|
||||
fileSystem.setAcl(convertToDefaultPath(path), aclSpec);
|
||||
}
|
||||
|
||||
@Override
|
||||
public AclStatus getAclStatus(Path path) throws IOException {
|
||||
return fileSystem.getAclStatus(convertToDefaultPath(path));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setXAttr(Path path, String name, byte[] value) throws IOException {
|
||||
fileSystem.setXAttr(convertToDefaultPath(path), name, value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setXAttr(Path path, String name, byte[] value, EnumSet<XAttrSetFlag> flag)
|
||||
throws IOException {
|
||||
fileSystem.setXAttr(convertToDefaultPath(path), name, value, flag);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getXAttr(Path path, String name) throws IOException {
|
||||
return fileSystem.getXAttr(convertToDefaultPath(path), name);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, byte[]> getXAttrs(Path path) throws IOException {
|
||||
return fileSystem.getXAttrs(convertToDefaultPath(path));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, byte[]> getXAttrs(Path path, List<String> names) throws IOException {
|
||||
return fileSystem.getXAttrs(convertToDefaultPath(path), names);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> listXAttrs(Path path) throws IOException {
|
||||
return fileSystem.listXAttrs(convertToDefaultPath(path));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removeXAttr(Path path, String name) throws IOException {
|
||||
fileSystem.removeXAttr(convertToDefaultPath(path), name);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Configuration getConf() {
|
||||
return fileSystem.getConf();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setConf(Configuration conf) {
|
||||
// ignore this. we will set conf on init
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return fileSystem.hashCode();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
return fileSystem.equals(obj);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return fileSystem.toString();
|
||||
}
|
||||
|
||||
public Path convertToHoodiePath(Path oldPath) {
|
||||
return convertPathWithScheme(oldPath, getHoodieScheme(fileSystem.getScheme()));
|
||||
}
|
||||
|
||||
private Path convertToDefaultPath(Path oldPath) {
|
||||
return convertPathWithScheme(oldPath, fileSystem.getScheme());
|
||||
}
|
||||
|
||||
private Path[] convertDefaults(Path[] psrcs) {
|
||||
Path[] psrcsNew = new Path[psrcs.length];
|
||||
for (int i = 0; i < psrcs.length; i++) {
|
||||
psrcsNew[i] = convertToDefaultPath(psrcs[i]);
|
||||
}
|
||||
return psrcsNew;
|
||||
}
|
||||
|
||||
public long getBytesWritten(Path file) {
|
||||
if (openStreams.containsKey(file.getName())) {
|
||||
return openStreams.get(file.getName()).getBytesWritten();
|
||||
}
|
||||
// When the file is first written, we do not have a track of it
|
||||
throw new IllegalArgumentException(file.toString()
|
||||
+ " does not have a open stream. Cannot get the bytes written on the stream");
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,77 @@
|
||||
/*
|
||||
* Copyright (c) 2016 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.io.storage;
|
||||
|
||||
import com.uber.hoodie.common.util.ConsistencyGuard;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
/**
|
||||
* Wrapper over <code>FSDataOutputStream</code> to keep track of the size of the written bytes. This
|
||||
* gives a cheap way to check on the underlying file size.
|
||||
*/
|
||||
public class SizeAwareFSDataOutputStream extends FSDataOutputStream {
|
||||
|
||||
// A callback to call when the output stream is closed.
|
||||
private final Runnable closeCallback;
|
||||
// Keep track of the bytes written
|
||||
private final AtomicLong bytesWritten = new AtomicLong(0L);
|
||||
// Path
|
||||
private final Path path;
|
||||
// Consistency guard
|
||||
private final ConsistencyGuard consistencyGuard;
|
||||
|
||||
public SizeAwareFSDataOutputStream(Path path, FSDataOutputStream out,
|
||||
ConsistencyGuard consistencyGuard, Runnable closeCallback) throws IOException {
|
||||
super(out);
|
||||
this.path = path;
|
||||
this.closeCallback = closeCallback;
|
||||
this.consistencyGuard = consistencyGuard;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void write(byte[] b, int off, int len) throws IOException {
|
||||
bytesWritten.addAndGet(len);
|
||||
super.write(b, off, len);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(byte[] b) throws IOException {
|
||||
bytesWritten.addAndGet(b.length);
|
||||
super.write(b);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
super.close();
|
||||
try {
|
||||
consistencyGuard.waitTillFileAppears(path);
|
||||
} catch (TimeoutException e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
closeCallback.run();
|
||||
}
|
||||
|
||||
public long getBytesWritten() {
|
||||
return bytesWritten.get();
|
||||
}
|
||||
|
||||
}
|
||||
@@ -59,7 +59,7 @@ public class FileSlice implements Serializable {
|
||||
this.fileGroupId = fileGroupId;
|
||||
this.baseInstantTime = baseInstantTime;
|
||||
this.dataFile = null;
|
||||
this.logFiles = new TreeSet<>(HoodieLogFile.getBaseInstantAndLogVersionComparator());
|
||||
this.logFiles = new TreeSet<>(HoodieLogFile.getReverseLogFileComparator());
|
||||
}
|
||||
|
||||
public void setDataFile(HoodieDataFile dataFile) {
|
||||
@@ -94,6 +94,10 @@ public class FileSlice implements Serializable {
|
||||
return Optional.ofNullable(dataFile);
|
||||
}
|
||||
|
||||
public Optional<HoodieLogFile> getLatestLogFile() {
|
||||
return logFiles.stream().findFirst();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns true if there is no data file and no log files. Happens as part of pending compaction
|
||||
* @return
|
||||
|
||||
@@ -72,6 +72,10 @@ public class HoodieLogFile implements Serializable {
|
||||
return FSUtils.getFileVersionFromLog(getPath());
|
||||
}
|
||||
|
||||
public String getLogWriteToken() {
|
||||
return FSUtils.getWriteTokenFromLogPath(getPath());
|
||||
}
|
||||
|
||||
public String getFileExtension() {
|
||||
return FSUtils.getFileExtensionFromLog(getPath());
|
||||
}
|
||||
@@ -96,7 +100,11 @@ public class HoodieLogFile implements Serializable {
|
||||
return fileStatus;
|
||||
}
|
||||
|
||||
public HoodieLogFile rollOver(FileSystem fs) throws IOException {
|
||||
public void setFileStatus(FileStatus fileStatus) {
|
||||
this.fileStatus = fileStatus;
|
||||
}
|
||||
|
||||
public HoodieLogFile rollOver(FileSystem fs, String logWriteToken) throws IOException {
|
||||
String fileId = getFileId();
|
||||
String baseCommitTime = getBaseCommitTime();
|
||||
Path path = getPath();
|
||||
@@ -105,28 +113,50 @@ public class HoodieLogFile implements Serializable {
|
||||
.computeNextLogVersion(fs, path.getParent(), fileId,
|
||||
extension, baseCommitTime);
|
||||
return new HoodieLogFile(new Path(path.getParent(),
|
||||
FSUtils.makeLogFileName(fileId, extension, baseCommitTime, newVersion)));
|
||||
FSUtils.makeLogFileName(fileId, extension, baseCommitTime, newVersion, logWriteToken)));
|
||||
}
|
||||
|
||||
public static Comparator<HoodieLogFile> getBaseInstantAndLogVersionComparator() {
|
||||
return new BaseInstantAndLogVersionComparator();
|
||||
public static Comparator<HoodieLogFile> getLogFileComparator() {
|
||||
return new LogFileComparator();
|
||||
}
|
||||
|
||||
public static Comparator<HoodieLogFile> getReverseLogFileComparator() {
|
||||
return new LogFileComparator().reversed();
|
||||
}
|
||||
|
||||
/**
|
||||
* Comparator to order log-files
|
||||
*/
|
||||
private static class BaseInstantAndLogVersionComparator implements Comparator<HoodieLogFile>, Serializable {
|
||||
public static class LogFileComparator implements Comparator<HoodieLogFile>, Serializable {
|
||||
|
||||
private transient Comparator<HoodieLogFile> writeTokenComparator;
|
||||
|
||||
private Comparator<HoodieLogFile> getWriteTokenComparator() {
|
||||
if (null == writeTokenComparator) {
|
||||
// writeTokenComparator is not serializable. Hence, lazy loading
|
||||
writeTokenComparator = Comparator.nullsFirst(Comparator.comparing(HoodieLogFile::getLogWriteToken));
|
||||
}
|
||||
return writeTokenComparator;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compare(HoodieLogFile o1, HoodieLogFile o2) {
|
||||
String baseInstantTime1 = o1.getBaseCommitTime();
|
||||
String baseInstantTime2 = o2.getBaseCommitTime();
|
||||
|
||||
if (baseInstantTime1.equals(baseInstantTime2)) {
|
||||
// reverse the order by log-version when base-commit is same
|
||||
return new Integer(o2.getLogVersion()).compareTo(o1.getLogVersion());
|
||||
|
||||
if (o1.getLogVersion() == o2.getLogVersion()) {
|
||||
// Compare by write token when base-commit and log-version is same
|
||||
return getWriteTokenComparator().compare(o1, o2);
|
||||
}
|
||||
|
||||
// compare by log-version when base-commit is same
|
||||
return Integer.compare(o1.getLogVersion(), o2.getLogVersion());
|
||||
}
|
||||
// reverse the order by base-commits
|
||||
return baseInstantTime2.compareTo(baseInstantTime1);
|
||||
|
||||
// compare by base-commits
|
||||
return baseInstantTime1.compareTo(baseInstantTime2);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -310,11 +310,8 @@ public class HoodieWriteStat implements Serializable {
|
||||
/**
|
||||
* Set path and tempPath relative to the given basePath.
|
||||
*/
|
||||
public void setPaths(Path basePath, Path path, Path tempPath) {
|
||||
public void setPath(Path basePath, Path path) {
|
||||
this.path = path.toString().replace(basePath + "/", "");
|
||||
if (tempPath != null) {
|
||||
this.tempPath = tempPath.toString().replace(basePath + "/", "");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -61,6 +61,7 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
public static String METAFOLDER_NAME = ".hoodie";
|
||||
public static String TEMPFOLDER_NAME = METAFOLDER_NAME + File.separator + ".temp";
|
||||
public static String AUXILIARYFOLDER_NAME = METAFOLDER_NAME + File.separator + ".aux";
|
||||
public static final String MARKER_EXTN = ".marker";
|
||||
|
||||
private String basePath;
|
||||
private transient FileSystem fs;
|
||||
@@ -142,6 +143,22 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
return metaPath;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Temp Folder path
|
||||
*/
|
||||
public String getTempFolderPath() {
|
||||
return basePath + File.separator + TEMPFOLDER_NAME;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns Marker folder path
|
||||
* @param instantTs Instant Timestamp
|
||||
* @return
|
||||
*/
|
||||
public String getMarkerFolderPath(String instantTs) {
|
||||
return String.format("%s%s%s", getTempFolderPath(), File.separator, instantTs);
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Auxiliary Meta path
|
||||
*/
|
||||
|
||||
@@ -19,9 +19,11 @@ package com.uber.hoodie.common.table.log;
|
||||
import com.uber.hoodie.common.model.HoodieLogFile;
|
||||
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.collection.Pair;
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
import java.util.Optional;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
@@ -48,6 +50,8 @@ public interface HoodieLogFormat {
|
||||
*/
|
||||
int currentVersion = 1;
|
||||
|
||||
String UNKNOWN_WRITE_TOKEN = "1-0-1";
|
||||
|
||||
/**
|
||||
* Writer interface to allow appending block to this file format
|
||||
*/
|
||||
@@ -106,6 +110,10 @@ public interface HoodieLogFormat {
|
||||
private Integer logVersion;
|
||||
// Location of the directory containing the log
|
||||
private Path parentPath;
|
||||
// Log File Write Token
|
||||
private String logWriteToken;
|
||||
// Rollover Log file write token
|
||||
private String rolloverLogWriteToken;
|
||||
|
||||
public WriterBuilder withBufferSize(int bufferSize) {
|
||||
this.bufferSize = bufferSize;
|
||||
@@ -117,6 +125,16 @@ public interface HoodieLogFormat {
|
||||
return this;
|
||||
}
|
||||
|
||||
public WriterBuilder withLogWriteToken(String writeToken) {
|
||||
this.logWriteToken = writeToken;
|
||||
return this;
|
||||
}
|
||||
|
||||
public WriterBuilder withRolloverLogWriteToken(String rolloverLogWriteToken) {
|
||||
this.rolloverLogWriteToken = rolloverLogWriteToken;
|
||||
return this;
|
||||
}
|
||||
|
||||
public WriterBuilder withFs(FileSystem fs) {
|
||||
this.fs = fs;
|
||||
return this;
|
||||
@@ -169,17 +187,37 @@ public interface HoodieLogFormat {
|
||||
if (parentPath == null) {
|
||||
throw new IllegalArgumentException("Log file parent location is not specified");
|
||||
}
|
||||
|
||||
if (rolloverLogWriteToken == null) {
|
||||
rolloverLogWriteToken = UNKNOWN_WRITE_TOKEN;
|
||||
}
|
||||
|
||||
if (logVersion == null) {
|
||||
log.info("Computing the next log version for " + logFileId + " in " + parentPath);
|
||||
logVersion =
|
||||
FSUtils.getCurrentLogVersion(fs, parentPath, logFileId, fileExtension, commitTime);
|
||||
Optional<Pair<Integer, String>> versionAndWriteToken =
|
||||
FSUtils.getLatestLogVersion(fs, parentPath, logFileId, fileExtension, commitTime);
|
||||
if (versionAndWriteToken.isPresent()) {
|
||||
logVersion = versionAndWriteToken.get().getKey();
|
||||
logWriteToken = versionAndWriteToken.get().getValue();
|
||||
} else {
|
||||
logVersion = HoodieLogFile.LOGFILE_BASE_VERSION;
|
||||
// this is the case where there is no existing log-file.
|
||||
// Use rollover write token as write token to create new log file with tokens
|
||||
logWriteToken = rolloverLogWriteToken;
|
||||
}
|
||||
log.info(
|
||||
"Computed the next log version for " + logFileId + " in " + parentPath + " as "
|
||||
+ logVersion);
|
||||
+ logVersion + " with write-token " + logWriteToken);
|
||||
}
|
||||
|
||||
if (logWriteToken == null) {
|
||||
// This is the case where we have existing log-file with old format. rollover to avoid any conflicts
|
||||
logVersion += 1;
|
||||
logWriteToken = rolloverLogWriteToken;
|
||||
}
|
||||
|
||||
Path logPath = new Path(parentPath,
|
||||
FSUtils.makeLogFileName(logFileId, fileExtension, commitTime, logVersion));
|
||||
FSUtils.makeLogFileName(logFileId, fileExtension, commitTime, logVersion, logWriteToken));
|
||||
log.info("HoodieLogFile on path " + logPath);
|
||||
HoodieLogFile logFile = new HoodieLogFile(logPath);
|
||||
|
||||
@@ -192,9 +230,9 @@ public interface HoodieLogFormat {
|
||||
if (sizeThreshold == null) {
|
||||
sizeThreshold = DEFAULT_SIZE_THRESHOLD;
|
||||
}
|
||||
return new HoodieLogFormatWriter(fs, logFile, bufferSize, replication, sizeThreshold);
|
||||
return new HoodieLogFormatWriter(fs, logFile, bufferSize, replication, sizeThreshold, logWriteToken,
|
||||
rolloverLogWriteToken);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
static WriterBuilder newWriterBuilder() {
|
||||
|
||||
@@ -48,6 +48,8 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
|
||||
private final long sizeThreshold;
|
||||
private final Integer bufferSize;
|
||||
private final Short replication;
|
||||
private final String logWriteToken;
|
||||
private final String rolloverLogWriteToken;
|
||||
private FSDataOutputStream output;
|
||||
private static final String APPEND_UNAVAILABLE_EXCEPTION_MESSAGE = "not sufficiently replicated yet";
|
||||
|
||||
@@ -59,14 +61,15 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
|
||||
* @param sizeThreshold
|
||||
*/
|
||||
HoodieLogFormatWriter(FileSystem fs, HoodieLogFile logFile, Integer bufferSize,
|
||||
Short replication, Long sizeThreshold)
|
||||
Short replication, Long sizeThreshold, String logWriteToken, String rolloverLogWriteToken)
|
||||
throws IOException, InterruptedException {
|
||||
this.fs = fs;
|
||||
this.logFile = logFile;
|
||||
this.sizeThreshold = sizeThreshold;
|
||||
this.bufferSize = bufferSize;
|
||||
this.replication = replication;
|
||||
|
||||
this.logWriteToken = logWriteToken;
|
||||
this.rolloverLogWriteToken = rolloverLogWriteToken;
|
||||
Path path = logFile.getPath();
|
||||
if (fs.exists(path)) {
|
||||
boolean isAppendSupported = StorageSchemes.isAppendSupported(fs.getScheme());
|
||||
@@ -87,7 +90,7 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
|
||||
}
|
||||
}
|
||||
if (!isAppendSupported) {
|
||||
this.logFile = logFile.rollOver(fs);
|
||||
this.logFile = logFile.rollOver(fs, rolloverLogWriteToken);
|
||||
log.info("Append not supported.. Rolling over to " + logFile);
|
||||
createNewFile();
|
||||
}
|
||||
@@ -180,10 +183,11 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
|
||||
// file).
|
||||
log.info("CurrentSize " + getCurrentSize() + " has reached threshold " + sizeThreshold
|
||||
+ ". Rolling over to the next version");
|
||||
HoodieLogFile newLogFile = logFile.rollOver(fs);
|
||||
HoodieLogFile newLogFile = logFile.rollOver(fs, rolloverLogWriteToken);
|
||||
// close this writer and return the new writer
|
||||
close();
|
||||
return new HoodieLogFormatWriter(fs, newLogFile, bufferSize, replication, sizeThreshold);
|
||||
return new HoodieLogFormatWriter(fs, newLogFile, bufferSize, replication, sizeThreshold, logWriteToken,
|
||||
rolloverLogWriteToken);
|
||||
}
|
||||
return this;
|
||||
}
|
||||
@@ -231,10 +235,15 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
|
||||
// last block. Find more information here : https://issues.apache.org/jira/browse/HDFS-6325
|
||||
log.warn("Failed to open an append stream to the log file. Opening a new log file..", e);
|
||||
// Rollover the current log file (since cannot get a stream handle) and create new one
|
||||
this.logFile = logFile.rollOver(fs);
|
||||
this.logFile = logFile.rollOver(fs, rolloverLogWriteToken);
|
||||
createNewFile();
|
||||
} else if ((e.getClassName().contentEquals(AlreadyBeingCreatedException.class.getName()) || e.getClassName()
|
||||
.contentEquals(RecoveryInProgressException.class.getName())) && (fs instanceof DistributedFileSystem)) {
|
||||
} else if (e.getClassName().contentEquals(AlreadyBeingCreatedException.class.getName())) {
|
||||
log.warn("Another task executor writing to the same log file(" + logFile + ". Rolling over");
|
||||
// Rollover the current log file (since cannot get a stream handle) and create new one
|
||||
this.logFile = logFile.rollOver(fs, rolloverLogWriteToken);
|
||||
createNewFile();
|
||||
} else if (e.getClassName().contentEquals(RecoveryInProgressException.class.getName())
|
||||
&& (fs instanceof DistributedFileSystem)) {
|
||||
// this happens when either another task executor writing to this file died or
|
||||
// data node is going down. Note that we can only try to recover lease for a DistributedFileSystem.
|
||||
// ViewFileSystem unfortunately does not support this operation
|
||||
|
||||
@@ -0,0 +1,89 @@
|
||||
/*
|
||||
* Copyright (c) 2019 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.util;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
/**
|
||||
* Ensures file create/delete operation is visible
|
||||
*/
|
||||
public interface ConsistencyGuard {
|
||||
|
||||
/**
|
||||
* File Visibility
|
||||
*/
|
||||
enum FileVisibility {
|
||||
APPEAR,
|
||||
DISAPPEAR,
|
||||
}
|
||||
|
||||
/**
|
||||
* Wait for file to be listable based on configurable timeout
|
||||
* @param filePath
|
||||
* @throws IOException when having trouble listing the path
|
||||
* @throws TimeoutException when retries exhausted
|
||||
*/
|
||||
void waitTillFileAppears(Path filePath) throws IOException, TimeoutException;
|
||||
|
||||
/**
|
||||
* Wait for file to be listable based on configurable timeout
|
||||
* @param filePath
|
||||
* @throws IOException when having trouble listing the path
|
||||
* @throws TimeoutException when retries exhausted
|
||||
*/
|
||||
void waitTillFileDisappears(Path filePath) throws IOException, TimeoutException;
|
||||
|
||||
/**
|
||||
* Wait till all passed files belonging to a directory shows up in the listing
|
||||
*/
|
||||
void waitTillAllFilesAppear(String dirPath, List<String> files) throws IOException, TimeoutException;
|
||||
|
||||
/**
|
||||
* Wait till all passed files belonging to a directory disappears from listing
|
||||
*/
|
||||
void waitTillAllFilesDisappear(String dirPath, List<String> files) throws IOException, TimeoutException;
|
||||
|
||||
|
||||
/**
|
||||
* Wait Till target visibility is reached
|
||||
* @param dirPath Directory Path
|
||||
* @param files Files
|
||||
* @param targetVisibility Target Visibitlity
|
||||
* @throws IOException
|
||||
* @throws TimeoutException
|
||||
*/
|
||||
default void waitTill(String dirPath, List<String> files, FileVisibility targetVisibility)
|
||||
throws IOException, TimeoutException {
|
||||
switch (targetVisibility) {
|
||||
case APPEAR: {
|
||||
waitTillAllFilesAppear(dirPath, files);
|
||||
break;
|
||||
}
|
||||
case DISAPPEAR: {
|
||||
waitTillAllFilesDisappear(dirPath, files);
|
||||
break;
|
||||
}
|
||||
default:
|
||||
throw new IllegalStateException("Unknown File Visibility");
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -16,21 +16,28 @@
|
||||
|
||||
package com.uber.hoodie.common.util;
|
||||
|
||||
import static com.uber.hoodie.common.table.HoodieTableMetaClient.MARKER_EXTN;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.uber.hoodie.common.model.HoodieFileFormat;
|
||||
import com.uber.hoodie.common.model.HoodieLogFile;
|
||||
import com.uber.hoodie.common.model.HoodiePartitionMetadata;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.util.collection.Pair;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import com.uber.hoodie.exception.InvalidHoodiePathException;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Optional;
|
||||
import java.util.UUID;
|
||||
import java.util.function.Function;
|
||||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
import java.util.stream.Stream;
|
||||
@@ -52,13 +59,15 @@ public class FSUtils {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(FSUtils.class);
|
||||
// Log files are of this pattern - .b5068208-e1a4-11e6-bf01-fe55135034f3_20170101134598.log.1
|
||||
private static final Pattern LOG_FILE_PATTERN = Pattern.compile("\\.(.*)_(.*)\\.(.*)\\.([0-9]*)");
|
||||
private static final Pattern LOG_FILE_PATTERN =
|
||||
Pattern.compile("\\.(.*)_(.*)\\.(.*)\\.([0-9]*)(_(([0-9]*)-([0-9]*)-([0-9]*)))?");
|
||||
private static final String LOG_FILE_PREFIX = ".";
|
||||
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_";
|
||||
|
||||
|
||||
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());
|
||||
@@ -76,7 +85,6 @@ public class FSUtils {
|
||||
return conf;
|
||||
}
|
||||
|
||||
|
||||
public static FileSystem getFs(String path, Configuration conf) {
|
||||
FileSystem fs;
|
||||
conf = prepareHadoopConf(conf);
|
||||
@@ -92,26 +100,36 @@ public class FSUtils {
|
||||
return fs;
|
||||
}
|
||||
|
||||
public static String makeDataFileName(String commitTime, int taskPartitionId, String fileId) {
|
||||
return String.format("%s_%d_%s.parquet", fileId, taskPartitionId, commitTime);
|
||||
/**
|
||||
* A write token uniquely identifies an attempt at one of the IOHandle operations (Merge/Create/Append)
|
||||
*/
|
||||
public static String makeWriteToken(int taskPartitionId, int stageId, long taskAttemptId) {
|
||||
return String.format("%d-%d-%d", taskPartitionId, stageId, taskAttemptId);
|
||||
}
|
||||
|
||||
public static String makeTempDataFileName(String partitionPath, String commitTime,
|
||||
int taskPartitionId, String fileId, int stageId, long taskAttemptId) {
|
||||
return String.format("%s_%s_%d_%s_%d_%d.parquet", partitionPath.replace("/", "-"), fileId,
|
||||
taskPartitionId, commitTime, stageId, taskAttemptId);
|
||||
|
||||
public static String makeDataFileName(String commitTime, String writeToken, String fileId) {
|
||||
return String.format("%s_%s_%s.parquet", fileId, writeToken, commitTime);
|
||||
}
|
||||
|
||||
public static String makeMarkerFile(String commitTime, String writeToken, String fileId) {
|
||||
return String.format("%s_%s_%s%s", fileId, writeToken, commitTime, MARKER_EXTN);
|
||||
}
|
||||
|
||||
public static String translateMarkerToDataPath(String basePath, String markerPath, String instantTs) {
|
||||
Preconditions.checkArgument(markerPath.endsWith(MARKER_EXTN));
|
||||
String markerRootPath = Path.getPathWithoutSchemeAndAuthority(new Path(
|
||||
String.format("%s/%s/%s", basePath, HoodieTableMetaClient.TEMPFOLDER_NAME, instantTs))).toString();
|
||||
int begin = markerPath.indexOf(markerRootPath);
|
||||
Preconditions.checkArgument(begin >= 0, "Not in marker dir. Marker Path=" + markerPath
|
||||
+ ", Expected Marker Root=" + markerRootPath);
|
||||
String rPath = markerPath.substring(begin + markerRootPath.length() + 1);
|
||||
return String.format("%s/%s%s", basePath, rPath.replace(MARKER_EXTN, ""),
|
||||
HoodieFileFormat.PARQUET.getFileExtension());
|
||||
}
|
||||
|
||||
public static String maskWithoutFileId(String commitTime, int taskPartitionId) {
|
||||
return String.format("*_%s_%s.parquet", taskPartitionId, commitTime);
|
||||
}
|
||||
|
||||
public static String maskWithoutTaskPartitionId(String commitTime, String fileId) {
|
||||
return String.format("%s_*_%s.parquet", fileId, commitTime);
|
||||
}
|
||||
|
||||
public static String maskWithOnlyCommitTime(String commitTime) {
|
||||
return String.format("*_*_%s.parquet", commitTime);
|
||||
return String.format("*_%s_%s%s", taskPartitionId, commitTime, HoodieFileFormat.PARQUET.getFileExtension());
|
||||
}
|
||||
|
||||
public static String getCommitFromCommitFile(String commitFileName) {
|
||||
@@ -175,18 +193,43 @@ public class FSUtils {
|
||||
*/
|
||||
public static List<String> getAllFoldersWithPartitionMetaFile(FileSystem fs, String basePathStr)
|
||||
throws IOException {
|
||||
List<String> partitions = new ArrayList<>();
|
||||
Path basePath = new Path(basePathStr);
|
||||
RemoteIterator<LocatedFileStatus> allFiles = fs.listFiles(new Path(basePathStr), true);
|
||||
while (allFiles.hasNext()) {
|
||||
Path filePath = allFiles.next().getPath();
|
||||
final Path basePath = new Path(basePathStr);
|
||||
final List<String> partitions = new ArrayList<>();
|
||||
processFiles(fs, basePathStr, (locatedFileStatus) -> {
|
||||
Path filePath = locatedFileStatus.getPath();
|
||||
if (filePath.getName().equals(HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE)) {
|
||||
partitions.add(getRelativePartitionPath(basePath, filePath.getParent()));
|
||||
}
|
||||
}
|
||||
return true;
|
||||
});
|
||||
return partitions;
|
||||
}
|
||||
|
||||
public static final List<String> getAllDataFilesForMarkers(FileSystem fs, String basePath, String instantTs,
|
||||
String markerDir) throws IOException {
|
||||
List<String> dataFiles = new LinkedList<>();
|
||||
FSUtils.processFiles(fs, markerDir, (status) -> {
|
||||
String pathStr = status.getPath().toString();
|
||||
if (pathStr.endsWith(MARKER_EXTN)) {
|
||||
dataFiles.add(FSUtils.translateMarkerToDataPath(basePath, pathStr, instantTs));
|
||||
}
|
||||
return true;
|
||||
});
|
||||
return dataFiles;
|
||||
}
|
||||
|
||||
private static final void processFiles(FileSystem fs, String basePathStr,
|
||||
Function<LocatedFileStatus, Boolean> consumer) throws IOException {
|
||||
RemoteIterator<LocatedFileStatus> allFiles = fs.listFiles(new Path(basePathStr), true);
|
||||
while (allFiles.hasNext()) {
|
||||
LocatedFileStatus status = allFiles.next();
|
||||
boolean success = consumer.apply(status);
|
||||
if (!success) {
|
||||
throw new HoodieException("Failed to process file-status=" + status);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static List<String> getAllPartitionPaths(FileSystem fs, String basePathStr,
|
||||
boolean assumeDatePartitioning)
|
||||
throws IOException {
|
||||
@@ -208,6 +251,12 @@ public class FSUtils {
|
||||
return name.replace(getFileExtension(name), "");
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a new unique prefix for creating a file group.
|
||||
*/
|
||||
public static String createNewFileIdPfx() {
|
||||
return UUID.randomUUID().toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the file extension from the log file
|
||||
@@ -254,6 +303,53 @@ public class FSUtils {
|
||||
return matcher.group(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get TaskId used in log-path
|
||||
*/
|
||||
public static Integer getTaskPartitionIdFromLogPath(Path path) {
|
||||
Matcher matcher = LOG_FILE_PATTERN.matcher(path.getName());
|
||||
if (!matcher.find()) {
|
||||
throw new InvalidHoodiePathException(path, "LogFile");
|
||||
}
|
||||
String val = matcher.group(7);
|
||||
return val == null ? null : Integer.parseInt(val);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get Write-Token used in log-path
|
||||
*/
|
||||
public static String getWriteTokenFromLogPath(Path path) {
|
||||
Matcher matcher = LOG_FILE_PATTERN.matcher(path.getName());
|
||||
if (!matcher.find()) {
|
||||
throw new InvalidHoodiePathException(path, "LogFile");
|
||||
}
|
||||
return matcher.group(6);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get StageId used in log-path
|
||||
*/
|
||||
public static Integer getStageIdFromLogPath(Path path) {
|
||||
Matcher matcher = LOG_FILE_PATTERN.matcher(path.getName());
|
||||
if (!matcher.find()) {
|
||||
throw new InvalidHoodiePathException(path, "LogFile");
|
||||
}
|
||||
String val = matcher.group(8);
|
||||
return val == null ? null : Integer.parseInt(val);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get Task Attempt Id used in log-path
|
||||
*/
|
||||
public static Integer getTaskAttemptIdFromLogPath(Path path) {
|
||||
Matcher matcher = LOG_FILE_PATTERN.matcher(path.getName());
|
||||
if (!matcher.find()) {
|
||||
throw new InvalidHoodiePathException(path, "LogFile");
|
||||
}
|
||||
String val = matcher.group(9);
|
||||
return val == null ? null : Integer.parseInt(val);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the last part of the file name in the log file and convert to int.
|
||||
*/
|
||||
@@ -266,14 +362,10 @@ public class FSUtils {
|
||||
}
|
||||
|
||||
public static String makeLogFileName(String fileId, String logFileExtension,
|
||||
String baseCommitTime, int version) {
|
||||
return LOG_FILE_PREFIX + String
|
||||
.format("%s_%s%s.%d", fileId, baseCommitTime, logFileExtension, version);
|
||||
}
|
||||
|
||||
public static String maskWithoutLogVersion(String commitTime, String fileId,
|
||||
String logFileExtension) {
|
||||
return LOG_FILE_PREFIX + String.format("%s_%s%s*", fileId, commitTime, logFileExtension);
|
||||
String baseCommitTime, int version, String writeToken) {
|
||||
String suffix = (writeToken == null) ? String.format("%s_%s%s.%d",fileId, baseCommitTime, logFileExtension, version)
|
||||
: String.format("%s_%s%s.%d_%s", fileId, baseCommitTime, logFileExtension, version, writeToken);
|
||||
return LOG_FILE_PREFIX + suffix;
|
||||
}
|
||||
|
||||
public static boolean isLogFile(Path logPath) {
|
||||
@@ -288,9 +380,7 @@ public class FSUtils {
|
||||
* Get the latest log file written from the list of log files passed in
|
||||
*/
|
||||
public static Optional<HoodieLogFile> getLatestLogFile(Stream<HoodieLogFile> logFiles) {
|
||||
return logFiles.sorted(Comparator
|
||||
.comparing(s -> s.getLogVersion(),
|
||||
Comparator.reverseOrder())).findFirst();
|
||||
return logFiles.sorted(HoodieLogFile.getReverseLogFileComparator()).findFirst();
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -308,36 +398,28 @@ public class FSUtils {
|
||||
/**
|
||||
* Get the latest log version for the fileId in the partition path
|
||||
*/
|
||||
public static Optional<Integer> getLatestLogVersion(FileSystem fs, Path partitionPath,
|
||||
public static Optional<Pair<Integer, String>> getLatestLogVersion(FileSystem fs, Path partitionPath,
|
||||
final String fileId, final String logFileExtension, final String baseCommitTime)
|
||||
throws IOException {
|
||||
Optional<HoodieLogFile> latestLogFile =
|
||||
getLatestLogFile(
|
||||
getAllLogFiles(fs, partitionPath, fileId, logFileExtension, baseCommitTime));
|
||||
if (latestLogFile.isPresent()) {
|
||||
return Optional.of(latestLogFile.get().getLogVersion());
|
||||
return Optional.of(Pair.of(latestLogFile.get().getLogVersion(),
|
||||
getWriteTokenFromLogPath(latestLogFile.get().getPath())));
|
||||
}
|
||||
return Optional.empty();
|
||||
}
|
||||
|
||||
public static int getCurrentLogVersion(FileSystem fs, Path partitionPath,
|
||||
final String fileId, final String logFileExtension, final String baseCommitTime)
|
||||
throws IOException {
|
||||
Optional<Integer> currentVersion =
|
||||
getLatestLogVersion(fs, partitionPath, fileId, logFileExtension, baseCommitTime);
|
||||
// handle potential overflow
|
||||
return (currentVersion.isPresent()) ? currentVersion.get() : HoodieLogFile.LOGFILE_BASE_VERSION;
|
||||
}
|
||||
|
||||
/**
|
||||
* computes the next log version for the specified fileId in the partition path
|
||||
*/
|
||||
public static int computeNextLogVersion(FileSystem fs, Path partitionPath, final String fileId,
|
||||
final String logFileExtension, final String baseCommitTime) throws IOException {
|
||||
Optional<Integer> currentVersion =
|
||||
Optional<Pair<Integer, String>> currentVersionWithWriteToken =
|
||||
getLatestLogVersion(fs, partitionPath, fileId, logFileExtension, baseCommitTime);
|
||||
// handle potential overflow
|
||||
return (currentVersion.isPresent()) ? currentVersion.get() + 1
|
||||
return (currentVersionWithWriteToken.isPresent()) ? currentVersionWithWriteToken.get().getKey() + 1
|
||||
: HoodieLogFile.LOGFILE_BASE_VERSION;
|
||||
}
|
||||
|
||||
@@ -349,10 +431,6 @@ public class FSUtils {
|
||||
return fs.getDefaultReplication(path);
|
||||
}
|
||||
|
||||
public static Long getDefaultBlockSize(FileSystem fs, Path path) {
|
||||
return fs.getDefaultBlockSize(path);
|
||||
}
|
||||
|
||||
/**
|
||||
* When a file was opened and the task died without closing the stream, another task executor
|
||||
* cannot open because the existing lease will be active. We will try to recover the lease, from
|
||||
@@ -431,8 +509,12 @@ public class FSUtils {
|
||||
}
|
||||
|
||||
public static Path getPartitionPath(String basePath, String partitionPath) {
|
||||
return getPartitionPath(new Path(basePath), partitionPath);
|
||||
}
|
||||
|
||||
public static Path getPartitionPath(Path basePath, String partitionPath) {
|
||||
// FOr non-partitioned table, return only base-path
|
||||
return ((partitionPath == null) || (partitionPath.isEmpty())) ? new Path(basePath) :
|
||||
return ((partitionPath == null) || (partitionPath.isEmpty())) ? basePath :
|
||||
new Path(basePath, partitionPath);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,200 @@
|
||||
/*
|
||||
* Copyright (c) 2019 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.util;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* A consistency checker that fails if it is unable to meet the required condition within a specified timeout
|
||||
*/
|
||||
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;
|
||||
|
||||
public FailSafeConsistencyGuard(FileSystem fs, int maxAttempts, long initalDelayMs, long maxDelayMs) {
|
||||
this.fs = fs;
|
||||
this.maxAttempts = maxAttempts;
|
||||
this.initialDelayMs = initalDelayMs;
|
||||
this.maxDelayMs = maxDelayMs;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void waitTillFileAppears(Path filePath) throws TimeoutException {
|
||||
waitForFileVisibility(filePath, FileVisibility.APPEAR);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void waitTillFileDisappears(Path filePath)
|
||||
throws TimeoutException {
|
||||
waitForFileVisibility(filePath, FileVisibility.DISAPPEAR);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void waitTillAllFilesAppear(String dirPath, List<String> files) throws TimeoutException {
|
||||
waitForFilesVisibility(dirPath, files, FileVisibility.APPEAR);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void waitTillAllFilesDisappear(String dirPath, List<String> files) throws TimeoutException {
|
||||
waitForFilesVisibility(dirPath, files, FileVisibility.DISAPPEAR);
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper function to wait for all files belonging to single directory to appear
|
||||
* @param dirPath Dir Path
|
||||
* @param files Files to appear/disappear
|
||||
* @param event Appear/Disappear
|
||||
* @throws TimeoutException
|
||||
*/
|
||||
public void waitForFilesVisibility(String dirPath, List<String> files, FileVisibility event)
|
||||
throws TimeoutException {
|
||||
Path dir = new Path(dirPath);
|
||||
List<String> filesWithoutSchemeAndAuthority =
|
||||
files.stream().map(f -> Path.getPathWithoutSchemeAndAuthority(new Path(f))).map(p -> p.toString())
|
||||
.collect(Collectors.toList());
|
||||
|
||||
retryTillSuccess((retryNum) -> {
|
||||
try {
|
||||
log.info("Trying " + retryNum);
|
||||
FileStatus[] entries = fs.listStatus(dir);
|
||||
List<String> gotFiles = Arrays.stream(entries).map(e -> Path.getPathWithoutSchemeAndAuthority(e.getPath()))
|
||||
.map(p -> p.toString()).collect(Collectors.toList());
|
||||
List<String> candidateFiles = new ArrayList<>(filesWithoutSchemeAndAuthority);
|
||||
boolean altered = candidateFiles.removeAll(gotFiles);
|
||||
|
||||
switch (event) {
|
||||
case DISAPPEAR:
|
||||
log.info("Following files are visible" + candidateFiles);
|
||||
// If no candidate files gets removed, it means all of them have disappeared
|
||||
return !altered;
|
||||
case APPEAR:
|
||||
default:
|
||||
// if all files appear, the list is empty
|
||||
return candidateFiles.isEmpty();
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
log.warn("Got IOException waiting for file event. Have tried " + retryNum + " time(s)", ioe);
|
||||
}
|
||||
return false;
|
||||
}, "Timed out waiting for filles to become visible");
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper to check of file visibility
|
||||
* @param filePath File Path
|
||||
* @param visibility Visibility
|
||||
* @return
|
||||
* @throws IOException
|
||||
*/
|
||||
private boolean checkFileVisibility(Path filePath, FileVisibility visibility) throws IOException {
|
||||
try {
|
||||
FileStatus[] status = fs.listStatus(filePath);
|
||||
switch (visibility) {
|
||||
case APPEAR:
|
||||
return status.length != 0;
|
||||
case DISAPPEAR:
|
||||
default:
|
||||
return status.length == 0;
|
||||
}
|
||||
} catch (FileNotFoundException nfe) {
|
||||
switch (visibility) {
|
||||
case APPEAR:
|
||||
return false;
|
||||
case DISAPPEAR:
|
||||
default:
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper function to wait till file either appears/disappears
|
||||
* @param filePath File Path
|
||||
* @param visibility
|
||||
* @throws TimeoutException
|
||||
*/
|
||||
private void waitForFileVisibility(Path filePath, FileVisibility visibility) throws TimeoutException {
|
||||
long waitMs = initialDelayMs;
|
||||
int attempt = 0;
|
||||
while (attempt < maxAttempts) {
|
||||
try {
|
||||
if (checkFileVisibility(filePath, visibility)) {
|
||||
return;
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
log.warn("Got IOException waiting for file visibility. Retrying", ioe);
|
||||
}
|
||||
|
||||
sleepSafe(waitMs);
|
||||
waitMs = waitMs * 2; // double check interval every attempt
|
||||
waitMs = waitMs > maxDelayMs ? maxDelayMs : waitMs;
|
||||
attempt++;
|
||||
}
|
||||
throw new TimeoutException("Timed-out waiting for the file to " + visibility.name());
|
||||
}
|
||||
|
||||
/**
|
||||
* Retries the predicate for condfigurable number of times till we the predicate returns success
|
||||
* @param predicate Predicate Function
|
||||
* @param timedOutMessage Timed-Out message for logging
|
||||
* @throws TimeoutException when retries are exhausted
|
||||
*/
|
||||
private void retryTillSuccess(Function<Integer, Boolean> predicate, String timedOutMessage) throws TimeoutException {
|
||||
long waitMs = initialDelayMs;
|
||||
int attempt = 0;
|
||||
log.warn("Max Attempts=" + maxAttempts);
|
||||
while (attempt < maxAttempts) {
|
||||
boolean success = predicate.apply(attempt);
|
||||
if (success) {
|
||||
return;
|
||||
}
|
||||
sleepSafe(waitMs);
|
||||
waitMs = waitMs * 2; // double check interval every attempt
|
||||
waitMs = waitMs > maxDelayMs ? maxDelayMs : waitMs;
|
||||
attempt++;
|
||||
}
|
||||
throw new TimeoutException(timedOutMessage);
|
||||
|
||||
}
|
||||
|
||||
void sleepSafe(long waitMs) {
|
||||
try {
|
||||
Thread.sleep(waitMs);
|
||||
} catch (InterruptedException e) {
|
||||
// ignore & continue next attempt
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,46 @@
|
||||
/*
|
||||
* Copyright (c) 2019 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.util;
|
||||
|
||||
import java.util.List;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
/**
|
||||
* Default Consistency guard that does nothing. Used for HDFS deployments
|
||||
*/
|
||||
public class NoOpConsistencyGuard implements ConsistencyGuard {
|
||||
|
||||
@Override
|
||||
public void waitTillFileAppears(Path filePath) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void waitTillFileDisappears(Path filePath) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void waitTillAllFilesAppear(String dirPath, List<String> files) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void waitTillAllFilesDisappear(String dirPath, List<String> files) {
|
||||
|
||||
}
|
||||
}
|
||||
@@ -113,7 +113,6 @@ public class ParquetUtils {
|
||||
return footer;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Get the schema of the given parquet file.
|
||||
*/
|
||||
@@ -121,7 +120,6 @@ public class ParquetUtils {
|
||||
return readMetadata(configuration, parquetFilePath).getFileMetaData().getSchema();
|
||||
}
|
||||
|
||||
|
||||
private static List<String> readParquetFooter(Configuration configuration, Path parquetFilePath,
|
||||
String... footerNames) {
|
||||
List<String> footerVals = new ArrayList<>();
|
||||
|
||||
@@ -79,7 +79,8 @@ public class HoodieTestUtils {
|
||||
|
||||
public static final String TEST_EXTENSION = ".test";
|
||||
public static final String RAW_TRIPS_TEST_NAME = "raw_trips";
|
||||
public static final int DEFAULT_TASK_PARTITIONID = 1;
|
||||
public static final String DEFAULT_WRITE_TOKEN = "1-0-1";
|
||||
public static final int DEFAULT_LOG_VERSION = 1;
|
||||
public static final String[] DEFAULT_PARTITION_PATHS = {"2016/03/15", "2015/03/16", "2015/03/17"};
|
||||
private static Random rand = new Random(46474747);
|
||||
|
||||
@@ -92,8 +93,7 @@ public class HoodieTestUtils {
|
||||
return init(basePath, HoodieTableType.COPY_ON_WRITE);
|
||||
}
|
||||
|
||||
public static HoodieTableMetaClient init(String basePath, HoodieTableType tableType)
|
||||
throws IOException {
|
||||
public static HoodieTableMetaClient init(String basePath, HoodieTableType tableType) throws IOException {
|
||||
return initTableType(getDefaultHadoopConf(), basePath, tableType);
|
||||
}
|
||||
|
||||
@@ -163,14 +163,30 @@ public class HoodieTestUtils {
|
||||
return createDataFile(basePath, partitionPath, commitTime, fileID);
|
||||
}
|
||||
|
||||
public static final String createNewMarkerFile(String basePath, String partitionPath, String commitTime)
|
||||
throws IOException {
|
||||
String fileID = UUID.randomUUID().toString();
|
||||
return createMarkerFile(basePath, partitionPath, commitTime, fileID);
|
||||
}
|
||||
|
||||
public static final String createDataFile(String basePath, String partitionPath, String commitTime, String fileID)
|
||||
throws IOException {
|
||||
String folderPath = basePath + "/" + partitionPath + "/";
|
||||
new File(folderPath).mkdirs();
|
||||
new File(folderPath + FSUtils.makeDataFileName(commitTime, DEFAULT_TASK_PARTITIONID, fileID)).createNewFile();
|
||||
new File(folderPath + FSUtils.makeDataFileName(commitTime, DEFAULT_WRITE_TOKEN, fileID)).createNewFile();
|
||||
return fileID;
|
||||
}
|
||||
|
||||
public static final String createMarkerFile(String basePath, String partitionPath, String commitTime, String fileID)
|
||||
throws IOException {
|
||||
String folderPath = basePath + "/" + HoodieTableMetaClient.TEMPFOLDER_NAME + "/" + commitTime + "/"
|
||||
+ partitionPath + "/";
|
||||
new File(folderPath).mkdirs();
|
||||
File f = new File(folderPath + FSUtils.makeMarkerFile(commitTime, DEFAULT_WRITE_TOKEN, fileID));
|
||||
f.createNewFile();
|
||||
return f.getAbsolutePath();
|
||||
}
|
||||
|
||||
public static final String createNewLogFile(FileSystem fs, String basePath, String partitionPath, String commitTime,
|
||||
String fileID, Optional<Integer> version) throws IOException {
|
||||
String folderPath = basePath + "/" + partitionPath + "/";
|
||||
@@ -179,7 +195,9 @@ public class HoodieTestUtils {
|
||||
throw new IOException("cannot create directory for path " + folderPath);
|
||||
}
|
||||
boolean createFile = fs.createNewFile(new Path(
|
||||
folderPath + FSUtils.makeLogFileName(fileID, ".log", commitTime, version.orElse(DEFAULT_TASK_PARTITIONID))));
|
||||
folderPath + FSUtils
|
||||
.makeLogFileName(fileID, ".log", commitTime, version.orElse(DEFAULT_LOG_VERSION),
|
||||
HoodieLogFormat.UNKNOWN_WRITE_TOKEN)));
|
||||
if (!createFile) {
|
||||
throw new IOException(
|
||||
StringUtils.format("cannot create data file for commit %s and fileId %s", commitTime, fileID));
|
||||
@@ -208,39 +226,38 @@ public class HoodieTestUtils {
|
||||
AvroUtils.serializeCompactionPlan(plan));
|
||||
}
|
||||
|
||||
public static final String getDataFilePath(String basePath, String partitionPath, String commitTime, String fileID)
|
||||
throws IOException {
|
||||
public static final String getDataFilePath(String basePath, String partitionPath, String commitTime, String fileID) {
|
||||
return basePath + "/" + partitionPath + "/" + FSUtils
|
||||
.makeDataFileName(commitTime, DEFAULT_TASK_PARTITIONID, fileID);
|
||||
.makeDataFileName(commitTime, DEFAULT_WRITE_TOKEN, fileID);
|
||||
}
|
||||
|
||||
public static final String getLogFilePath(String basePath, String partitionPath, String commitTime, String fileID,
|
||||
Optional<Integer> version) throws IOException {
|
||||
Optional<Integer> version) {
|
||||
return basePath + "/" + partitionPath + "/" + FSUtils.makeLogFileName(fileID, ".log", commitTime,
|
||||
version.orElse(DEFAULT_TASK_PARTITIONID));
|
||||
version.orElse(DEFAULT_LOG_VERSION), HoodieLogFormat.UNKNOWN_WRITE_TOKEN);
|
||||
}
|
||||
|
||||
public static final String getCommitFilePath(String basePath, String commitTime) throws IOException {
|
||||
public static final String getCommitFilePath(String basePath, String commitTime) {
|
||||
return basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime + HoodieTimeline.COMMIT_EXTENSION;
|
||||
}
|
||||
|
||||
public static final String getInflightCommitFilePath(String basePath, String commitTime) throws IOException {
|
||||
public static final String getInflightCommitFilePath(String basePath, String commitTime) {
|
||||
return basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime
|
||||
+ HoodieTimeline.INFLIGHT_COMMIT_EXTENSION;
|
||||
}
|
||||
|
||||
public static final String getRequestedCompactionFilePath(String basePath, String commitTime) throws IOException {
|
||||
public static final String getRequestedCompactionFilePath(String basePath, String commitTime) {
|
||||
return basePath + "/" + HoodieTableMetaClient.AUXILIARYFOLDER_NAME + "/" + commitTime
|
||||
+ HoodieTimeline.INFLIGHT_COMMIT_EXTENSION;
|
||||
}
|
||||
|
||||
public static final boolean doesDataFileExist(String basePath, String partitionPath, String commitTime, String fileID)
|
||||
throws IOException {
|
||||
public static final boolean doesDataFileExist(String basePath, String partitionPath, String commitTime,
|
||||
String fileID) {
|
||||
return new File(getDataFilePath(basePath, partitionPath, commitTime, fileID)).exists();
|
||||
}
|
||||
|
||||
public static final boolean doesLogFileExist(String basePath, String partitionPath, String commitTime, String fileID,
|
||||
Optional<Integer> version) throws IOException {
|
||||
Optional<Integer> version) {
|
||||
return new File(getLogFilePath(basePath, partitionPath, commitTime, fileID, version)).exists();
|
||||
}
|
||||
|
||||
@@ -256,10 +273,6 @@ public class HoodieTestUtils {
|
||||
.exists();
|
||||
}
|
||||
|
||||
public static String makeInflightTestFileName(String instant) {
|
||||
return instant + TEST_EXTENSION + HoodieTimeline.INFLIGHT_EXTENSION;
|
||||
}
|
||||
|
||||
public static void createCleanFiles(String basePath, String commitTime, Configuration configuration)
|
||||
throws IOException {
|
||||
Path commitFile = new Path(
|
||||
@@ -395,4 +408,4 @@ public class HoodieTestUtils {
|
||||
}
|
||||
return writeStatList;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -35,27 +35,20 @@ public class TestHoodieWriteStat {
|
||||
String basePathString = "/data/tables/some-hoodie-table";
|
||||
String partitionPathString = "2017/12/31";
|
||||
String fileName = UUID.randomUUID().toString();
|
||||
int taskPartitionId = Integer.MAX_VALUE;
|
||||
int stageId = Integer.MAX_VALUE;
|
||||
long taskAttemptId = Long.MAX_VALUE;
|
||||
String writeToken = "1-0-1";
|
||||
|
||||
Path basePath = new Path(basePathString);
|
||||
Path partitionPath = new Path(basePath, partitionPathString);
|
||||
Path tempPath = new Path(basePath, HoodieTableMetaClient.TEMPFOLDER_NAME);
|
||||
|
||||
Path finalizeFilePath = new Path(partitionPath, FSUtils.makeDataFileName(commitTime, taskPartitionId, fileName));
|
||||
Path tempFilePath = new Path(tempPath, FSUtils
|
||||
.makeTempDataFileName(partitionPathString, commitTime, taskPartitionId,
|
||||
fileName, stageId, taskAttemptId));
|
||||
|
||||
Path finalizeFilePath = new Path(partitionPath, FSUtils.makeDataFileName(commitTime, writeToken, fileName));
|
||||
HoodieWriteStat writeStat = new HoodieWriteStat();
|
||||
writeStat.setPaths(basePath, finalizeFilePath, tempFilePath);
|
||||
writeStat.setPath(basePath, finalizeFilePath);
|
||||
assertEquals(finalizeFilePath, new Path(basePath, writeStat.getPath()));
|
||||
assertEquals(tempFilePath, new Path(basePath, writeStat.getTempPath()));
|
||||
|
||||
// test for null tempFilePath
|
||||
writeStat = new HoodieWriteStat();
|
||||
writeStat.setPaths(basePath, finalizeFilePath, null);
|
||||
writeStat.setPath(basePath, finalizeFilePath);
|
||||
assertEquals(finalizeFilePath, new Path(basePath, writeStat.getPath()));
|
||||
assertNull(writeStat.getTempPath());
|
||||
}
|
||||
|
||||
@@ -19,6 +19,7 @@ package com.uber.hoodie.common.table.log;
|
||||
import static com.uber.hoodie.common.util.SchemaTestUtil.getSimpleSchema;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
@@ -175,6 +176,63 @@ public class HoodieLogFormatTest {
|
||||
writer.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConcurrentAppendOnExistingLogFileWithoutWriteToken() throws Exception {
|
||||
testConcurrentAppend(true, false);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConcurrentAppendOnExistingLogFileWithWriteToken() throws Exception {
|
||||
testConcurrentAppend(true, true);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConcurrentAppendOnFirstLogFileVersion() throws Exception {
|
||||
testConcurrentAppend(false, true);
|
||||
}
|
||||
|
||||
private void testConcurrentAppend(boolean logFileExists, boolean newLogFileFormat) throws Exception {
|
||||
HoodieLogFormat.WriterBuilder builder1 = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
|
||||
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
|
||||
.overBaseCommit("100").withFs(fs);
|
||||
HoodieLogFormat.WriterBuilder builder2 = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
|
||||
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
|
||||
.overBaseCommit("100").withFs(fs);
|
||||
|
||||
if (newLogFileFormat && logFileExists) {
|
||||
// Assume there is an existing log-file with write token
|
||||
builder1 = builder1.withLogVersion(1).withLogWriteToken(HoodieLogFormat.UNKNOWN_WRITE_TOKEN)
|
||||
.withRolloverLogWriteToken(HoodieLogFormat.UNKNOWN_WRITE_TOKEN);
|
||||
builder2 = builder2.withLogVersion(1).withLogWriteToken(HoodieLogFormat.UNKNOWN_WRITE_TOKEN)
|
||||
.withRolloverLogWriteToken(HoodieLogFormat.UNKNOWN_WRITE_TOKEN);
|
||||
} else if (newLogFileFormat) {
|
||||
// First log file of the file-slice
|
||||
builder1 = builder1.withLogVersion(HoodieLogFile.LOGFILE_BASE_VERSION)
|
||||
.withLogWriteToken(HoodieLogFormat.UNKNOWN_WRITE_TOKEN)
|
||||
.withRolloverLogWriteToken(HoodieLogFormat.UNKNOWN_WRITE_TOKEN);
|
||||
builder2 = builder2.withLogVersion(HoodieLogFile.LOGFILE_BASE_VERSION)
|
||||
.withLogWriteToken(HoodieLogFormat.UNKNOWN_WRITE_TOKEN)
|
||||
.withRolloverLogWriteToken(HoodieLogFormat.UNKNOWN_WRITE_TOKEN);
|
||||
} else {
|
||||
builder1 = builder1.withLogVersion(1).withRolloverLogWriteToken(HoodieLogFormat.UNKNOWN_WRITE_TOKEN);
|
||||
}
|
||||
Writer writer = builder1.build();
|
||||
Writer writer2 = builder2.build();
|
||||
HoodieLogFile logFile1 = writer.getLogFile();
|
||||
HoodieLogFile logFile2 = writer2.getLogFile();
|
||||
List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 100);
|
||||
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
|
||||
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
|
||||
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
|
||||
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header);
|
||||
writer = writer.appendBlock(dataBlock);
|
||||
writer2 = writer2.appendBlock(dataBlock);
|
||||
writer.close();
|
||||
writer2.close();
|
||||
assertNotNull(logFile1.getLogWriteToken());
|
||||
assertEquals("Log Files must have different versions", logFile1.getLogVersion(), logFile2.getLogVersion() - 1);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultipleAppend() throws IOException, URISyntaxException, InterruptedException {
|
||||
Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
|
||||
@@ -225,6 +283,12 @@ public class HoodieLogFormatTest {
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This is actually a test on concurrent append and not recovery lease.
|
||||
* Commenting this out.
|
||||
* https://issues.apache.org/jira/browse/HUDI-117
|
||||
*/
|
||||
/**
|
||||
@Test
|
||||
public void testLeaseRecovery() throws IOException, URISyntaxException, InterruptedException {
|
||||
Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath)
|
||||
@@ -253,6 +317,7 @@ public class HoodieLogFormatTest {
|
||||
fs.getFileStatus(writer.getLogFile().getPath()).getLen());
|
||||
writer.close();
|
||||
}
|
||||
**/
|
||||
|
||||
@Test
|
||||
public void testAppendNotSupported() throws IOException, URISyntaxException, InterruptedException {
|
||||
|
||||
@@ -65,6 +65,8 @@ import org.junit.rules.TemporaryFolder;
|
||||
@SuppressWarnings("ResultOfMethodCallIgnored")
|
||||
public class HoodieTableFileSystemViewTest {
|
||||
|
||||
private static String TEST_WRITE_TOKEN = "1-0-1";
|
||||
|
||||
protected HoodieTableMetaClient metaClient;
|
||||
protected String basePath;
|
||||
protected SyncableFileSystemView fsView;
|
||||
@@ -119,8 +121,10 @@ public class HoodieTableFileSystemViewTest {
|
||||
String instantTime1 = "1";
|
||||
String deltaInstantTime1 = "2";
|
||||
String deltaInstantTime2 = "3";
|
||||
String fileName1 = FSUtils.makeLogFileName(fileId, HoodieLogFile.DELTA_EXTENSION, instantTime1, 0);
|
||||
String fileName2 = FSUtils.makeLogFileName(fileId, HoodieLogFile.DELTA_EXTENSION, instantTime1, 1);
|
||||
String fileName1 = FSUtils.makeLogFileName(fileId, HoodieLogFile.DELTA_EXTENSION,
|
||||
instantTime1, 0, TEST_WRITE_TOKEN);
|
||||
String fileName2 = FSUtils.makeLogFileName(fileId, HoodieLogFile.DELTA_EXTENSION,
|
||||
instantTime1, 1, TEST_WRITE_TOKEN);
|
||||
new File(basePath + "/" + partitionPath + "/" + fileName1).createNewFile();
|
||||
new File(basePath + "/" + partitionPath + "/" + fileName2).createNewFile();
|
||||
HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline();
|
||||
@@ -248,11 +252,13 @@ public class HoodieTableFileSystemViewTest {
|
||||
|
||||
String dataFileName = null;
|
||||
if (!skipCreatingDataFile) {
|
||||
dataFileName = FSUtils.makeDataFileName(instantTime1, 1, fileId);
|
||||
dataFileName = FSUtils.makeDataFileName(instantTime1, TEST_WRITE_TOKEN, fileId);
|
||||
new File(basePath + "/" + partitionPath + "/" + dataFileName).createNewFile();
|
||||
}
|
||||
String fileName1 = FSUtils.makeLogFileName(fileId, HoodieLogFile.DELTA_EXTENSION, instantTime1, 0);
|
||||
String fileName2 = FSUtils.makeLogFileName(fileId, HoodieLogFile.DELTA_EXTENSION, instantTime1, 1);
|
||||
String fileName1 = FSUtils.makeLogFileName(fileId, HoodieLogFile.DELTA_EXTENSION,
|
||||
instantTime1, 0, TEST_WRITE_TOKEN);
|
||||
String fileName2 = FSUtils.makeLogFileName(fileId, HoodieLogFile.DELTA_EXTENSION,
|
||||
instantTime1, 1, TEST_WRITE_TOKEN);
|
||||
new File(basePath + "/" + partitionPath + "/" + fileName1).createNewFile();
|
||||
new File(basePath + "/" + partitionPath + "/" + fileName2).createNewFile();
|
||||
HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline();
|
||||
@@ -267,7 +273,7 @@ public class HoodieTableFileSystemViewTest {
|
||||
refreshFsView();
|
||||
List<FileSlice> fileSlices = rtView.getLatestFileSlices(partitionPath).collect(Collectors.toList());
|
||||
String compactionRequestedTime = "4";
|
||||
String compactDataFileName = FSUtils.makeDataFileName(compactionRequestedTime, 1, fileId);
|
||||
String compactDataFileName = FSUtils.makeDataFileName(compactionRequestedTime, TEST_WRITE_TOKEN, fileId);
|
||||
List<Pair<String, FileSlice>> partitionFileSlicesPairs = new ArrayList<>();
|
||||
partitionFileSlicesPairs.add(Pair.of(partitionPath, fileSlices.get(0)));
|
||||
HoodieCompactionPlan compactionPlan = CompactionUtils.buildFromFileSlices(partitionFileSlicesPairs,
|
||||
@@ -299,8 +305,10 @@ public class HoodieTableFileSystemViewTest {
|
||||
String deltaInstantTime5 = "6";
|
||||
List<String> allInstantTimes = Arrays.asList(instantTime1, deltaInstantTime1, deltaInstantTime2,
|
||||
compactionRequestedTime, deltaInstantTime4, deltaInstantTime5);
|
||||
String fileName3 = FSUtils.makeLogFileName(fileId, HoodieLogFile.DELTA_EXTENSION, compactionRequestedTime, 0);
|
||||
String fileName4 = FSUtils.makeLogFileName(fileId, HoodieLogFile.DELTA_EXTENSION, compactionRequestedTime, 1);
|
||||
String fileName3 = FSUtils.makeLogFileName(fileId, HoodieLogFile.DELTA_EXTENSION,
|
||||
compactionRequestedTime, 0, TEST_WRITE_TOKEN);
|
||||
String fileName4 = FSUtils.makeLogFileName(fileId, HoodieLogFile.DELTA_EXTENSION,
|
||||
compactionRequestedTime, 1, TEST_WRITE_TOKEN);
|
||||
new File(basePath + "/" + partitionPath + "/" + fileName3).createNewFile();
|
||||
new File(basePath + "/" + partitionPath + "/" + fileName4).createNewFile();
|
||||
HoodieInstant deltaInstant4 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime4);
|
||||
@@ -400,15 +408,15 @@ public class HoodieTableFileSystemViewTest {
|
||||
final String orphanFileId2 = UUID.randomUUID().toString();
|
||||
final String invalidInstantId = "INVALIDTIME";
|
||||
String inflightDeltaInstantTime = "7";
|
||||
String orphanDataFileName = FSUtils.makeDataFileName(invalidInstantId, 1, orphanFileId1);
|
||||
String orphanDataFileName = FSUtils.makeDataFileName(invalidInstantId, TEST_WRITE_TOKEN, orphanFileId1);
|
||||
new File(basePath + "/" + partitionPath + "/" + orphanDataFileName).createNewFile();
|
||||
String orphanLogFileName =
|
||||
FSUtils.makeLogFileName(orphanFileId2, HoodieLogFile.DELTA_EXTENSION, invalidInstantId, 0);
|
||||
FSUtils.makeLogFileName(orphanFileId2, HoodieLogFile.DELTA_EXTENSION, invalidInstantId, 0, TEST_WRITE_TOKEN);
|
||||
new File(basePath + "/" + partitionPath + "/" + orphanLogFileName).createNewFile();
|
||||
String inflightDataFileName = FSUtils.makeDataFileName(inflightDeltaInstantTime, 1, inflightFileId1);
|
||||
String inflightDataFileName = FSUtils.makeDataFileName(inflightDeltaInstantTime, TEST_WRITE_TOKEN, inflightFileId1);
|
||||
new File(basePath + "/" + partitionPath + "/" + inflightDataFileName).createNewFile();
|
||||
String inflightLogFileName =
|
||||
FSUtils.makeLogFileName(inflightFileId2, HoodieLogFile.DELTA_EXTENSION, inflightDeltaInstantTime, 0);
|
||||
String inflightLogFileName = FSUtils.makeLogFileName(inflightFileId2, HoodieLogFile.DELTA_EXTENSION,
|
||||
inflightDeltaInstantTime, 0, TEST_WRITE_TOKEN);
|
||||
new File(basePath + "/" + partitionPath + "/" + inflightLogFileName).createNewFile();
|
||||
// Mark instant as inflight
|
||||
commitTimeline.saveToInflight(new HoodieInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION,
|
||||
@@ -558,7 +566,7 @@ public class HoodieTableFileSystemViewTest {
|
||||
|
||||
// Only one commit, but is not safe
|
||||
String commitTime1 = "1";
|
||||
String fileName1 = FSUtils.makeDataFileName(commitTime1, 1, fileId);
|
||||
String fileName1 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId);
|
||||
new File(basePath + "/" + partitionPath + "/" + fileName1).createNewFile();
|
||||
refreshFsView();
|
||||
assertFalse("No commit, should not find any data file",
|
||||
@@ -576,7 +584,7 @@ public class HoodieTableFileSystemViewTest {
|
||||
|
||||
// Do another commit, but not safe
|
||||
String commitTime2 = "2";
|
||||
String fileName2 = FSUtils.makeDataFileName(commitTime2, 1, fileId);
|
||||
String fileName2 = FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, fileId);
|
||||
new File(basePath + "/" + partitionPath + "/" + fileName2).createNewFile();
|
||||
refreshFsView();
|
||||
assertEquals("", fileName1,
|
||||
@@ -610,21 +618,21 @@ public class HoodieTableFileSystemViewTest {
|
||||
String fileId3 = UUID.randomUUID().toString();
|
||||
String fileId4 = UUID.randomUUID().toString();
|
||||
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 1))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime3, 0))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeLogFileName(fileId4, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId1)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION,
|
||||
commitTime4, 0, TEST_WRITE_TOKEN)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION,
|
||||
commitTime4, 1, TEST_WRITE_TOKEN)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeLogFileName(fileId2,
|
||||
HoodieLogFile.DELTA_EXTENSION, commitTime3, 0, TEST_WRITE_TOKEN)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeLogFileName(fileId4, HoodieLogFile.DELTA_EXTENSION,
|
||||
commitTime4, 0, TEST_WRITE_TOKEN)).createNewFile();
|
||||
|
||||
new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile();
|
||||
new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile();
|
||||
@@ -653,9 +661,9 @@ public class HoodieTableFileSystemViewTest {
|
||||
for (HoodieDataFile status : dataFileList) {
|
||||
filenames.add(status.getFileName());
|
||||
}
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, 1, fileId1)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, 1, fileId3)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId1)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)));
|
||||
|
||||
filenames = Sets.newHashSet();
|
||||
List<HoodieLogFile> logFilesList = rtView.getLatestFileSlicesBeforeOrOn("2016/05/01", commitTime4)
|
||||
@@ -665,10 +673,14 @@ public class HoodieTableFileSystemViewTest {
|
||||
for (HoodieLogFile logFile : logFilesList) {
|
||||
filenames.add(logFile.getFileName());
|
||||
}
|
||||
assertTrue(filenames.contains(FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0)));
|
||||
assertTrue(filenames.contains(FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 1)));
|
||||
assertTrue(filenames.contains(FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime3, 0)));
|
||||
assertTrue(filenames.contains(FSUtils.makeLogFileName(fileId4, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0)));
|
||||
assertTrue(filenames.contains(FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION,
|
||||
commitTime4, 0, TEST_WRITE_TOKEN)));
|
||||
assertTrue(filenames.contains(FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION,
|
||||
commitTime4, 1, TEST_WRITE_TOKEN)));
|
||||
assertTrue(filenames.contains(FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION,
|
||||
commitTime3, 0, TEST_WRITE_TOKEN)));
|
||||
assertTrue(filenames.contains(FSUtils.makeLogFileName(fileId4, HoodieLogFile.DELTA_EXTENSION,
|
||||
commitTime4, 0, TEST_WRITE_TOKEN)));
|
||||
|
||||
// Reset the max commit time
|
||||
List<HoodieDataFile> dataFiles = roView.getLatestDataFilesBeforeOrOn("2016/05/01", commitTime3)
|
||||
@@ -679,12 +691,12 @@ public class HoodieTableFileSystemViewTest {
|
||||
}
|
||||
if (!isLatestFileSliceOnly) {
|
||||
assertEquals(3, dataFiles.size());
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime1, 1, fileId1)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId3)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)));
|
||||
} else {
|
||||
assertEquals(1, dataFiles.size());
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)));
|
||||
}
|
||||
|
||||
logFilesList =
|
||||
@@ -692,7 +704,7 @@ public class HoodieTableFileSystemViewTest {
|
||||
.flatMap(logFileList -> logFileList).collect(Collectors.toList());
|
||||
assertEquals(logFilesList.size(), 1);
|
||||
assertTrue(logFilesList.get(0).getFileName()
|
||||
.equals(FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime3, 0)));
|
||||
.equals(FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime3, 0, TEST_WRITE_TOKEN)));
|
||||
|
||||
}
|
||||
|
||||
@@ -713,13 +725,13 @@ public class HoodieTableFileSystemViewTest {
|
||||
String fileId2 = UUID.randomUUID().toString();
|
||||
String fileId3 = UUID.randomUUID().toString();
|
||||
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId1)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)).createNewFile();
|
||||
|
||||
new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile();
|
||||
new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile();
|
||||
@@ -744,22 +756,22 @@ public class HoodieTableFileSystemViewTest {
|
||||
Set<String> expFileNames = new HashSet<>();
|
||||
if (fileId.equals(fileId1)) {
|
||||
if (!isLatestFileSliceOnly) {
|
||||
expFileNames.add(FSUtils.makeDataFileName(commitTime1, 1, fileId1));
|
||||
expFileNames.add(FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1));
|
||||
}
|
||||
expFileNames.add(FSUtils.makeDataFileName(commitTime4, 1, fileId1));
|
||||
expFileNames.add(FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId1));
|
||||
assertEquals(expFileNames, filenames);
|
||||
} else if (fileId.equals(fileId2)) {
|
||||
if (!isLatestFileSliceOnly) {
|
||||
expFileNames.add(FSUtils.makeDataFileName(commitTime1, 1, fileId2));
|
||||
expFileNames.add(FSUtils.makeDataFileName(commitTime2, 1, fileId2));
|
||||
expFileNames.add(FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2));
|
||||
expFileNames.add(FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, fileId2));
|
||||
}
|
||||
expFileNames.add(FSUtils.makeDataFileName(commitTime3, 1, fileId2));
|
||||
expFileNames.add(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2));
|
||||
assertEquals(expFileNames, filenames);
|
||||
} else {
|
||||
if (!isLatestFileSliceOnly) {
|
||||
expFileNames.add(FSUtils.makeDataFileName(commitTime3, 1, fileId3));
|
||||
expFileNames.add(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId3));
|
||||
}
|
||||
expFileNames.add(FSUtils.makeDataFileName(commitTime4, 1, fileId3));
|
||||
expFileNames.add(FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId3));
|
||||
assertEquals(expFileNames, filenames);
|
||||
}
|
||||
}
|
||||
@@ -782,19 +794,19 @@ public class HoodieTableFileSystemViewTest {
|
||||
String fileId2 = UUID.randomUUID().toString();
|
||||
String fileId3 = UUID.randomUUID().toString();
|
||||
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime1, 0))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId1)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION,
|
||||
commitTime1, 0, TEST_WRITE_TOKEN)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId1)).createNewFile();
|
||||
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime3, 0))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION,
|
||||
commitTime3, 0, TEST_WRITE_TOKEN)).createNewFile();
|
||||
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)).createNewFile();
|
||||
|
||||
new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile();
|
||||
new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile();
|
||||
@@ -817,10 +829,10 @@ public class HoodieTableFileSystemViewTest {
|
||||
filenames.add(status.getFileName());
|
||||
}
|
||||
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId1)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId1)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)));
|
||||
if (!isLatestFileSliceOnly) {
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId3)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)));
|
||||
}
|
||||
|
||||
List<FileSlice> slices = rtView.getLatestFileSliceInRange(Lists.newArrayList(commitTime3, commitTime4))
|
||||
@@ -861,13 +873,13 @@ public class HoodieTableFileSystemViewTest {
|
||||
String fileId2 = UUID.randomUUID().toString();
|
||||
String fileId3 = UUID.randomUUID().toString();
|
||||
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId1)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)).createNewFile();
|
||||
|
||||
new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile();
|
||||
new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile();
|
||||
@@ -887,8 +899,8 @@ public class HoodieTableFileSystemViewTest {
|
||||
for (HoodieDataFile status : dataFiles) {
|
||||
filenames.add(status.getFileName());
|
||||
}
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime1, 1, fileId1)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime2, 1, fileId2)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)));
|
||||
} else {
|
||||
assertEquals(0, dataFiles.size());
|
||||
}
|
||||
@@ -912,28 +924,31 @@ public class HoodieTableFileSystemViewTest {
|
||||
String fileId2 = UUID.randomUUID().toString();
|
||||
String fileId3 = UUID.randomUUID().toString();
|
||||
|
||||
new File(fullPartitionPath + "/" + FSUtils.makeDataFileName(commitTime1, 1, fileId1)).createNewFile();
|
||||
new File(fullPartitionPath + "/" + FSUtils.makeDataFileName(commitTime1,
|
||||
TEST_WRITE_TOKEN, fileId1)).createNewFile();
|
||||
new File(fullPartitionPath + "/"
|
||||
+ FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime1, 0)).createNewFile();
|
||||
+ FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION,
|
||||
commitTime1, 0, TEST_WRITE_TOKEN)).createNewFile();
|
||||
new File(fullPartitionPath + "/"
|
||||
+ FSUtils.makeDataFileName(commitTime4, 1, fileId1)).createNewFile();
|
||||
+ FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId1)).createNewFile();
|
||||
new File(fullPartitionPath + "/"
|
||||
+ FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0)).createNewFile();
|
||||
+ FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION,
|
||||
commitTime4, 0, TEST_WRITE_TOKEN)).createNewFile();
|
||||
|
||||
new File(fullPartitionPath + "/"
|
||||
+ FSUtils.makeDataFileName(commitTime1, 1, fileId2)).createNewFile();
|
||||
+ FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + "/"
|
||||
+ FSUtils.makeDataFileName(commitTime2, 1, fileId2)).createNewFile();
|
||||
+ FSUtils.makeDataFileName(commitTime2, TEST_WRITE_TOKEN, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + "/"
|
||||
+ FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime2, 0))
|
||||
+ FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime2, 0, TEST_WRITE_TOKEN))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + "/"
|
||||
+ FSUtils.makeDataFileName(commitTime3, 1, fileId2)).createNewFile();
|
||||
+ FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)).createNewFile();
|
||||
|
||||
new File(fullPartitionPath + "/"
|
||||
+ FSUtils.makeDataFileName(commitTime3, 1, fileId3)).createNewFile();
|
||||
+ FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId3)).createNewFile();
|
||||
new File(fullPartitionPath + "/"
|
||||
+ FSUtils.makeDataFileName(commitTime4, 1, fileId3)).createNewFile();
|
||||
+ FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)).createNewFile();
|
||||
|
||||
new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile();
|
||||
new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile();
|
||||
@@ -979,9 +994,9 @@ public class HoodieTableFileSystemViewTest {
|
||||
for (HoodieDataFile status : statuses1) {
|
||||
filenames.add(status.getFileName());
|
||||
}
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, 1, fileId1)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, 1, fileId3)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId1)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)));
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -1002,16 +1017,17 @@ public class HoodieTableFileSystemViewTest {
|
||||
String deltaInstantTime2 = "3";
|
||||
String fileId = UUID.randomUUID().toString();
|
||||
|
||||
String dataFileName = FSUtils.makeDataFileName(instantTime1, 1, fileId);
|
||||
String dataFileName = FSUtils.makeDataFileName(instantTime1, TEST_WRITE_TOKEN, fileId);
|
||||
new File(fullPartitionPath1 + dataFileName).createNewFile();
|
||||
|
||||
String fileName1 = FSUtils.makeLogFileName(fileId, HoodieLogFile.DELTA_EXTENSION, instantTime1, 0);
|
||||
String fileName1 = FSUtils.makeLogFileName(fileId, HoodieLogFile.DELTA_EXTENSION,
|
||||
instantTime1, 0, TEST_WRITE_TOKEN);
|
||||
new File(fullPartitionPath1 + fileName1)
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath2 + FSUtils.makeDataFileName(instantTime1, 1, fileId)).createNewFile();
|
||||
new File(fullPartitionPath2 + FSUtils.makeDataFileName(instantTime1, TEST_WRITE_TOKEN, fileId)).createNewFile();
|
||||
new File(fullPartitionPath2 + fileName1)
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath3 + FSUtils.makeDataFileName(instantTime1, 1, fileId)).createNewFile();
|
||||
new File(fullPartitionPath3 + FSUtils.makeDataFileName(instantTime1, TEST_WRITE_TOKEN, fileId)).createNewFile();
|
||||
new File(fullPartitionPath3 + fileName1)
|
||||
.createNewFile();
|
||||
|
||||
@@ -1052,7 +1068,7 @@ public class HoodieTableFileSystemViewTest {
|
||||
partitionFileSlicesPairs.add(Pair.of(partitionPath3, fileSlices.get(0)));
|
||||
|
||||
String compactionRequestedTime = "2";
|
||||
String compactDataFileName = FSUtils.makeDataFileName(compactionRequestedTime, 1, fileId);
|
||||
String compactDataFileName = FSUtils.makeDataFileName(compactionRequestedTime, TEST_WRITE_TOKEN, fileId);
|
||||
HoodieCompactionPlan compactionPlan = CompactionUtils.buildFromFileSlices(partitionFileSlicesPairs,
|
||||
Optional.empty(), Optional.empty());
|
||||
|
||||
@@ -1072,8 +1088,10 @@ public class HoodieTableFileSystemViewTest {
|
||||
String deltaInstantTime5 = "6";
|
||||
List<String> allInstantTimes = Arrays.asList(instantTime1, deltaInstantTime1, deltaInstantTime2,
|
||||
compactionRequestedTime, deltaInstantTime4, deltaInstantTime5);
|
||||
String fileName3 = FSUtils.makeLogFileName(fileId, HoodieLogFile.DELTA_EXTENSION, compactionRequestedTime, 0);
|
||||
String fileName4 = FSUtils.makeLogFileName(fileId, HoodieLogFile.DELTA_EXTENSION, compactionRequestedTime, 1);
|
||||
String fileName3 = FSUtils.makeLogFileName(fileId, HoodieLogFile.DELTA_EXTENSION,
|
||||
compactionRequestedTime, 0, TEST_WRITE_TOKEN);
|
||||
String fileName4 = FSUtils.makeLogFileName(fileId, HoodieLogFile.DELTA_EXTENSION,
|
||||
compactionRequestedTime, 1, TEST_WRITE_TOKEN);
|
||||
new File(basePath + "/" + partitionPath1 + "/" + fileName3).createNewFile();
|
||||
new File(basePath + "/" + partitionPath1 + "/" + fileName4).createNewFile();
|
||||
new File(basePath + "/" + partitionPath2 + "/" + fileName3).createNewFile();
|
||||
|
||||
@@ -69,6 +69,8 @@ import org.junit.rules.TemporaryFolder;
|
||||
|
||||
public class IncrementalFSViewSyncTest {
|
||||
|
||||
private static String TEST_WRITE_TOKEN = "1-0-1";
|
||||
|
||||
protected HoodieTableMetaClient metaClient;
|
||||
protected String basePath;
|
||||
|
||||
@@ -756,8 +758,8 @@ public class IncrementalFSViewSyncTest {
|
||||
return fileIdsPerPartition.stream().map(f -> {
|
||||
try {
|
||||
File file = new File(basePath + "/" + p + "/"
|
||||
+ (deltaCommit ? FSUtils.makeLogFileName(f, ".log", baseInstant, Integer.parseInt(instant)) :
|
||||
FSUtils.makeDataFileName(instant, 0, f)));
|
||||
+ (deltaCommit ? FSUtils.makeLogFileName(f, ".log", baseInstant,
|
||||
Integer.parseInt(instant), TEST_WRITE_TOKEN) : FSUtils.makeDataFileName(instant, TEST_WRITE_TOKEN, f)));
|
||||
file.createNewFile();
|
||||
HoodieWriteStat w = new HoodieWriteStat();
|
||||
w.setFileId(f);
|
||||
|
||||
@@ -49,6 +49,8 @@ import org.junit.Assert;
|
||||
|
||||
public class CompactionTestUtils {
|
||||
|
||||
private static String TEST_WRITE_TOKEN = "1-0-1";
|
||||
|
||||
public static Map<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>> setupAndValidateCompactionOperations(
|
||||
HoodieTableMetaClient metaClient, boolean inflight,
|
||||
int numEntriesInPlan1, int numEntriesInPlan2,
|
||||
@@ -151,7 +153,7 @@ public class CompactionTestUtils {
|
||||
FileSlice slice = new FileSlice(DEFAULT_PARTITION_PATHS[0], instantId, fileId);
|
||||
if (createDataFile) {
|
||||
slice.setDataFile(new TestHoodieDataFile(metaClient.getBasePath() + "/" + DEFAULT_PARTITION_PATHS[0]
|
||||
+ "/" + FSUtils.makeDataFileName(instantId, 1, fileId)));
|
||||
+ "/" + FSUtils.makeDataFileName(instantId, TEST_WRITE_TOKEN, fileId)));
|
||||
}
|
||||
String logFilePath1 = HoodieTestUtils
|
||||
.getLogFilePath(metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0], instantId, fileId,
|
||||
|
||||
@@ -50,6 +50,8 @@ import org.junit.rules.TemporaryFolder;
|
||||
|
||||
public class TestCompactionUtils {
|
||||
|
||||
private static String TEST_WRITE_TOKEN = "1-0-1";
|
||||
|
||||
private static final Map<String, Double> metrics =
|
||||
new ImmutableMap.Builder<String, Double>()
|
||||
.put("key1", 1.0)
|
||||
@@ -85,9 +87,9 @@ public class TestCompactionUtils {
|
||||
//File Slice with no data-file but log files present
|
||||
FileSlice noDataFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "noData1");
|
||||
noDataFileSlice.addLogFile(new HoodieLogFile(new Path(
|
||||
FSUtils.makeLogFileName("noData1", ".log", "000", 1))));
|
||||
FSUtils.makeLogFileName("noData1", ".log", "000", 1, TEST_WRITE_TOKEN))));
|
||||
noDataFileSlice.addLogFile(new HoodieLogFile(new Path(
|
||||
FSUtils.makeLogFileName("noData1", ".log", "000", 2))));
|
||||
FSUtils.makeLogFileName("noData1", ".log", "000", 2, TEST_WRITE_TOKEN))));
|
||||
op = CompactionUtils.buildFromFileSlice(
|
||||
DEFAULT_PARTITION_PATHS[0], noDataFileSlice, Optional.of(metricsCaptureFn));
|
||||
testFileSliceCompactionOpEquality(noDataFileSlice, op, DEFAULT_PARTITION_PATHS[0]);
|
||||
@@ -96,9 +98,9 @@ public class TestCompactionUtils {
|
||||
FileSlice fileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "noData1");
|
||||
fileSlice.setDataFile(new TestHoodieDataFile("/tmp/noLog_1_000.parquet"));
|
||||
fileSlice.addLogFile(new HoodieLogFile(new Path(
|
||||
FSUtils.makeLogFileName("noData1", ".log", "000", 1))));
|
||||
FSUtils.makeLogFileName("noData1", ".log", "000", 1, TEST_WRITE_TOKEN))));
|
||||
fileSlice.addLogFile(new HoodieLogFile(new Path(
|
||||
FSUtils.makeLogFileName("noData1", ".log", "000", 2))));
|
||||
FSUtils.makeLogFileName("noData1", ".log", "000", 2, TEST_WRITE_TOKEN))));
|
||||
op = CompactionUtils.buildFromFileSlice(
|
||||
DEFAULT_PARTITION_PATHS[0], fileSlice, Optional.of(metricsCaptureFn));
|
||||
testFileSliceCompactionOpEquality(fileSlice, op, DEFAULT_PARTITION_PATHS[0]);
|
||||
@@ -112,16 +114,16 @@ public class TestCompactionUtils {
|
||||
FileSlice fileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "noData1");
|
||||
fileSlice.setDataFile(new TestHoodieDataFile("/tmp/noLog_1_000.parquet"));
|
||||
fileSlice.addLogFile(new HoodieLogFile(new Path(
|
||||
FSUtils.makeLogFileName("noData1", ".log", "000", 1))));
|
||||
FSUtils.makeLogFileName("noData1", ".log", "000", 1, TEST_WRITE_TOKEN))));
|
||||
fileSlice.addLogFile(new HoodieLogFile(new Path(
|
||||
FSUtils.makeLogFileName("noData1", ".log", "000", 2))));
|
||||
FSUtils.makeLogFileName("noData1", ".log", "000", 2, TEST_WRITE_TOKEN))));
|
||||
FileSlice noLogFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "noLog1");
|
||||
noLogFileSlice.setDataFile(new TestHoodieDataFile("/tmp/noLog_1_000.parquet"));
|
||||
FileSlice noDataFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "noData1");
|
||||
noDataFileSlice.addLogFile(new HoodieLogFile(new Path(
|
||||
FSUtils.makeLogFileName("noData1", ".log", "000", 1))));
|
||||
FSUtils.makeLogFileName("noData1", ".log", "000", 1, TEST_WRITE_TOKEN))));
|
||||
noDataFileSlice.addLogFile(new HoodieLogFile(new Path(
|
||||
FSUtils.makeLogFileName("noData1", ".log", "000", 2))));
|
||||
FSUtils.makeLogFileName("noData1", ".log", "000", 2, TEST_WRITE_TOKEN))));
|
||||
List<FileSlice> fileSliceList = Arrays.asList(emptyFileSlice, noDataFileSlice, fileSlice, noLogFileSlice);
|
||||
List<Pair<String, FileSlice>> input = fileSliceList.stream().map(f -> Pair.of(DEFAULT_PARTITION_PATHS[0], f))
|
||||
.collect(Collectors.toList());
|
||||
|
||||
@@ -23,14 +23,18 @@ import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.Date;
|
||||
import java.util.UUID;
|
||||
import java.util.regex.Pattern;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.contrib.java.lang.system.EnvironmentVariables;
|
||||
|
||||
public class TestFSUtils {
|
||||
|
||||
private static String TEST_WRITE_TOKEN = "1-0-1";
|
||||
|
||||
@Rule
|
||||
public final EnvironmentVariables environmentVariables = new EnvironmentVariables();
|
||||
|
||||
@@ -39,22 +43,8 @@ public class TestFSUtils {
|
||||
String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
|
||||
int taskPartitionId = 2;
|
||||
String fileName = UUID.randomUUID().toString();
|
||||
assertTrue(FSUtils.makeDataFileName(commitTime, taskPartitionId, fileName)
|
||||
.equals(fileName + "_" + taskPartitionId + "_" + commitTime + ".parquet"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMakeTempDataFileName() {
|
||||
String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
|
||||
String partitionPath = "2017/12/31";
|
||||
int taskPartitionId = Integer.MAX_VALUE;
|
||||
int stageId = Integer.MAX_VALUE;
|
||||
long taskAttemptId = Long.MAX_VALUE;
|
||||
String fileName = UUID.randomUUID().toString();
|
||||
assertTrue(
|
||||
FSUtils.makeTempDataFileName(partitionPath, commitTime, taskPartitionId, fileName, stageId, taskAttemptId)
|
||||
.equals(partitionPath.replace("/", "-") + "_" + fileName + "_" + taskPartitionId + "_" + commitTime + "_"
|
||||
+ stageId + "_" + taskAttemptId + ".parquet"));
|
||||
assertTrue(FSUtils.makeDataFileName(commitTime, TEST_WRITE_TOKEN, fileName)
|
||||
.equals(fileName + "_" + TEST_WRITE_TOKEN + "_" + commitTime + ".parquet"));
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -70,7 +60,7 @@ public class TestFSUtils {
|
||||
String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
|
||||
int taskPartitionId = 2;
|
||||
String fileName = UUID.randomUUID().toString();
|
||||
String fullFileName = FSUtils.makeDataFileName(commitTime, taskPartitionId, fileName);
|
||||
String fullFileName = FSUtils.makeDataFileName(commitTime, TEST_WRITE_TOKEN, fileName);
|
||||
assertTrue(FSUtils.getCommitTime(fullFileName).equals(commitTime));
|
||||
}
|
||||
|
||||
@@ -79,7 +69,7 @@ public class TestFSUtils {
|
||||
String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
|
||||
int taskPartitionId = 2;
|
||||
String fileName = UUID.randomUUID().toString();
|
||||
String fullFileName = FSUtils.makeDataFileName(commitTime, taskPartitionId, fileName);
|
||||
String fullFileName = FSUtils.makeDataFileName(commitTime, TEST_WRITE_TOKEN, fileName);
|
||||
assertTrue(FSUtils.getFileId(fullFileName).equals(fileName));
|
||||
}
|
||||
|
||||
@@ -121,4 +111,47 @@ public class TestFSUtils {
|
||||
Path partitionPath = new Path("/test/apache/apache/hudi");
|
||||
assertEquals("apache/hudi", FSUtils.getRelativePartitionPath(basePath, partitionPath));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testOldLogFileName() {
|
||||
// Check if old log file names are still parseable by FSUtils method
|
||||
String partitionPath = "2019/01/01/";
|
||||
String fileName = UUID.randomUUID().toString();
|
||||
String oldLogFile = makeOldLogFileName(fileName, ".log", "100", 1);
|
||||
Path rlPath = new Path(new Path(partitionPath), oldLogFile);
|
||||
Assert.assertTrue(FSUtils.isLogFile(rlPath));
|
||||
Assert.assertEquals(fileName, FSUtils.getFileIdFromLogPath(rlPath));
|
||||
Assert.assertEquals("100", FSUtils.getBaseCommitTimeFromLogPath(rlPath));
|
||||
Assert.assertEquals(1, FSUtils.getFileVersionFromLog(rlPath));
|
||||
Assert.assertNull(FSUtils.getTaskPartitionIdFromLogPath(rlPath));
|
||||
Assert.assertNull(FSUtils.getStageIdFromLogPath(rlPath));
|
||||
Assert.assertNull(FSUtils.getTaskAttemptIdFromLogPath(rlPath));
|
||||
Assert.assertNull(FSUtils.getWriteTokenFromLogPath(rlPath));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void tesLogFileName() {
|
||||
// Check if log file names are parseable by FSUtils method
|
||||
String partitionPath = "2019/01/01/";
|
||||
String fileName = UUID.randomUUID().toString();
|
||||
String logFile = FSUtils.makeLogFileName(fileName, ".log", "100", 2, "1-0-1");
|
||||
System.out.println("Log File =" + logFile);
|
||||
Path rlPath = new Path(new Path(partitionPath), logFile);
|
||||
Assert.assertTrue(FSUtils.isLogFile(rlPath));
|
||||
Assert.assertEquals(fileName, FSUtils.getFileIdFromLogPath(rlPath));
|
||||
Assert.assertEquals("100", FSUtils.getBaseCommitTimeFromLogPath(rlPath));
|
||||
Assert.assertEquals(2, FSUtils.getFileVersionFromLog(rlPath));
|
||||
Assert.assertEquals(new Integer(1), FSUtils.getTaskPartitionIdFromLogPath(rlPath));
|
||||
Assert.assertEquals(new Integer(0), FSUtils.getStageIdFromLogPath(rlPath));
|
||||
Assert.assertEquals(new Integer(1), FSUtils.getTaskAttemptIdFromLogPath(rlPath));
|
||||
|
||||
}
|
||||
|
||||
public static String makeOldLogFileName(String fileId, String logFileExtension,
|
||||
String baseCommitTime, int version) {
|
||||
Pattern oldLogFilePattern =
|
||||
Pattern.compile("\\.(.*)_(.*)\\.(.*)\\.([0-9]*)(\\.([0-9]*))");
|
||||
return "." + String
|
||||
.format("%s_%s%s.%d", fileId, baseCommitTime, logFileExtension, version);
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user