HADOOP-17622. Avoid usage of deprecated IOUtils#cleanup API. (#2862)
Signed-off-by: Takanobu Asanuma <tasanuma@apache.org>
This commit is contained in:
parent
26b8f678b2
commit
3f2682b92b
@ -312,7 +312,7 @@ public void testHasFileDescriptor() throws IOException {
|
||||
.new LocalFSFileInputStream(path), 1024);
|
||||
assertNotNull(bis.getFileDescriptor());
|
||||
} finally {
|
||||
IOUtils.cleanup(null, bis);
|
||||
IOUtils.cleanupWithLogger(null, bis);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -115,7 +115,7 @@ public void testFileSink() throws IOException {
|
||||
IOUtils.copyBytes(is, baos, 1024, true);
|
||||
outFileContent = new String(baos.toByteArray(), "UTF-8");
|
||||
} finally {
|
||||
IOUtils.cleanup(null, baos, is);
|
||||
IOUtils.cleanupWithLogger(null, baos, is);
|
||||
}
|
||||
|
||||
// Check the out file content. Should be something like the following:
|
||||
|
@ -759,6 +759,6 @@ public void run() {
|
||||
readerThread.join();
|
||||
Assert.assertFalse(failed.get());
|
||||
Assert.assertEquals(3, bytesRead.get());
|
||||
IOUtils.cleanup(null, socks);
|
||||
IOUtils.cleanupWithLogger(null, socks);
|
||||
}
|
||||
}
|
||||
|
@ -618,8 +618,8 @@ void join() {
|
||||
private synchronized void cleanUp() {
|
||||
|
||||
shouldServiceRun = false;
|
||||
IOUtils.cleanup(null, bpNamenode);
|
||||
IOUtils.cleanup(null, lifelineSender);
|
||||
IOUtils.cleanupWithLogger(null, bpNamenode);
|
||||
IOUtils.cleanupWithLogger(null, lifelineSender);
|
||||
bpos.shutdownActor(this);
|
||||
}
|
||||
|
||||
@ -992,7 +992,7 @@ public void close() {
|
||||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
IOUtils.cleanup(null, lifelineNamenode);
|
||||
IOUtils.cleanupWithLogger(null, lifelineNamenode);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -369,7 +369,7 @@ public void close() throws IOException {
|
||||
streams.close();
|
||||
}
|
||||
if (replicaHandler != null) {
|
||||
IOUtils.cleanup(null, replicaHandler);
|
||||
IOUtils.cleanupWithLogger(null, replicaHandler);
|
||||
replicaHandler = null;
|
||||
}
|
||||
if (measuredFlushTime) {
|
||||
|
@ -252,7 +252,7 @@ public synchronized void addVolumeScanner(FsVolumeReference ref) {
|
||||
if (!success) {
|
||||
// If we didn't create a new VolumeScanner object, we don't
|
||||
// need this reference to the volume.
|
||||
IOUtils.cleanup(null, ref);
|
||||
IOUtils.cleanupWithLogger(null, ref);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -432,7 +432,7 @@ public void requestShortCircuitFds(final ExtendedBlock blk,
|
||||
blk.getBlockId(), dnR.getDatanodeUuid(), success));
|
||||
}
|
||||
if (fis != null) {
|
||||
IOUtils.cleanup(null, fis);
|
||||
IOUtils.cleanupWithLogger(null, fis);
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -555,7 +555,7 @@ public void requestShortCircuitShm(String clientName) throws IOException {
|
||||
LOG.warn("Failed to shut down socket in error handler", e);
|
||||
}
|
||||
}
|
||||
IOUtils.cleanup(null, shmInfo);
|
||||
IOUtils.cleanupWithLogger(null, shmInfo);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -451,7 +451,7 @@ private long scanBlock(ExtendedBlock cblock, long bytesPerSec) {
|
||||
} catch (IOException e) {
|
||||
resultHandler.handle(block, e);
|
||||
} finally {
|
||||
IOUtils.cleanup(null, blockSender);
|
||||
IOUtils.cleanupWithLogger(null, blockSender);
|
||||
}
|
||||
metrics.incrBlockVerificationFailures();
|
||||
return -1;
|
||||
@ -674,13 +674,13 @@ public void run() {
|
||||
// Save the current position of all block iterators and close them.
|
||||
for (BlockIterator iter : blockIters) {
|
||||
saveBlockIterator(iter);
|
||||
IOUtils.cleanup(null, iter);
|
||||
IOUtils.cleanupWithLogger(null, iter);
|
||||
}
|
||||
} finally {
|
||||
VolumeScannerCBInjector.get().terminationCallBack(this);
|
||||
// When the VolumeScanner exits, release the reference we were holding
|
||||
// on the volume. This will allow the volume to be removed later.
|
||||
IOUtils.cleanup(null, ref);
|
||||
IOUtils.cleanupWithLogger(null, ref);
|
||||
}
|
||||
}
|
||||
|
||||
@ -767,7 +767,7 @@ public synchronized void disableBlockPoolId(String bpid) {
|
||||
if (iter.getBlockPoolId().equals(bpid)) {
|
||||
LOG.trace("{}: disabling scanning on block pool {}", this, bpid);
|
||||
i.remove();
|
||||
IOUtils.cleanup(null, iter);
|
||||
IOUtils.cleanupWithLogger(null, iter);
|
||||
if (curBlockIter == iter) {
|
||||
curBlockIter = null;
|
||||
}
|
||||
|
@ -234,7 +234,7 @@ public void call(Set<FsVolumeSpi> ignored1,
|
||||
}
|
||||
}), MoreExecutors.directExecutor());
|
||||
} else {
|
||||
IOUtils.cleanup(null, reference);
|
||||
IOUtils.cleanupWithLogger(null, reference);
|
||||
if (numVolumes.decrementAndGet() == 0) {
|
||||
latch.countDown();
|
||||
}
|
||||
@ -311,7 +311,7 @@ public boolean checkVolume(
|
||||
);
|
||||
return true;
|
||||
} else {
|
||||
IOUtils.cleanup(null, volumeReference);
|
||||
IOUtils.cleanupWithLogger(null, volumeReference);
|
||||
}
|
||||
return false;
|
||||
}
|
||||
@ -404,7 +404,7 @@ private void markFailed() {
|
||||
}
|
||||
|
||||
private void cleanup() {
|
||||
IOUtils.cleanup(null, reference);
|
||||
IOUtils.cleanupWithLogger(null, reference);
|
||||
invokeCallback();
|
||||
}
|
||||
|
||||
|
@ -158,7 +158,7 @@ private Peer newConnectedPeer(ExtendedBlock b, InetSocketAddress addr,
|
||||
return peer;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.cleanup(null, peer);
|
||||
IOUtils.cleanupWithLogger(null, peer);
|
||||
IOUtils.closeSocket(sock);
|
||||
}
|
||||
}
|
||||
|
@ -130,7 +130,7 @@ public void closeStreams() throws IOException {
|
||||
dataInFd = null;
|
||||
}
|
||||
if (volumeRef != null) {
|
||||
IOUtils.cleanup(null, volumeRef);
|
||||
IOUtils.cleanupWithLogger(null, volumeRef);
|
||||
volumeRef = null;
|
||||
}
|
||||
// throw IOException if there is any
|
||||
@ -146,7 +146,7 @@ public void close() {
|
||||
dataInFd = null;
|
||||
IOUtils.closeStream(checksumIn);
|
||||
checksumIn = null;
|
||||
IOUtils.cleanup(null, volumeRef);
|
||||
IOUtils.cleanupWithLogger(null, volumeRef);
|
||||
volumeRef = null;
|
||||
}
|
||||
}
|
@ -335,7 +335,7 @@ public void run() {
|
||||
+ block.getLocalBlock() + " URI " + replicaToDelete.getBlockURI());
|
||||
}
|
||||
updateDeletedBlockId(block);
|
||||
IOUtils.cleanup(null, volumeRef);
|
||||
IOUtils.cleanupWithLogger(null, volumeRef);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -949,11 +949,11 @@ public ReplicaInputStreams getTmpInputStreams(ExtendedBlock b,
|
||||
return new ReplicaInputStreams(
|
||||
blockInStream, metaInStream, ref, datanode.getFileIoProvider());
|
||||
} catch (IOException e) {
|
||||
IOUtils.cleanup(null, blockInStream);
|
||||
IOUtils.cleanupWithLogger(null, blockInStream);
|
||||
throw e;
|
||||
}
|
||||
} catch (IOException e) {
|
||||
IOUtils.cleanup(null, ref);
|
||||
IOUtils.cleanupWithLogger(null, ref);
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
@ -1421,7 +1421,7 @@ public ReplicaHandler append(ExtendedBlock b,
|
||||
replica = append(b.getBlockPoolId(), replicaInfo, newGS,
|
||||
b.getNumBytes());
|
||||
} catch (IOException e) {
|
||||
IOUtils.cleanup(null, ref);
|
||||
IOUtils.cleanupWithLogger(null, ref);
|
||||
throw e;
|
||||
}
|
||||
return new ReplicaHandler(replica, ref);
|
||||
@ -1553,7 +1553,7 @@ public ReplicaHandler recoverAppend(
|
||||
replica = (ReplicaInPipeline) replicaInfo;
|
||||
}
|
||||
} catch (IOException e) {
|
||||
IOUtils.cleanup(null, ref);
|
||||
IOUtils.cleanupWithLogger(null, ref);
|
||||
throw e;
|
||||
}
|
||||
return new ReplicaHandler(replica, ref);
|
||||
@ -1648,7 +1648,7 @@ public ReplicaHandler createRbw(
|
||||
+ " for block " + b.getBlockId());
|
||||
}
|
||||
} catch (IOException e) {
|
||||
IOUtils.cleanup(null, ref);
|
||||
IOUtils.cleanupWithLogger(null, ref);
|
||||
throw e;
|
||||
}
|
||||
|
||||
@ -1752,7 +1752,7 @@ private ReplicaHandler recoverRbwImpl(ReplicaInPipeline rbw,
|
||||
// bump the replica's generation stamp to newGS
|
||||
rbw.getReplicaInfo().bumpReplicaGS(newGS);
|
||||
} catch (IOException e) {
|
||||
IOUtils.cleanup(null, ref);
|
||||
IOUtils.cleanupWithLogger(null, ref);
|
||||
throw e;
|
||||
}
|
||||
return new ReplicaHandler(rbw, ref);
|
||||
@ -1905,7 +1905,7 @@ public ReplicaHandler createTemporary(StorageType storageType,
|
||||
try {
|
||||
newReplicaInfo = v.createTemporary(b);
|
||||
} catch (IOException e) {
|
||||
IOUtils.cleanup(null, ref);
|
||||
IOUtils.cleanupWithLogger(null, ref);
|
||||
throw e;
|
||||
}
|
||||
|
||||
|
@ -117,7 +117,7 @@ public static FileDescriptor openAndSeek(File file, long offset)
|
||||
}
|
||||
return raf.getFD();
|
||||
} catch(IOException ioe) {
|
||||
IOUtils.cleanup(null, raf);
|
||||
IOUtils.cleanupWithLogger(null, raf);
|
||||
throw ioe;
|
||||
}
|
||||
}
|
||||
|
@ -357,7 +357,7 @@ void addVolume(FsVolumeReference ref) throws IOException {
|
||||
} else {
|
||||
// If the volume is not put into a volume scanner, it does not need to
|
||||
// hold the reference.
|
||||
IOUtils.cleanup(null, ref);
|
||||
IOUtils.cleanupWithLogger(null, ref);
|
||||
}
|
||||
// If the volume is used to replace a failed volume, it needs to reset the
|
||||
// volume failure info for this volume.
|
||||
|
@ -76,7 +76,7 @@ public void close() throws IOException {
|
||||
+ " bytes still to be flushed and cannot be closed.");
|
||||
}
|
||||
|
||||
IOUtils.cleanup(null, bufCurrent, bufReady);
|
||||
IOUtils.cleanupWithLogger(null, bufCurrent, bufReady);
|
||||
bufCurrent = bufReady = null;
|
||||
}
|
||||
|
||||
|
@ -202,7 +202,7 @@ int run(List<String> args) throws IOException {
|
||||
blockFile);
|
||||
return 0;
|
||||
} finally {
|
||||
IOUtils.cleanup(null, metaStream, dataStream, checksumStream);
|
||||
IOUtils.cleanupWithLogger(null, metaStream, dataStream, checksumStream);
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -287,7 +287,7 @@ int run(List<String> args) throws IOException {
|
||||
+ " saved metadata to meta file " + outFile);
|
||||
return 0;
|
||||
} finally {
|
||||
IOUtils.cleanup(null, metaOut);
|
||||
IOUtils.cleanupWithLogger(null, metaOut);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -319,10 +319,10 @@ private static class LevelDBStore implements Closeable {
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (batch != null) {
|
||||
IOUtils.cleanup(null, batch);
|
||||
IOUtils.cleanupWithLogger(null, batch);
|
||||
batch = null;
|
||||
}
|
||||
IOUtils.cleanup(null, db);
|
||||
IOUtils.cleanupWithLogger(null, db);
|
||||
db = null;
|
||||
}
|
||||
|
||||
@ -388,13 +388,13 @@ protected boolean removeEldestEntry(Map.Entry<Long, String> entry) {
|
||||
dirMap = new LevelDBStore(new File(dbDir, "dirMap"));
|
||||
} catch (IOException e) {
|
||||
LOG.error("Failed to open LevelDBs", e);
|
||||
IOUtils.cleanup(null, this);
|
||||
IOUtils.cleanupWithLogger(null, this);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
IOUtils.cleanup(null, dirChildMap, dirMap);
|
||||
IOUtils.cleanupWithLogger(null, dirChildMap, dirMap);
|
||||
dirChildMap = null;
|
||||
dirMap = null;
|
||||
}
|
||||
@ -515,7 +515,7 @@ public long getParentId(long id) throws IOException {
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
out.flush();
|
||||
IOUtils.cleanup(null, metadataMap);
|
||||
IOUtils.cleanupWithLogger(null, metadataMap);
|
||||
}
|
||||
|
||||
void append(StringBuffer buffer, int field) {
|
||||
|
@ -841,7 +841,7 @@ public void test2GBMmapLimit() throws Exception {
|
||||
if (buf2 != null) {
|
||||
fsIn2.releaseBuffer(buf2);
|
||||
}
|
||||
IOUtils.cleanup(null, fsIn, fsIn2);
|
||||
IOUtils.cleanupWithLogger(null, fsIn, fsIn2);
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
|
@ -87,7 +87,7 @@ public void testUnbufferClosesSockets() throws Exception {
|
||||
Assert.assertEquals(b, b2);
|
||||
} finally {
|
||||
if (stream != null) {
|
||||
IOUtils.cleanup(null, stream);
|
||||
IOUtils.cleanupWithLogger(null, stream);
|
||||
}
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
@ -122,7 +122,7 @@ public void testOpenManyFilesViaTcp() throws Exception {
|
||||
}
|
||||
} finally {
|
||||
for (FSDataInputStream stream : streams) {
|
||||
IOUtils.cleanup(null, stream);
|
||||
IOUtils.cleanupWithLogger(null, stream);
|
||||
}
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
|
@ -96,7 +96,7 @@ public void setup() throws Exception {
|
||||
|
||||
@AfterClass
|
||||
public static void shutdown() throws Exception {
|
||||
IOUtils.cleanup(null, hdfs, hdfsAsUser1, hdfsAsUser2);
|
||||
IOUtils.cleanupWithLogger(null, hdfs, hdfsAsUser1, hdfsAsUser2);
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
@ -121,7 +121,7 @@ private void confirmCanAppend(Configuration conf, Path p) throws Exception {
|
||||
h.close();
|
||||
h = null;
|
||||
} finally {
|
||||
IOUtils.cleanup(null, h);
|
||||
IOUtils.cleanupWithLogger(null, h);
|
||||
}
|
||||
}
|
||||
|
||||
@ -500,7 +500,7 @@ static private void writeFile(FileSystem hdfs, Path p) throws IOException {
|
||||
o.close();
|
||||
o = null;
|
||||
} finally {
|
||||
IOUtils.cleanup(null, o);
|
||||
IOUtils.cleanupWithLogger(null, o);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -227,7 +227,7 @@ public void testManyClosedSocketsInCache() throws Exception {
|
||||
IOUtils.copyBytes(stm, new IOUtils.NullOutputStream(), 1024);
|
||||
}
|
||||
} finally {
|
||||
IOUtils.cleanup(null, stms);
|
||||
IOUtils.cleanupWithLogger(null, stms);
|
||||
}
|
||||
|
||||
assertEquals(5, peerCache.size());
|
||||
|
@ -177,7 +177,7 @@ public void hSyncEndBlock_00() throws IOException {
|
||||
blocks = fileSystem.dfs.getLocatedBlocks(path.toString(), 0);
|
||||
assertEquals(3, blocks.getLocatedBlocks().size());
|
||||
} finally {
|
||||
IOUtils.cleanup(null, stm, fileSystem);
|
||||
IOUtils.cleanupWithLogger(null, stm, fileSystem);
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
|
@ -363,8 +363,8 @@ public Void answer(InvocationOnMock invocation) throws Throwable {
|
||||
assertTrue(false);
|
||||
} finally {
|
||||
Mockito.reset(injector);
|
||||
IOUtils.cleanup(null, input);
|
||||
IOUtils.cleanup(null, output);
|
||||
IOUtils.cleanupWithLogger(null, input);
|
||||
IOUtils.cleanupWithLogger(null, output);
|
||||
fileSys.close();
|
||||
cluster.shutdown();
|
||||
}
|
||||
|
@ -741,7 +741,7 @@ public void testCheckpointWithSNN() throws Exception {
|
||||
// do checkpoint in SNN again
|
||||
snn.doCheckpoint();
|
||||
} finally {
|
||||
IOUtils.cleanup(null, dfs);
|
||||
IOUtils.cleanupWithLogger(null, dfs);
|
||||
if (snn != null) {
|
||||
snn.shutdown();
|
||||
}
|
||||
|
@ -273,7 +273,7 @@ public void setup() throws Exception {
|
||||
|
||||
@After
|
||||
public void shutdown() throws IOException {
|
||||
IOUtils.cleanup(null, fs);
|
||||
IOUtils.cleanupWithLogger(null, fs);
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
cluster = null;
|
||||
|
@ -229,7 +229,7 @@ public static void init() throws Exception {
|
||||
|
||||
@AfterClass
|
||||
public static void destroy() {
|
||||
IOUtils.cleanup(null, fs);
|
||||
IOUtils.cleanupWithLogger(null, fs);
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
cluster = null;
|
||||
|
@ -54,7 +54,7 @@ public void testWriteConf() throws Exception {
|
||||
fs.close();
|
||||
fs = null;
|
||||
} finally {
|
||||
IOUtils.cleanup(null, os, fs);
|
||||
IOUtils.cleanupWithLogger(null, os, fs);
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
|
@ -256,7 +256,7 @@ public DataEncryptionKey newDataEncryptionKey() {
|
||||
} catch (SocketTimeoutException e) {
|
||||
GenericTestUtils.assertExceptionContains("Read timed out", e);
|
||||
} finally {
|
||||
IOUtils.cleanup(null, socket, serverSocket);
|
||||
IOUtils.cleanupWithLogger(null, socket, serverSocket);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -173,7 +173,7 @@ public static long recoverAndReturnLastTxn(QuorumJournalManager qjm)
|
||||
lastRecoveredTxn = elis.getLastTxId();
|
||||
}
|
||||
} finally {
|
||||
IOUtils.cleanup(null, streams.toArray(new Closeable[0]));
|
||||
IOUtils.cleanupWithLogger(null, streams.toArray(new Closeable[0]));
|
||||
}
|
||||
return lastRecoveredTxn;
|
||||
}
|
||||
|
@ -163,7 +163,7 @@ public void setup() throws Exception {
|
||||
|
||||
@After
|
||||
public void shutdown() throws IOException {
|
||||
IOUtils.cleanup(null, fs);
|
||||
IOUtils.cleanupWithLogger(null, fs);
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
cluster = null;
|
||||
@ -213,7 +213,7 @@ private void doNNWithQJMTest() throws IOException {
|
||||
* @throws IOException if there is an I/O error
|
||||
*/
|
||||
private void restartNameNode() throws IOException {
|
||||
IOUtils.cleanup(null, fs);
|
||||
IOUtils.cleanupWithLogger(null, fs);
|
||||
cluster.restartNameNode();
|
||||
fs = cluster.getFileSystem();
|
||||
}
|
||||
|
@ -479,7 +479,7 @@ public void testDeletedBlockWhenAddBlockIsInEdit() throws Exception {
|
||||
assertEquals("The block should be only on 1 datanode ", 1,
|
||||
locatedBlocks1.get(0).getLocations().length);
|
||||
} finally {
|
||||
IOUtils.cleanup(null, client);
|
||||
IOUtils.cleanupWithLogger(null, client);
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
|
@ -332,7 +332,7 @@ public void testFadviseSkippedForSmallReads() throws Exception {
|
||||
stats = tracker.getStats(fadvisedFileName);
|
||||
stats.assertNotDroppedInRange(0, TEST_PATH_LEN - WRITE_PACKET_SIZE);
|
||||
} finally {
|
||||
IOUtils.cleanup(null, fis);
|
||||
IOUtils.cleanupWithLogger(null, fis);
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
|
@ -230,7 +230,7 @@ private void runBasicTest(boolean shutdown) throws Exception {
|
||||
}
|
||||
|
||||
void shutdownCluster() throws Exception {
|
||||
IOUtils.cleanup(null, dfs);
|
||||
IOUtils.cleanupWithLogger(null, dfs);
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
|
@ -108,7 +108,8 @@ public void setUp() throws Exception {
|
||||
|
||||
@After
|
||||
public void destroyFileSystems() {
|
||||
IOUtils.cleanup(null, fs, fsAsBruce, fsAsDiana, fsAsSupergroupMember);
|
||||
IOUtils.cleanupWithLogger(null, fs, fsAsBruce, fsAsDiana,
|
||||
fsAsSupergroupMember);
|
||||
fs = fsAsBruce = fsAsDiana = fsAsSupergroupMember = fsAsBob = null;
|
||||
}
|
||||
|
||||
|
@ -127,7 +127,7 @@ public void setUp() throws Exception {
|
||||
|
||||
@After
|
||||
public void destroyFileSystems() {
|
||||
IOUtils.cleanup(null, fs);
|
||||
IOUtils.cleanupWithLogger(null, fs);
|
||||
fs = null;
|
||||
}
|
||||
|
||||
|
@ -55,7 +55,7 @@ public class TestAclConfigFlag {
|
||||
|
||||
@After
|
||||
public void shutdown() throws Exception {
|
||||
IOUtils.cleanup(null, fs);
|
||||
IOUtils.cleanupWithLogger(null, fs);
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
cluster = null;
|
||||
|
@ -206,7 +206,7 @@ public void testAddStripedBlock() throws Exception {
|
||||
assertTrue(blocks[0].isStriped());
|
||||
checkStripedBlockUC((BlockInfoStriped) fileNode.getLastBlock(), false);
|
||||
} finally {
|
||||
IOUtils.cleanup(null, out);
|
||||
IOUtils.cleanupWithLogger(null, out);
|
||||
}
|
||||
}
|
||||
|
||||
@ -271,7 +271,7 @@ public void testGetLocatedStripedBlocks() throws Exception {
|
||||
assertArrayEquals(indices, blockIndices);
|
||||
assertArrayEquals(expectedDNs, datanodes);
|
||||
} finally {
|
||||
IOUtils.cleanup(null, out);
|
||||
IOUtils.cleanupWithLogger(null, out);
|
||||
}
|
||||
}
|
||||
|
||||
@ -327,7 +327,7 @@ public void testAddUCReplica() throws Exception {
|
||||
assertTrue(storageIDs.contains(newstorage.getStorageID()));
|
||||
}
|
||||
} finally {
|
||||
IOUtils.cleanup(null, out);
|
||||
IOUtils.cleanupWithLogger(null, out);
|
||||
}
|
||||
|
||||
// 3. restart the namenode. mimic the full block reports and check the
|
||||
|
@ -95,7 +95,7 @@ public void testCommitWithInvalidGenStamp() throws Exception {
|
||||
dfs.getClient().getClientName(), previous, fileNode.getId());
|
||||
Assert.assertTrue("should complete successfully", complete);
|
||||
} finally {
|
||||
IOUtils.cleanup(null, out);
|
||||
IOUtils.cleanupWithLogger(null, out);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -1548,7 +1548,8 @@ public boolean accept(File dir, String name) {
|
||||
LOG.error("edit log failover didn't work", e);
|
||||
fail("Edit log failover didn't work");
|
||||
} finally {
|
||||
IOUtils.cleanup(null, streams.toArray(new EditLogInputStream[0]));
|
||||
IOUtils.cleanupWithLogger(null,
|
||||
streams.toArray(new EditLogInputStream[0]));
|
||||
}
|
||||
}
|
||||
|
||||
@ -1598,7 +1599,8 @@ public boolean accept(File dir, String name) {
|
||||
LOG.error("edit log failover didn't work", e);
|
||||
fail("Edit log failover didn't work");
|
||||
} finally {
|
||||
IOUtils.cleanup(null, streams.toArray(new EditLogInputStream[0]));
|
||||
IOUtils.cleanupWithLogger(null,
|
||||
streams.toArray(new EditLogInputStream[0]));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -154,7 +154,7 @@ public void testEditLogFileOutputStreamAbortAbort() throws IOException {
|
||||
editLogStream.abort();
|
||||
editLogStream.abort();
|
||||
} finally {
|
||||
IOUtils.cleanup(null, editLogStream);
|
||||
IOUtils.cleanupWithLogger(null, editLogStream);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -568,7 +568,7 @@ public void checkSalvagedRemains() throws IOException {
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
IOUtils.cleanup(null, in);
|
||||
IOUtils.cleanupWithLogger(null, in);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -212,7 +212,7 @@ public void testMetaSaveOverwrite() throws Exception {
|
||||
line = rdr.readLine();
|
||||
}
|
||||
} finally {
|
||||
IOUtils.cleanup(null, rdr, isr, fis);
|
||||
IOUtils.cleanupWithLogger(null, rdr, isr, fis);
|
||||
}
|
||||
}
|
||||
|
||||
@ -271,7 +271,7 @@ public void testConcurrentMetaSave() throws Exception {
|
||||
line = rdr.readLine();
|
||||
}
|
||||
} finally {
|
||||
IOUtils.cleanup(null, rdr, isr, fis);
|
||||
IOUtils.cleanupWithLogger(null, rdr, isr, fis);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -140,7 +140,7 @@ public void testUpdatingQuotaCount() throws Exception {
|
||||
Assert.assertEquals(cellSize * groupSize,
|
||||
actualDiskUsed);
|
||||
} finally {
|
||||
IOUtils.cleanup(null, out);
|
||||
IOUtils.cleanupWithLogger(null, out);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -48,7 +48,7 @@ public class TestXAttrConfigFlag {
|
||||
|
||||
@After
|
||||
public void shutdown() throws Exception {
|
||||
IOUtils.cleanup(null, fs);
|
||||
IOUtils.cleanupWithLogger(null, fs);
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
cluster = null;
|
||||
|
@ -1315,7 +1315,7 @@ public void run() {
|
||||
} catch (Exception e) {
|
||||
LOG.info("Got Exception while calling " + op.name, e);
|
||||
} finally {
|
||||
IOUtils.cleanup(null, op.client);
|
||||
IOUtils.cleanupWithLogger(null, op.client);
|
||||
}
|
||||
}
|
||||
}.start();
|
||||
|
@ -86,7 +86,7 @@ public static void init() throws Exception {
|
||||
|
||||
@AfterClass
|
||||
public static void shutdown() throws Exception {
|
||||
IOUtils.cleanup(null, hdfs, fsAsBruce, fsAsDiana);
|
||||
IOUtils.cleanupWithLogger(null, hdfs, fsAsBruce, fsAsDiana);
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
|
@ -79,7 +79,7 @@ public static void init() throws Exception {
|
||||
|
||||
@AfterClass
|
||||
public static void shutdown() throws Exception {
|
||||
IOUtils.cleanup(null, hdfs);
|
||||
IOUtils.cleanupWithLogger(null, hdfs);
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
|
@ -142,7 +142,7 @@ public void testFailToRename() throws IOException {
|
||||
fos = null;
|
||||
}
|
||||
} finally {
|
||||
IOUtils.cleanup(null, fos);
|
||||
IOUtils.cleanupWithLogger(null, fos);
|
||||
FileUtil.setWritable(TEST_DIR, true);
|
||||
}
|
||||
}
|
||||
|
@ -95,7 +95,7 @@ public void testHA() throws IOException {
|
||||
final Path dir2 = new Path("/test2");
|
||||
Assert.assertTrue(fs.mkdirs(dir2));
|
||||
} finally {
|
||||
IOUtils.cleanup(null, fs);
|
||||
IOUtils.cleanupWithLogger(null, fs);
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
@ -130,7 +130,7 @@ public void testSecureHAToken() throws IOException, InterruptedException {
|
||||
verify(fs).renewDelegationToken(token);
|
||||
verify(fs).cancelDelegationToken(token);
|
||||
} finally {
|
||||
IOUtils.cleanup(null, fs);
|
||||
IOUtils.cleanupWithLogger(null, fs);
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
@ -199,7 +199,7 @@ public void testClientFailoverWhenStandbyNNHasStaleCredentials()
|
||||
Exception unwrapped = re.unwrapRemoteException(StandbyException.class);
|
||||
Assert.assertTrue(unwrapped instanceof StandbyException);
|
||||
} finally {
|
||||
IOUtils.cleanup(null, fs);
|
||||
IOUtils.cleanupWithLogger(null, fs);
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
@ -238,7 +238,7 @@ public void testFailoverAfterOpen() throws IOException {
|
||||
IOUtils.readFully(in, buf, 0, buf.length);
|
||||
Assert.assertArrayEquals(data, buf);
|
||||
} finally {
|
||||
IOUtils.cleanup(null, fs);
|
||||
IOUtils.cleanupWithLogger(null, fs);
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
@ -264,7 +264,7 @@ public void testMultipleNamespacesConfigured() throws Exception {
|
||||
fs = (WebHdfsFileSystem)FileSystem.get(WEBHDFS_URI, conf);
|
||||
Assert.assertEquals(2, fs.getResolvedNNAddr().length);
|
||||
} finally {
|
||||
IOUtils.cleanup(null, fs);
|
||||
IOUtils.cleanupWithLogger(null, fs);
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
@ -304,7 +304,7 @@ public void run() {
|
||||
} catch (IOException e) {
|
||||
result = false;
|
||||
} finally {
|
||||
IOUtils.cleanup(null, fs);
|
||||
IOUtils.cleanupWithLogger(null, fs);
|
||||
}
|
||||
synchronized (TestWebHDFSForHA.this) {
|
||||
resultMap.put("mkdirs", result);
|
||||
|
@ -83,7 +83,7 @@ public void setUp() throws Exception {
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
IOUtils.cleanup(null, dfs);
|
||||
IOUtils.cleanupWithLogger(null, dfs);
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
|
@ -153,7 +153,7 @@ public void setUp() throws Exception {
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
IOUtils.cleanup(null, dfs);
|
||||
IOUtils.cleanupWithLogger(null, dfs);
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
|
@ -96,7 +96,7 @@ public static void init() throws Exception {
|
||||
|
||||
@AfterClass
|
||||
public static void shutdown() {
|
||||
IOUtils.cleanup(null, fs);
|
||||
IOUtils.cleanupWithLogger(null, fs);
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
|
@ -81,7 +81,7 @@ public static void init() throws Exception {
|
||||
|
||||
@AfterClass
|
||||
public static void shutdown() {
|
||||
IOUtils.cleanup(null, fs);
|
||||
IOUtils.cleanupWithLogger(null, fs);
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
|
@ -114,7 +114,7 @@ public static void init() throws Exception {
|
||||
|
||||
@AfterClass
|
||||
public static void shutdown() {
|
||||
IOUtils.cleanup(null, fs);
|
||||
IOUtils.cleanupWithLogger(null, fs);
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
|
@ -72,7 +72,7 @@ private static void mkdirs(String path) throws Exception {
|
||||
recordInExpectedValues(path);
|
||||
}
|
||||
finally {
|
||||
IOUtils.cleanup(null, fileSystem);
|
||||
IOUtils.cleanupWithLogger(null, fileSystem);
|
||||
}
|
||||
}
|
||||
|
||||
@ -85,7 +85,7 @@ private static void touchFile(String path) throws Exception {
|
||||
recordInExpectedValues(path);
|
||||
}
|
||||
finally {
|
||||
IOUtils.cleanup(null, fileSystem, outputStream);
|
||||
IOUtils.cleanupWithLogger(null, fileSystem, outputStream);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -261,7 +261,7 @@ private static void touchFile(String path, boolean createMultipleBlocks,
|
||||
System.out.println(fileStatus.getReplication());
|
||||
}
|
||||
finally {
|
||||
IOUtils.cleanup(null, outputStream);
|
||||
IOUtils.cleanupWithLogger(null, outputStream);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -91,7 +91,7 @@ private static int createFile(String path, int fileSize) throws Exception {
|
||||
return size;
|
||||
}
|
||||
finally {
|
||||
IOUtils.cleanup(null, fileSystem, outputStream);
|
||||
IOUtils.cleanupWithLogger(null, fileSystem, outputStream);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -100,7 +100,7 @@ private static void createFile(String path) throws Exception {
|
||||
new Path(path))[0].getPath().toString());
|
||||
}
|
||||
finally {
|
||||
IOUtils.cleanup(null, fileSystem, outputStream);
|
||||
IOUtils.cleanupWithLogger(null, fileSystem, outputStream);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -79,7 +79,7 @@ public float getProgress() throws IOException {
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
IOUtils.cleanup(null, src);
|
||||
IOUtils.cleanupWithLogger(null, src);
|
||||
factory.close();
|
||||
}
|
||||
}
|
||||
|
@ -112,7 +112,7 @@ public void run() {
|
||||
} catch (InterruptedException e) {
|
||||
// exit thread; ignore any jobs remaining in the trace
|
||||
} finally {
|
||||
IOUtils.cleanup(null, jobProducer);
|
||||
IOUtils.cleanupWithLogger(null, jobProducer);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -143,7 +143,7 @@ public void run() {
|
||||
} catch (InterruptedException e) {
|
||||
return;
|
||||
} finally {
|
||||
IOUtils.cleanup(null, jobProducer);
|
||||
IOUtils.cleanupWithLogger(null, jobProducer);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -247,7 +247,7 @@ public void run() {
|
||||
LOG.error("[STRESS] Interrupted in the main block!", e);
|
||||
return;
|
||||
} finally {
|
||||
IOUtils.cleanup(null, jobProducer);
|
||||
IOUtils.cleanupWithLogger(null, jobProducer);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -470,7 +470,7 @@ public int compare(Pair<LoggedJob, JobTraceReader> p1,
|
||||
next = heap.poll();
|
||||
}
|
||||
} finally {
|
||||
IOUtils.cleanup(null, reader);
|
||||
IOUtils.cleanupWithLogger(null, reader);
|
||||
if (outGen != null) {
|
||||
outGen.close();
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user