HADOOP-17788. Replace IOUtils#closeQuietly usages by Hadoop's own utility (#3171)

Reviewed-by: Steve Loughran <stevel@apache.org>
Reviewed-by: Akira Ajisaka <aajisaka@apache.org>
Signed-off-by: Takanobu Asanuma <tasanuma@apache.org>
This commit is contained in:
Viraj Jasani 2021-07-08 12:33:40 +05:30 committed by GitHub
parent b4c2647d0d
commit 618c9218ee
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
39 changed files with 130 additions and 215 deletions

View File

@ -16,10 +16,10 @@
package org.apache.hadoop.io.compress.zstd;
import org.apache.commons.io.FileUtils;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.DataInputBuffer;
import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.compress.CompressionInputStream;
import org.apache.hadoop.io.compress.CompressionOutputStream;
import org.apache.hadoop.io.compress.Compressor;
@ -198,18 +198,16 @@ public class TestZStandardCompressorDecompressor {
@Test
public void testCompressorDecompressorLogicWithCompressionStreams()
throws Exception {
DataOutputStream deflateOut = null;
DataInputStream inflateIn = null;
int byteSize = 1024 * 100;
byte[] bytes = generate(byteSize);
int bufferSize = IO_FILE_BUFFER_SIZE_DEFAULT;
try {
DataOutputBuffer compressedDataBuffer = new DataOutputBuffer();
CompressionOutputStream deflateFilter =
new CompressorStream(compressedDataBuffer, new ZStandardCompressor(),
bufferSize);
deflateOut =
new DataOutputStream(new BufferedOutputStream(deflateFilter));
DataOutputBuffer compressedDataBuffer = new DataOutputBuffer();
CompressionOutputStream deflateFilter =
new CompressorStream(compressedDataBuffer, new ZStandardCompressor(),
bufferSize);
try (DataOutputStream deflateOut =
new DataOutputStream(new BufferedOutputStream(deflateFilter))) {
deflateOut.write(bytes, 0, bytes.length);
deflateOut.flush();
deflateFilter.finish();
@ -229,8 +227,7 @@ public class TestZStandardCompressorDecompressor {
assertArrayEquals("original array not equals compress/decompressed array",
result, bytes);
} finally {
IOUtils.closeQuietly(deflateOut);
IOUtils.closeQuietly(inflateIn);
IOUtils.closeStream(inflateIn);
}
}
@ -358,18 +355,15 @@ public class TestZStandardCompressorDecompressor {
codec.createDecompressor());
byte[] toDecompress = new byte[100];
ByteArrayOutputStream baos = new ByteArrayOutputStream();
byte[] decompressedResult;
int totalFileSize = 0;
int result = toDecompress.length;
try {
try (ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
int result = toDecompress.length;
while ((result = inputStream.read(toDecompress, 0, result)) != -1) {
baos.write(toDecompress, 0, result);
totalFileSize += result;
}
decompressedResult = baos.toByteArray();
} finally {
IOUtils.closeQuietly(baos);
}
assertEquals(decompressedResult.length, totalFileSize);
@ -435,20 +429,16 @@ public class TestZStandardCompressorDecompressor {
ZStandardCodec codec = new ZStandardCodec();
codec.setConf(CONFIGURATION);
Decompressor decompressor = codec.createDecompressor();
CompressionInputStream cis =
codec.createInputStream(inputStream, decompressor);
ByteArrayOutputStream baos = new ByteArrayOutputStream();
byte[] resultOfDecompression;
try {
try (CompressionInputStream cis =
codec.createInputStream(inputStream, decompressor);
ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
byte[] buffer = new byte[100];
int n;
while ((n = cis.read(buffer, 0, buffer.length)) != -1) {
baos.write(buffer, 0, n);
}
resultOfDecompression = baos.toByteArray();
} finally {
IOUtils.closeQuietly(baos);
IOUtils.closeQuietly(cis);
}
byte[] expected = FileUtils.readFileToByteArray(uncompressedFile);

View File

@ -47,13 +47,13 @@ import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Supplier;
import java.util.regex.Pattern;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.RandomStringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.util.BlockingThreadPoolExecutorService;
import org.apache.hadoop.util.DurationInfo;
import org.apache.hadoop.util.StringUtils;
@ -500,7 +500,7 @@ public abstract class GenericTestUtils {
@Override
public void close() throws Exception {
IOUtils.closeQuietly(bytesPrintStream);
IOUtils.closeStream(bytesPrintStream);
System.setErr(oldErr);
}
}
@ -821,12 +821,10 @@ public abstract class GenericTestUtils {
*/
public static String getFilesDiff(File a, File b) throws IOException {
StringBuilder bld = new StringBuilder();
BufferedReader ra = null, rb = null;
try {
ra = new BufferedReader(
new InputStreamReader(new FileInputStream(a)));
rb = new BufferedReader(
new InputStreamReader(new FileInputStream(b)));
try (BufferedReader ra = new BufferedReader(
new InputStreamReader(new FileInputStream(a)));
BufferedReader rb = new BufferedReader(
new InputStreamReader(new FileInputStream(b)))) {
while (true) {
String la = ra.readLine();
String lb = rb.readLine();
@ -846,9 +844,6 @@ public abstract class GenericTestUtils {
bld.append(" + ").append(lb).append("\n");
}
}
} finally {
IOUtils.closeQuietly(ra);
IOUtils.closeQuietly(rb);
}
return bld.toString();
}

View File

@ -23,8 +23,6 @@ import java.io.FileWriter;
import java.io.IOException;
import java.util.Random;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.test.GenericTestUtils;
import org.junit.Test;
@ -521,12 +519,9 @@ public class TestSysInfoLinux {
private void writeFakeCPUInfoFile(String content) throws IOException {
File tempFile = new File(FAKE_CPUFILE);
FileWriter fWriter = new FileWriter(FAKE_CPUFILE);
tempFile.deleteOnExit();
try {
try (FileWriter fWriter = new FileWriter(FAKE_CPUFILE)) {
tempFile.deleteOnExit();
fWriter.write(content);
} finally {
IOUtils.closeQuietly(fWriter);
}
}

View File

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

View File

@ -44,7 +44,6 @@ import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.ByteBufferPositionedReadable;
import org.apache.hadoop.fs.ByteBufferReadable;
@ -77,6 +76,7 @@ import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
import org.apache.hadoop.hdfs.util.IOUtilsClient;
import org.apache.hadoop.io.ByteBufferPool;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.ipc.RetriableException;
@ -1919,7 +1919,7 @@ public class DFSInputStream extends FSInputStream
success = true;
} finally {
if (!success) {
IOUtils.closeQuietly(clientMmap);
IOUtils.closeStream(clientMmap);
}
}
return buffer;
@ -1934,7 +1934,7 @@ public class DFSInputStream extends FSInputStream
"that was not created by this stream, " + buffer);
}
if (val instanceof ClientMmap) {
IOUtils.closeQuietly((ClientMmap)val);
IOUtils.closeStream((ClientMmap)val);
} else if (val instanceof ByteBufferPool) {
((ByteBufferPool)val).putBuffer(buffer);
}

View File

@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs.client;
import java.io.IOException;
import java.net.URI;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
@ -63,24 +62,18 @@ public class HdfsUtils {
conf.setInt(
CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 0);
DistributedFileSystem fs = null;
try {
fs = (DistributedFileSystem)FileSystem.get(uri, conf);
try (DistributedFileSystem fs =
(DistributedFileSystem) FileSystem.get(uri, conf)) {
final boolean safemode = fs.setSafeMode(SafeModeAction.SAFEMODE_GET);
if (LOG.isDebugEnabled()) {
LOG.debug("Is namenode in safemode? " + safemode + "; uri=" + uri);
}
fs.close();
fs = null;
return !safemode;
} catch(IOException e) {
} catch (IOException e) {
if (LOG.isDebugEnabled()) {
LOG.debug("Got an exception for uri=" + uri, e);
}
return false;
} finally {
IOUtils.closeQuietly(fs);
}
}
}

View File

@ -21,8 +21,8 @@ import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.hdfs.DFSUtilClient;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
@ -174,7 +174,7 @@ public class DomainSocketFactory {
} finally {
if (!success) {
if (sock != null) {
IOUtils.closeQuietly(sock);
IOUtils.closeStream(sock);
}
pathMap.put(info.getPath(), PathState.UNUSABLE);
sock = null;

View File

@ -434,8 +434,8 @@ class BlockSender implements java.io.Closeable {
} catch (IOException ioe) {
IOUtils.cleanupWithLogger(null, volumeRef);
IOUtils.closeStream(this);
org.apache.commons.io.IOUtils.closeQuietly(blockIn);
org.apache.commons.io.IOUtils.closeQuietly(checksumIn);
IOUtils.closeStream(blockIn);
IOUtils.closeStream(checksumIn);
throw ioe;
}
}

View File

@ -27,12 +27,12 @@ import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.net.Peer;
import org.apache.hadoop.hdfs.net.PeerServer;
import org.apache.hadoop.hdfs.util.DataTransferThrottler;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
@ -248,10 +248,10 @@ class DataXceiverServer implements Runnable {
LOG.warn("{}:DataXceiverServer", datanode.getDisplayName(), ace);
}
} catch (IOException ie) {
IOUtils.closeQuietly(peer);
IOUtils.closeStream(peer);
LOG.warn("{}:DataXceiverServer", datanode.getDisplayName(), ie);
} catch (OutOfMemoryError ie) {
IOUtils.closeQuietly(peer);
IOUtils.closeStream(peer);
// DataNode can run out of memory if there is too many transfers.
// Log the event, Sleep for 30 seconds, other transfers may complete by
// then.
@ -334,7 +334,7 @@ class DataXceiverServer implements Runnable {
peers.remove(peer);
peersXceiver.remove(peer);
datanode.metrics.decrDataNodeActiveXceiversCount();
IOUtils.closeQuietly(peer);
IOUtils.closeStream(peer);
if (peers.isEmpty()) {
this.noPeers.signalAll();
}
@ -396,7 +396,7 @@ class DataXceiverServer implements Runnable {
LOG.info("Closing all peers.");
lock.lock();
try {
peers.keySet().forEach(p -> IOUtils.closeQuietly(p));
peers.keySet().forEach(IOUtils::closeStream);
peers.clear();
peersXceiver.clear();
datanode.metrics.setDataNodeActiveXceiversCount(0);

View File

@ -336,7 +336,7 @@ public class FileIoProvider {
profilingEventHook.afterMetadataOp(volume, OPEN, begin);
return fis;
} catch(Exception e) {
org.apache.commons.io.IOUtils.closeQuietly(fis);
IOUtils.closeStream(fis);
onFailure(volume, begin);
throw e;
}
@ -367,7 +367,7 @@ public class FileIoProvider {
profilingEventHook.afterMetadataOp(volume, OPEN, begin);
return fos;
} catch(Exception e) {
org.apache.commons.io.IOUtils.closeQuietly(fos);
IOUtils.closeStream(fos);
onFailure(volume, begin);
throw e;
}
@ -432,7 +432,7 @@ public class FileIoProvider {
profilingEventHook.afterMetadataOp(volume, OPEN, begin);
return fis;
} catch(Exception e) {
org.apache.commons.io.IOUtils.closeQuietly(fis);
IOUtils.closeStream(fis);
onFailure(volume, begin);
throw e;
}
@ -464,7 +464,7 @@ public class FileIoProvider {
profilingEventHook.afterMetadataOp(volume, OPEN, begin);
return fis;
} catch(Exception e) {
org.apache.commons.io.IOUtils.closeQuietly(fis);
IOUtils.closeStream(fis);
onFailure(volume, begin);
throw e;
}
@ -495,7 +495,7 @@ public class FileIoProvider {
profilingEventHook.afterMetadataOp(volume, OPEN, begin);
return raf;
} catch(Exception e) {
org.apache.commons.io.IOUtils.closeQuietly(raf);
IOUtils.closeStream(raf);
onFailure(volume, begin);
throw e;
}

View File

@ -30,8 +30,8 @@ import java.util.HashSet;
import java.util.Iterator;
import java.util.Set;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.commons.io.IOUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
@ -322,7 +322,7 @@ public class ShortCircuitRegistry {
shm = new RegisteredShm(clientName, shmId, fis, this);
} finally {
if (shm == null) {
IOUtils.closeQuietly(fis);
IOUtils.closeStream(fis);
}
}
info = new NewShmInfo(shmId, fis);
@ -392,7 +392,7 @@ public class ShortCircuitRegistry {
if (!enabled) return;
enabled = false;
}
IOUtils.closeQuietly(watcher);
IOUtils.closeStream(watcher);
}
public static interface Visitor {

View File

@ -23,6 +23,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_CACHE_REVOCATION
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_CACHE_REVOCATION_POLLING_MS;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_CACHE_REVOCATION_POLLING_MS_DEFAULT;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
@ -44,7 +45,6 @@ import java.util.concurrent.ScheduledThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.LongAdder;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.time.DurationFormatUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
@ -473,15 +473,15 @@ public class FsDatasetCache {
dataset.datanode.getMetrics().incrBlocksCached(1);
success = true;
} finally {
IOUtils.closeQuietly(blockIn);
IOUtils.closeQuietly(metaIn);
IOUtils.closeStream(blockIn);
IOUtils.closeStream(metaIn);
if (!success) {
if (reservedBytes) {
cacheLoader.release(key, length);
}
LOG.debug("Caching of {} was aborted. We are now caching only {} "
+ "bytes in total.", key, cacheLoader.getCacheUsed());
IOUtils.closeQuietly(mappableBlock);
IOUtils.closeStream(mappableBlock);
numBlocksFailedToCache.increment();
synchronized (FsDatasetCache.this) {
@ -555,7 +555,7 @@ public class FsDatasetCache {
Preconditions.checkNotNull(value);
Preconditions.checkArgument(value.state == State.UNCACHING);
IOUtils.closeQuietly(value.mappableBlock);
IOUtils.closeStream(value.mappableBlock);
synchronized (FsDatasetCache.this) {
mappableBlockMap.remove(key);
}

View File

@ -19,7 +19,6 @@
package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.ExtendedBlockId;
@ -136,9 +135,7 @@ public abstract class MappableBlockLoader {
BlockMetadataHeader.readHeader(new DataInputStream(
new BufferedInputStream(metaIn, BlockMetadataHeader
.getHeaderSize())));
FileChannel metaChannel = null;
try {
metaChannel = metaIn.getChannel();
try (FileChannel metaChannel = metaIn.getChannel()) {
if (metaChannel == null) {
throw new IOException(
"Block InputStream meta file has no FileChannel.");
@ -172,8 +169,6 @@ public abstract class MappableBlockLoader {
blockBuf.clear();
checksumBuf.clear();
}
} finally {
IOUtils.closeQuietly(metaChannel);
}
}

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.ExtendedBlockId;
@ -73,9 +72,7 @@ public class MemoryMappableBlockLoader extends MappableBlockLoader {
throws IOException {
MemoryMappedBlock mappableBlock = null;
MappedByteBuffer mmap = null;
FileChannel blockChannel = null;
try {
blockChannel = blockIn.getChannel();
try (FileChannel blockChannel = blockIn.getChannel()) {
if (blockChannel == null) {
throw new IOException("Block InputStream has no FileChannel.");
}
@ -84,7 +81,6 @@ public class MemoryMappableBlockLoader extends MappableBlockLoader {
verifyChecksum(length, metaIn, blockChannel, blockFileName);
mappableBlock = new MemoryMappedBlock(mmap, length);
} finally {
IOUtils.closeQuietly(blockChannel);
if (mappableBlock == null) {
if (mmap != null) {
NativeIO.POSIX.munmap(mmap); // unmapping also unlocks

View File

@ -19,7 +19,6 @@
package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.ExtendedBlockId;
@ -83,9 +82,7 @@ public class NativePmemMappableBlockLoader extends PmemMappableBlockLoader {
POSIX.PmemMappedRegion region = null;
String filePath = null;
FileChannel blockChannel = null;
try {
blockChannel = blockIn.getChannel();
try (FileChannel blockChannel = blockIn.getChannel()) {
if (blockChannel == null) {
throw new IOException("Block InputStream has no FileChannel.");
}
@ -102,10 +99,9 @@ public class NativePmemMappableBlockLoader extends PmemMappableBlockLoader {
mappableBlock = new NativePmemMappedBlock(region.getAddress(),
region.getLength(), key);
LOG.info("Successfully cached one replica:{} into persistent memory"
+ ", [cached path={}, address={}, length={}]", key, filePath,
+ ", [cached path={}, address={}, length={}]", key, filePath,
region.getAddress(), length);
} finally {
IOUtils.closeQuietly(blockChannel);
if (mappableBlock == null) {
if (region != null) {
// unmap content from persistent memory
@ -131,9 +127,7 @@ public class NativePmemMappableBlockLoader extends PmemMappableBlockLoader {
BlockMetadataHeader.readHeader(new DataInputStream(
new BufferedInputStream(metaIn, BlockMetadataHeader
.getHeaderSize())));
FileChannel metaChannel = null;
try {
metaChannel = metaIn.getChannel();
try (FileChannel metaChannel = metaIn.getChannel()) {
if (metaChannel == null) {
throw new IOException("Cannot get FileChannel" +
" from Block InputStream meta file.");
@ -181,8 +175,6 @@ public class NativePmemMappableBlockLoader extends PmemMappableBlockLoader {
if (region != null) {
POSIX.Pmem.memSync(region);
}
} finally {
IOUtils.closeQuietly(metaChannel);
}
}

View File

@ -18,11 +18,11 @@
package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.ExtendedBlockId;
import org.apache.hadoop.hdfs.server.datanode.DNConf;
import org.apache.hadoop.io.IOUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -105,8 +105,8 @@ public class PmemMappableBlockLoader extends MappableBlockLoader {
LOG.info("Successfully cached one replica:{} into persistent memory"
+ ", [cached path={}, length={}]", key, cachePath, length);
} finally {
IOUtils.closeQuietly(blockChannel);
IOUtils.closeQuietly(cacheFile);
IOUtils.closeStream(blockChannel);
IOUtils.closeStream(cacheFile);
if (mappableBlock == null) {
LOG.debug("Delete {} due to unsuccessful mapping.", cachePath);
FsDatasetUtil.deleteMappedFile(cachePath);

View File

@ -18,9 +18,9 @@
package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.commons.io.FileUtils;
import org.apache.commons.io.IOUtils;
import org.apache.commons.io.filefilter.TrueFileFilter;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
@ -357,7 +357,7 @@ public final class PmemVolumeManager {
out.clear();
}
if (testFile != null) {
IOUtils.closeQuietly(testFile);
IOUtils.closeStream(testFile);
NativeIO.POSIX.munmap(out);
try {
FsDatasetUtil.deleteMappedFile(testFilePath);

View File

@ -44,7 +44,6 @@ import java.util.TreeMap;
import java.util.concurrent.locks.ReentrantLock;
import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
@ -83,6 +82,7 @@ import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress.Co
import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step;
import org.apache.hadoop.hdfs.server.namenode.startupprogress.StepType;
import org.apache.hadoop.hdfs.util.ReadOnlyList;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.util.GSet;
import org.apache.hadoop.util.LightWeightGSet;
@ -281,7 +281,7 @@ public class CacheManager {
if (this.monitor != null) {
CacheReplicationMonitor prevMonitor = this.monitor;
this.monitor = null;
IOUtils.closeQuietly(prevMonitor);
IOUtils.closeStream(prevMonitor);
}
} finally {
crmLock.unlock();

View File

@ -37,8 +37,8 @@ import java.util.Map;
import java.util.Set;
import java.util.concurrent.ThreadLocalRandom;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockUnderConstructionFeature;
import org.apache.hadoop.io.IOUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
@ -1109,7 +1109,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
blockToken, datanodeId, HdfsConstants.READ_TIMEOUT);
} finally {
if (peer == null) {
IOUtils.closeQuietly(s);
IOUtils.closeStream(s);
}
}
return peer;

View File

@ -28,7 +28,6 @@ import java.net.Socket;
import java.util.List;
import java.util.Random;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.BlockReader;
@ -57,6 +56,7 @@ import org.apache.hadoop.hdfs.shortcircuit.DfsClientShmManager;
import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache;
import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitReplica;
import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.test.GenericTestUtils;
@ -220,7 +220,7 @@ public class BlockReaderTestUtil {
peer = DFSUtilClient.peerFromSocket(sock);
} finally {
if (peer == null) {
IOUtils.closeQuietly(sock);
IOUtils.closeStream(sock);
}
}
return peer;

View File

@ -45,8 +45,8 @@ import java.util.Set;
import java.util.UUID;
import java.util.concurrent.TimeoutException;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
@ -151,7 +151,7 @@ public abstract class LazyPersistTestCase {
jmx = null;
}
IOUtils.closeQuietly(sockDir);
IOUtils.closeStream(sockDir);
sockDir = null;
}
@ -252,16 +252,13 @@ public abstract class LazyPersistTestCase {
createFlags.add(LAZY_PERSIST);
}
FSDataOutputStream fos = null;
try {
fos =
fs.create(path,
FsPermission.getFileDefault(),
createFlags,
BUFFER_LENGTH,
REPL_FACTOR,
BLOCK_SIZE,
null);
try (FSDataOutputStream fos = fs.create(path,
FsPermission.getFileDefault(),
createFlags,
BUFFER_LENGTH,
REPL_FACTOR,
BLOCK_SIZE,
null)) {
// Allocate a block.
byte[] buffer = new byte[BUFFER_LENGTH];
@ -272,8 +269,6 @@ public abstract class LazyPersistTestCase {
if (length > 0) {
fos.hsync();
}
} finally {
IOUtils.closeQuietly(fos);
}
}

View File

@ -16,8 +16,8 @@
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.fs.ChecksumException;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.ClientContext;
@ -125,8 +125,7 @@ public class TestScrLazyPersistFiles extends LazyPersistTestCase {
ensureFileReplicasOnStorageType(path1, RAM_DISK);
waitForMetric("RamDiskBlocksLazyPersisted", 1);
HdfsDataInputStream fis = (HdfsDataInputStream) fs.open(path1);
try {
try (HdfsDataInputStream fis = (HdfsDataInputStream) fs.open(path1)) {
// Keep and open read handle to path1 while creating path2
byte[] buf = new byte[BUFFER_LENGTH];
fis.read(0, buf, 0, BUFFER_LENGTH);
@ -138,8 +137,6 @@ public class TestScrLazyPersistFiles extends LazyPersistTestCase {
is((long) 2 * BUFFER_LENGTH));
assertThat(fis.getReadStatistics().getTotalShortCircuitBytesRead(),
is((long) 2 * BUFFER_LENGTH));
} finally {
IOUtils.closeQuietly(fis);
}
}

View File

@ -19,7 +19,8 @@
package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
import java.util.function.Supplier;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.io.IOUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
@ -554,7 +555,7 @@ public class TestSpaceReservation {
String filename = "/file-" + rand.nextLong();
os = localClient.create(filename, false);
os.write(data, 0, rand.nextInt(data.length));
IOUtils.closeQuietly(os);
IOUtils.closeStream(os);
os = null;
localClient.delete(filename, false);
Thread.sleep(50); // Sleep for a bit to avoid killing the system.
@ -566,7 +567,7 @@ public class TestSpaceReservation {
return;
} finally {
if (os != null) {
IOUtils.closeQuietly(os);
IOUtils.closeStream(os);
}
}
}

View File

@ -41,7 +41,6 @@ import java.util.regex.Pattern;
import javax.crypto.KeyGenerator;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
@ -1474,9 +1473,8 @@ public class MRAppMaster extends CompositeService {
private List<AMInfo> readJustAMInfos() {
List<AMInfo> amInfos = new ArrayList<AMInfo>();
FSDataInputStream inputStream = null;
try {
inputStream = getPreviousJobHistoryStream(getConfig(), appAttemptID);
try (FSDataInputStream inputStream =
getPreviousJobHistoryStream(getConfig(), appAttemptID)) {
EventReader jobHistoryEventReader = new EventReader(inputStream);
// All AMInfos are contiguous. Track when the first AMStartedEvent
@ -1492,11 +1490,11 @@ public class MRAppMaster extends CompositeService {
}
AMStartedEvent amStartedEvent = (AMStartedEvent) event;
amInfos.add(MRBuilderUtils.newAMInfo(
amStartedEvent.getAppAttemptId(), amStartedEvent.getStartTime(),
amStartedEvent.getContainerId(),
StringInterner.weakIntern(amStartedEvent.getNodeManagerHost()),
amStartedEvent.getNodeManagerPort(),
amStartedEvent.getNodeManagerHttpPort()));
amStartedEvent.getAppAttemptId(), amStartedEvent.getStartTime(),
amStartedEvent.getContainerId(),
StringInterner.weakIntern(amStartedEvent.getNodeManagerHost()),
amStartedEvent.getNodeManagerPort(),
amStartedEvent.getNodeManagerHttpPort()));
} else if (amStartedEventsBegan) {
// This means AMStartedEvents began and this event is a
// non-AMStarted event.
@ -1507,10 +1505,6 @@ public class MRAppMaster extends CompositeService {
} catch (IOException e) {
LOG.warn("Could not parse the old history file. "
+ "Will not have old AMinfos ", e);
} finally {
if (inputStream != null) {
IOUtils.closeQuietly(inputStream);
}
}
return amInfos;
}

View File

@ -1619,12 +1619,9 @@ public class ApplicationMaster {
}
private String readContent(String filePath) throws IOException {
DataInputStream ds = null;
try {
ds = new DataInputStream(new FileInputStream(filePath));
try (DataInputStream ds = new DataInputStream(
new FileInputStream(filePath))) {
return ds.readUTF();
} finally {
org.apache.commons.io.IOUtils.closeQuietly(ds);
}
}

View File

@ -42,7 +42,6 @@ import org.apache.commons.cli.HelpFormatter;
import org.apache.commons.cli.Option;
import org.apache.commons.cli.Options;
import org.apache.commons.cli.ParseException;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
@ -1221,13 +1220,9 @@ public class Client {
Path dst =
new Path(fs.getHomeDirectory(), suffix);
if (fileSrcPath == null) {
FSDataOutputStream ostream = null;
try {
ostream = FileSystem
.create(fs, dst, new FsPermission((short) 0710));
try (FSDataOutputStream ostream = FileSystem.create(fs, dst,
new FsPermission((short) 0710))) {
ostream.writeUTF(resources);
} finally {
IOUtils.closeQuietly(ostream);
}
} else {
fs.copyFromLocalFile(new Path(fileSrcPath), dst);

View File

@ -23,21 +23,17 @@ import java.io.InputStream;
import java.util.Map.Entry;
import java.util.Properties;
import org.apache.commons.io.IOUtils;
import org.apache.log4j.LogManager;
import org.apache.log4j.PropertyConfigurator;
public class Log4jPropertyHelper {
public static void updateLog4jConfiguration(Class<?> targetClass,
String log4jPath) throws Exception {
Properties customProperties = new Properties();
FileInputStream fs = null;
InputStream is = null;
try {
fs = new FileInputStream(log4jPath);
is = targetClass.getResourceAsStream("/log4j.properties");
try (
FileInputStream fs = new FileInputStream(log4jPath);
InputStream is = targetClass.getResourceAsStream("/log4j.properties")) {
customProperties.load(fs);
Properties originalProperties = new Properties();
originalProperties.load(is);
@ -47,9 +43,6 @@ public class Log4jPropertyHelper {
}
LogManager.resetConfiguration();
PropertyConfigurator.configure(originalProperties);
}finally {
IOUtils.closeQuietly(is);
IOUtils.closeQuietly(fs);
}
}
}

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.yarn.client.cli;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
import com.sun.jersey.api.client.Client;
import com.sun.jersey.api.client.ClientHandlerException;
@ -55,7 +56,6 @@ import org.apache.commons.cli.HelpFormatter;
import org.apache.commons.cli.Option;
import org.apache.commons.cli.Options;
import org.apache.commons.cli.ParseException;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.math3.util.Pair;
import org.apache.hadoop.classification.InterfaceAudience.Private;
@ -643,7 +643,7 @@ public class LogsCLI extends Configured implements Tool {
+ " for the container:" + containerIdStr + " in NodeManager:"
+ nodeId);
} finally {
IOUtils.closeQuietly(is);
IOUtils.closeStream(is);
}
}

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.yarn.client.api.impl;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileSystemTestHelper;
@ -28,6 +27,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.io.DataInputByteBuffer;
import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.hadoop.security.Credentials;
@ -119,7 +119,7 @@ public class TestYarnClientImpl extends ParameterizedSchedulerTestBase {
Assert.assertEquals(
expectedTimeoutEnforcement, client.enforceAsyncAPITimeout());
} finally {
IOUtils.closeQuietly(client);
IOUtils.closeStream(client);
}
}

View File

@ -36,6 +36,7 @@ import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableList;
import com.sun.jersey.api.client.Client;
import com.sun.jersey.api.client.ClientResponse;
@ -59,7 +60,6 @@ import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
@ -939,7 +939,7 @@ public class TestLogsCLI {
logMessage(containerId1, "syslog")));
sysOutStream.reset();
} finally {
IOUtils.closeQuietly(fis);
IOUtils.closeStream(fis);
fs.delete(new Path(rootLogDir), true);
}
}
@ -1477,19 +1477,13 @@ public class TestLogsCLI {
FileSystem fs) throws IOException {
assertTrue(fs.exists(containerPath));
StringBuffer inputLine = new StringBuffer();
BufferedReader reader = null;
try {
reader = new BufferedReader(new InputStreamReader(
fs.open(containerPath)));
try (BufferedReader reader = new BufferedReader(new InputStreamReader(
fs.open(containerPath)))) {
String tmp;
while ((tmp = reader.readLine()) != null) {
inputLine.append(tmp);
}
return inputLine.toString();
} finally {
if (reader != null) {
IOUtils.closeQuietly(reader);
}
}
}

View File

@ -26,7 +26,6 @@ import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.conf.Configurable;
@ -35,6 +34,7 @@ import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.api.records.ApplicationId;
@ -407,7 +407,7 @@ public class LogCLIHelpers implements Configurable {
public void closePrintStream(PrintStream out) {
if (out != System.out) {
IOUtils.closeQuietly(out);
IOUtils.closeStream(out);
}
}

View File

@ -28,12 +28,13 @@ import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.compress.Decompressor;
import org.apache.hadoop.io.file.tfile.BoundedRangeFileInputStream;
import org.apache.hadoop.io.file.tfile.Compression;
@ -232,7 +233,7 @@ public class IndexedFileAggregatedLogsBlock extends LogAggregationHtmlBlock {
LOG.error("Error getting logs for " + logEntity, ex);
continue;
} finally {
IOUtils.closeQuietly(in);
IOUtils.closeStream(in);
}
}
}

View File

@ -806,7 +806,7 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
} catch (Throwable t) {
LOG.error(t.toString());
} finally {
IOUtils.closeQuietly(in);
org.apache.hadoop.io.IOUtils.closeStream(in);
}
}

View File

@ -32,7 +32,6 @@ import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.FSDataOutputStream;
@ -41,6 +40,7 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@ -267,7 +267,7 @@ public class TestLogAggregationIndexedFileController
Charset.forName("UTF-8")));
fInput.writeLong(0);
} finally {
IOUtils.closeQuietly(fInput);
IOUtils.closeStream(fInput);
}
meta = fileFormat.readAggregatedLogsMeta(
logRequest);
@ -463,7 +463,7 @@ public class TestLogAggregationIndexedFileController
writer.close();
return file;
} finally {
IOUtils.closeQuietly(writer);
IOUtils.closeStream(writer);
}
}
@ -578,7 +578,7 @@ public class TestLogAggregationIndexedFileController
Charset.forName("UTF-8")));
fInput.writeLong(0);
} finally {
IOUtils.closeQuietly(fInput);
IOUtils.closeStream(fInput);
}
Path nodePath = LogAggregationUtils.getRemoteAppLogDir(

View File

@ -30,6 +30,7 @@ import java.util.List;
import java.util.Map.Entry;
import java.util.Set;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.records.AuxServiceRecord;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.records.AuxServiceRecords;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.ResourcePlugin;
@ -55,7 +56,6 @@ import javax.ws.rs.core.Response.ResponseBuilder;
import javax.ws.rs.core.Response.Status;
import javax.ws.rs.core.StreamingOutput;
import javax.ws.rs.core.UriInfo;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
@ -518,7 +518,7 @@ public class NMWebServices {
}
}
} finally {
IOUtils.closeQuietly(fis);
IOUtils.closeStream(fis);
}
}
};

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.yarn.server.nodemanager.containermanager.container;
import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableList;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.yarn.server.nodemanager.api.deviceplugin.Device;
import org.junit.Assert;
import org.junit.BeforeClass;
@ -103,15 +102,11 @@ public class TestResourceMappings {
* @throws IOException
*/
private byte[] toBytes(List<Serializable> resources) throws IOException {
ObjectOutputStream oos = null;
byte[] bytes;
try {
ByteArrayOutputStream bos = new ByteArrayOutputStream();
oos = new ObjectOutputStream(bos);
ByteArrayOutputStream bos = new ByteArrayOutputStream();
try (ObjectOutputStream oos = new ObjectOutputStream(bos)) {
oos.writeObject(resources);
bytes = bos.toByteArray();
} finally {
IOUtils.closeQuietly(oos);
}
return bytes;
}

View File

@ -25,10 +25,10 @@ import java.net.MalformedURLException;
import java.net.URL;
import java.util.TimerTask;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.nodelabels.NodeLabelTestBase;
import org.junit.After;
@ -154,7 +154,7 @@ public class TestConfigurationNodeLabelsProvider extends NodeLabelTestBase {
conf.set(YarnConfiguration.NM_PROVIDER_CONFIGURED_NODE_PARTITION, nodeLabels);
FileOutputStream confStream = new FileOutputStream(nodeLabelsConfigFile);
conf.writeXml(confStream);
IOUtils.closeQuietly(confStream);
IOUtils.closeStream(confStream);
}
private static class XMLPathClassLoader extends ClassLoader {

View File

@ -39,9 +39,9 @@ import javax.ws.rs.core.MediaType;
import javax.xml.bind.JAXBContext;
import javax.xml.bind.Marshaller;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.minikdc.MiniKdc;
import org.apache.hadoop.security.AuthenticationFilterInitializer;
import org.apache.hadoop.security.UserGroupInformation;
@ -367,8 +367,8 @@ public class TestRMWebServicesDelegationTokenAuthentication {
}
}
} finally {
IOUtils.closeQuietly(reader);
IOUtils.closeQuietly(response);
IOUtils.closeStream(reader);
IOUtils.closeStream(response);
}
Assert.assertEquals("client2", owner);
Token<RMDelegationTokenIdentifier> realToken = new Token<RMDelegationTokenIdentifier>();
@ -431,10 +431,10 @@ public class TestRMWebServicesDelegationTokenAuthentication {
setupConn(conn, "POST", MediaType.APPLICATION_JSON, body);
InputStream response = conn.getInputStream();
assertEquals(Status.OK.getStatusCode(), conn.getResponseCode());
BufferedReader reader = null;
try {
reader = new BufferedReader(new InputStreamReader(response, "UTF8"));
for (String line; (line = reader.readLine()) != null;) {
try (BufferedReader reader = new BufferedReader(new InputStreamReader(
response, "UTF8"))) {
String line;
while ((line = reader.readLine()) != null) {
JSONObject obj = new JSONObject(line);
if (obj.has("token")) {
reader.close();
@ -444,8 +444,7 @@ public class TestRMWebServicesDelegationTokenAuthentication {
}
}
} finally {
IOUtils.closeQuietly(reader);
IOUtils.closeQuietly(response);
IOUtils.closeStream(response);
}
return ret;
}

View File

@ -23,8 +23,8 @@ import java.util.EnumSet;
import javax.servlet.http.HttpServletRequest;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
@ -180,7 +180,7 @@ public final class TimelineReaderWebServicesUtils {
try {
return parser.parse();
} finally {
IOUtils.closeQuietly(parser);
IOUtils.closeStream(parser);
}
}