HADOOP-15085. Output streams closed with IOUtils suppressing write errors. Contributed by Jim Brennan

This commit is contained in:
Jason Lowe 2017-12-14 10:18:08 -06:00
parent 80db744ee5
commit f8af0e2feb
8 changed files with 38 additions and 69 deletions

View File

@ -2121,17 +2121,13 @@ public class FileContext {
content.getPath().getName())), deleteSource, overwrite);
}
} else {
InputStream in=null;
OutputStream out = null;
try {
in = open(qSrc);
EnumSet<CreateFlag> createFlag = overwrite ? EnumSet.of(
CreateFlag.CREATE, CreateFlag.OVERWRITE) :
EnumSet.of(CreateFlag.CREATE);
out = create(qDst, createFlag);
EnumSet<CreateFlag> createFlag = overwrite ? EnumSet.of(
CreateFlag.CREATE, CreateFlag.OVERWRITE) :
EnumSet.of(CreateFlag.CREATE);
InputStream in = open(qSrc);
try (OutputStream out = create(qDst, createFlag)) {
IOUtils.copyBytes(in, out, conf, true);
} finally {
IOUtils.closeStream(out);
IOUtils.closeStream(in);
}
}

View File

@ -1279,15 +1279,10 @@ public class FileUtil {
// Write the manifest to output JAR file
File classPathJar = File.createTempFile("classpath-", ".jar", workingDir);
FileOutputStream fos = null;
BufferedOutputStream bos = null;
JarOutputStream jos = null;
try {
fos = new FileOutputStream(classPathJar);
bos = new BufferedOutputStream(fos);
jos = new JarOutputStream(bos, jarManifest);
} finally {
IOUtils.cleanupWithLogger(LOG, jos, bos, fos);
try (FileOutputStream fos = new FileOutputStream(classPathJar);
BufferedOutputStream bos = new BufferedOutputStream(fos)) {
JarOutputStream jos = new JarOutputStream(bos, jarManifest);
jos.close();
}
String[] jarCp = {classPathJar.getCanonicalPath(),
unexpandedWildcardClasspath.toString()};

View File

@ -20,7 +20,6 @@ package org.apache.hadoop.fs.shell;
import java.io.FileInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.Iterator;
@ -464,10 +463,8 @@ class CopyCommands {
dst.fs.create(dst.path, false).close();
}
InputStream is = null;
FSDataOutputStream fos = dst.fs.append(dst.path);
try {
FileInputStream is = null;
try (FSDataOutputStream fos = dst.fs.append(dst.path)) {
if (readStdin) {
if (args.size() == 0) {
IOUtils.copyBytes(System.in, fos, DEFAULT_IO_LENGTH);
@ -488,10 +485,6 @@ class CopyCommands {
if (is != null) {
IOUtils.closeStream(is);
}
if (fos != null) {
IOUtils.closeStream(fos);
}
}
}
}

View File

@ -986,23 +986,22 @@ public class MapFile {
Configuration conf = new Configuration();
FileSystem fs = FileSystem.getLocal(conf);
MapFile.Reader reader = null;
MapFile.Writer writer = null;
try {
reader = new MapFile.Reader(fs, in, conf);
writer =
new MapFile.Writer(conf, fs, out,
reader.getKeyClass().asSubclass(WritableComparable.class),
reader.getValueClass());
WritableComparable<?> key = ReflectionUtils.newInstance(
reader.getKeyClass().asSubclass(WritableComparable.class), conf);
Writable value = ReflectionUtils.newInstance(reader.getValueClass()
.asSubclass(Writable.class), conf);
while (reader.next(key, value)) // copy all entries
writer.append(key, value);
try (MapFile.Writer writer = new MapFile.Writer(conf, fs, out,
reader.getKeyClass().asSubclass(WritableComparable.class),
reader.getValueClass())) {
while (reader.next(key, value)) { // copy all entries
writer.append(key, value);
}
}
} finally {
IOUtils.cleanupWithLogger(LOG, writer, reader);
IOUtils.cleanupWithLogger(LOG, reader);
}
}
}

View File

@ -956,28 +956,23 @@ public class NativeIO {
if (nativeLoaded && Shell.WINDOWS) {
copyFileUnbuffered0(src.getAbsolutePath(), dst.getAbsolutePath());
} else {
FileInputStream fis = null;
FileOutputStream fos = null;
FileInputStream fis = new FileInputStream(src);
FileChannel input = null;
FileChannel output = null;
try {
fis = new FileInputStream(src);
fos = new FileOutputStream(dst);
input = fis.getChannel();
output = fos.getChannel();
long remaining = input.size();
long position = 0;
long transferred = 0;
while (remaining > 0) {
transferred = input.transferTo(position, remaining, output);
remaining -= transferred;
position += transferred;
try (FileOutputStream fos = new FileOutputStream(dst);
FileChannel output = fos.getChannel()) {
long remaining = input.size();
long position = 0;
long transferred = 0;
while (remaining > 0) {
transferred = input.transferTo(position, remaining, output);
remaining -= transferred;
position += transferred;
}
}
} finally {
IOUtils.cleanupWithLogger(LOG, output);
IOUtils.cleanupWithLogger(LOG, fos);
IOUtils.cleanupWithLogger(LOG, input);
IOUtils.cleanupWithLogger(LOG, fis);
IOUtils.cleanupWithLogger(LOG, input, fis);
}
}
}

View File

@ -56,7 +56,6 @@ import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.conf.Configuration.IntegerRanges;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.alias.CredentialProvider;
import org.apache.hadoop.security.alias.CredentialProviderFactory;
@ -385,11 +384,9 @@ public class TestConfiguration {
Configuration conf = new Configuration(false);
conf.addResource(new Path(CONFIG_MULTI_BYTE));
assertEquals(value, conf.get(name));
FileOutputStream fos = new FileOutputStream(CONFIG_MULTI_BYTE_SAVED);
try {
try (FileOutputStream fos =
new FileOutputStream(CONFIG_MULTI_BYTE_SAVED)) {
conf.writeXml(fos);
} finally {
IOUtils.closeStream(fos);
}
conf = new Configuration(false);

View File

@ -362,11 +362,10 @@ public class TestCodec {
final Path file = new Path(wd, "test" + codec.getDefaultExtension());
final byte[] b = new byte[REC_SIZE];
final Base64 b64 = new Base64(0, null);
DataOutputStream fout = null;
Compressor cmp = CodecPool.getCompressor(codec);
try {
fout = new DataOutputStream(codec.createOutputStream(
fs.create(file, true), cmp));
try (DataOutputStream fout =
new DataOutputStream(codec.createOutputStream(fs.create(file,
true), cmp))) {
final DataOutputBuffer dob = new DataOutputBuffer(REC_SIZE * 4 / 3 + 4);
int seq = 0;
while (infLen > 0) {
@ -382,7 +381,6 @@ public class TestCodec {
}
LOG.info("Wrote " + seq + " records to " + file);
} finally {
IOUtils.cleanupWithLogger(LOG, fout);
CodecPool.returnCompressor(cmp);
}
return file;

View File

@ -105,15 +105,11 @@ public class MiniKMS {
private void copyResource(String inputResourceName, File outputFile) throws
IOException {
InputStream is = null;
OutputStream os = null;
try {
is = ThreadUtil.getResourceAsStream(inputResourceName);
os = new FileOutputStream(outputFile);
InputStream is = ThreadUtil.getResourceAsStream(inputResourceName);
try (OutputStream os = new FileOutputStream(outputFile)) {
IOUtils.copy(is, os);
} finally {
IOUtils.closeQuietly(is);
IOUtils.closeQuietly(os);
}
}