HADOOP-16885. Encryption zone file copy failure leaks a temp file

Contributed by Xiaoyu Yao.

Contains HDFS-14892. Close the output stream if createWrappedOutputStream() fails

Copying file through the FsShell command into an HDFS encryption zone where
the caller lacks permissions is leaks a temp ._COPYING file
and potentially a wrapped stream unclosed.

This is a convergence of a fix for S3 meeting an issue in HDFS.

S3: a HEAD against a file can cache a 404, 
 -you must not do any existence checks, including deleteOnExit(),
  until the file is written. 

Hence: HADOOP-16490, only register files for deletion the create worked
and the upload is not direct. 

HDFS-14892. HDFS doesn't close wrapped streams when IOEs are raised on
create() failures. Which means that an entry is retained on the NN.
-you need to register a file with deleteOnExit() even if the file wasn't
created.

This patch:

* Moves the deleteOnExit to ensure the created file get deleted cleanly.
* Fixes HDFS to close the wrapped stream on failures.
This commit is contained in:
Xiaoyu Yao 2020-03-02 08:22:00 -05:00 committed by GitHub
parent 1a636da041
commit 0dd8956f2e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
2 changed files with 31 additions and 22 deletions

View File

@ -415,7 +415,6 @@ protected void copyStreamToTarget(InputStream in, PathData target)
targetFs.setWriteChecksum(writeChecksum); targetFs.setWriteChecksum(writeChecksum);
targetFs.writeStreamToFile(in, tempTarget, lazyPersist, direct); targetFs.writeStreamToFile(in, tempTarget, lazyPersist, direct);
if (!direct) { if (!direct) {
targetFs.deleteOnExit(tempTarget.path);
targetFs.rename(tempTarget, target); targetFs.rename(tempTarget, target);
} }
} finally { } finally {
@ -491,25 +490,18 @@ void writeStreamToFile(InputStream in, PathData target,
throws IOException { throws IOException {
FSDataOutputStream out = null; FSDataOutputStream out = null;
try { try {
out = create(target, lazyPersist, direct); out = create(target, lazyPersist);
IOUtils.copyBytes(in, out, getConf(), true); IOUtils.copyBytes(in, out, getConf(), true);
} catch (IOException e) {
// failure: clean up if we got as far as creating the file
if (!direct && out != null) {
try {
fs.delete(target.path, false);
} catch (IOException ignored) {
}
}
throw e;
} finally { } finally {
if (!direct) {
deleteOnExit(target.path);
}
IOUtils.closeStream(out); // just in case copyBytes didn't IOUtils.closeStream(out); // just in case copyBytes didn't
} }
} }
// tag created files as temp files // tag created files as temp files
FSDataOutputStream create(PathData item, boolean lazyPersist, FSDataOutputStream create(PathData item, boolean lazyPersist)
boolean direct)
throws IOException { throws IOException {
if (lazyPersist) { if (lazyPersist) {
long defaultBlockSize; long defaultBlockSize;

View File

@ -129,8 +129,6 @@
import java.util.NoSuchElementException; import java.util.NoSuchElementException;
import java.util.Optional; import java.util.Optional;
import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs;
/**************************************************************** /****************************************************************
* Implementation of the abstract FileSystem for the DFS system. * Implementation of the abstract FileSystem for the DFS system.
* This object is the way end-user code interacts with a Hadoop * This object is the way end-user code interacts with a Hadoop
@ -329,7 +327,12 @@ public FSDataInputStream open(Path f, final int bufferSize)
public FSDataInputStream doCall(final Path p) throws IOException { public FSDataInputStream doCall(final Path p) throws IOException {
final DFSInputStream dfsis = final DFSInputStream dfsis =
dfs.open(getPathName(p), bufferSize, verifyChecksum); dfs.open(getPathName(p), bufferSize, verifyChecksum);
return dfs.createWrappedInputStream(dfsis); try {
return dfs.createWrappedInputStream(dfsis);
} catch (IOException ex){
dfsis.close();
throw ex;
}
} }
@Override @Override
public FSDataInputStream next(final FileSystem fs, final Path p) public FSDataInputStream next(final FileSystem fs, final Path p)
@ -510,7 +513,7 @@ public HdfsDataOutputStream doCall(final Path p) throws IOException {
: EnumSet.of(CreateFlag.CREATE), : EnumSet.of(CreateFlag.CREATE),
true, replication, blockSize, progress, bufferSize, null, true, replication, blockSize, progress, bufferSize, null,
favoredNodes); favoredNodes);
return dfs.createWrappedOutputStream(out, statistics); return safelyCreateWrappedOutputStream(out);
} }
@Override @Override
public HdfsDataOutputStream next(final FileSystem fs, final Path p) public HdfsDataOutputStream next(final FileSystem fs, final Path p)
@ -542,7 +545,7 @@ public FSDataOutputStream doCall(final Path p) throws IOException {
final DFSOutputStream dfsos = dfs.create(getPathName(p), permission, final DFSOutputStream dfsos = dfs.create(getPathName(p), permission,
cflags, replication, blockSize, progress, bufferSize, cflags, replication, blockSize, progress, bufferSize,
checksumOpt); checksumOpt);
return dfs.createWrappedOutputStream(dfsos, statistics); return safelyCreateWrappedOutputStream(dfsos);
} }
@Override @Override
public FSDataOutputStream next(final FileSystem fs, final Path p) public FSDataOutputStream next(final FileSystem fs, final Path p)
@ -590,7 +593,7 @@ public HdfsDataOutputStream doCall(final Path p) throws IOException {
final DFSOutputStream out = dfs.create(getPathName(f), permission, final DFSOutputStream out = dfs.create(getPathName(f), permission,
flag, true, replication, blockSize, progress, bufferSize, flag, true, replication, blockSize, progress, bufferSize,
checksumOpt, favoredNodes, ecPolicyName, storagePolicy); checksumOpt, favoredNodes, ecPolicyName, storagePolicy);
return dfs.createWrappedOutputStream(out, statistics); return safelyCreateWrappedOutputStream(out);
} }
@Override @Override
public HdfsDataOutputStream next(final FileSystem fs, final Path p) public HdfsDataOutputStream next(final FileSystem fs, final Path p)
@ -619,7 +622,7 @@ protected HdfsDataOutputStream primitiveCreate(Path f,
getPathName(fixRelativePart(f)), getPathName(fixRelativePart(f)),
absolutePermission, flag, true, replication, blockSize, absolutePermission, flag, true, replication, blockSize,
progress, bufferSize, checksumOpt); progress, bufferSize, checksumOpt);
return dfs.createWrappedOutputStream(dfsos, statistics); return safelyCreateWrappedOutputStream(dfsos);
} }
/** /**
@ -648,7 +651,7 @@ public HdfsDataOutputStream doCall(final Path p) throws IOException {
final DFSOutputStream out = dfs.create(getPathName(f), permission, final DFSOutputStream out = dfs.create(getPathName(f), permission,
flag, false, replication, blockSize, progress, bufferSize, flag, false, replication, blockSize, progress, bufferSize,
checksumOpt, favoredNodes, ecPolicyName, storagePolicyName); checksumOpt, favoredNodes, ecPolicyName, storagePolicyName);
return dfs.createWrappedOutputStream(out, statistics); return safelyCreateWrappedOutputStream(out);
} }
@Override @Override
public HdfsDataOutputStream next(final FileSystem fs, final Path p) public HdfsDataOutputStream next(final FileSystem fs, final Path p)
@ -685,7 +688,7 @@ public FSDataOutputStream createNonRecursive(final Path f,
public FSDataOutputStream doCall(final Path p) throws IOException { public FSDataOutputStream doCall(final Path p) throws IOException {
final DFSOutputStream dfsos = dfs.create(getPathName(p), permission, final DFSOutputStream dfsos = dfs.create(getPathName(p), permission,
flag, false, replication, blockSize, progress, bufferSize, null); flag, false, replication, blockSize, progress, bufferSize, null);
return dfs.createWrappedOutputStream(dfsos, statistics); return safelyCreateWrappedOutputStream(dfsos);
} }
@Override @Override
@ -697,6 +700,20 @@ public FSDataOutputStream next(final FileSystem fs, final Path p)
}.resolve(this, absF); }.resolve(this, absF);
} }
// Private helper to ensure the wrapped inner stream is closed safely
// upon IOException throw during wrap.
// Assuming the caller owns the inner stream which needs to be closed upon
// wrap failure.
private HdfsDataOutputStream safelyCreateWrappedOutputStream(
DFSOutputStream dfsos) throws IOException {
try {
return dfs.createWrappedOutputStream(dfsos, statistics);
} catch (IOException ex) {
dfsos.close();
throw ex;
}
}
@Override @Override
public boolean setReplication(Path src, final short replication) public boolean setReplication(Path src, final short replication)
throws IOException { throws IOException {