HADOOP-17622. Avoid usage of deprecated IOUtils#cleanup API. (#2862)

Signed-off-by: Takanobu Asanuma <tasanuma@apache.org>
This commit is contained in:
Viraj Jasani 2021-04-06 10:09:10 +05:30 committed by GitHub
parent 26b8f678b2
commit 3f2682b92b
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
64 changed files with 102 additions and 99 deletions

View File

@ -312,7 +312,7 @@ public class TestLocalFileSystem {
.new LocalFSFileInputStream(path), 1024); .new LocalFSFileInputStream(path), 1024);
assertNotNull(bis.getFileDescriptor()); assertNotNull(bis.getFileDescriptor());
} finally { } finally {
IOUtils.cleanup(null, bis); IOUtils.cleanupWithLogger(null, bis);
} }
} }

View File

@ -115,7 +115,7 @@ public class TestFileSink {
IOUtils.copyBytes(is, baos, 1024, true); IOUtils.copyBytes(is, baos, 1024, true);
outFileContent = new String(baos.toByteArray(), "UTF-8"); outFileContent = new String(baos.toByteArray(), "UTF-8");
} finally { } finally {
IOUtils.cleanup(null, baos, is); IOUtils.cleanupWithLogger(null, baos, is);
} }
// Check the out file content. Should be something like the following: // Check the out file content. Should be something like the following:

View File

@ -759,6 +759,6 @@ public class TestDomainSocket {
readerThread.join(); readerThread.join();
Assert.assertFalse(failed.get()); Assert.assertFalse(failed.get());
Assert.assertEquals(3, bytesRead.get()); Assert.assertEquals(3, bytesRead.get());
IOUtils.cleanup(null, socks); IOUtils.cleanupWithLogger(null, socks);
} }
} }

View File

@ -618,8 +618,8 @@ class BPServiceActor implements Runnable {
private synchronized void cleanUp() { private synchronized void cleanUp() {
shouldServiceRun = false; shouldServiceRun = false;
IOUtils.cleanup(null, bpNamenode); IOUtils.cleanupWithLogger(null, bpNamenode);
IOUtils.cleanup(null, lifelineSender); IOUtils.cleanupWithLogger(null, lifelineSender);
bpos.shutdownActor(this); bpos.shutdownActor(this);
} }
@ -992,7 +992,7 @@ class BPServiceActor implements Runnable {
} catch (InterruptedException e) { } catch (InterruptedException e) {
Thread.currentThread().interrupt(); Thread.currentThread().interrupt();
} }
IOUtils.cleanup(null, lifelineNamenode); IOUtils.cleanupWithLogger(null, lifelineNamenode);
} }
@Override @Override

View File

@ -369,7 +369,7 @@ class BlockReceiver implements Closeable {
streams.close(); streams.close();
} }
if (replicaHandler != null) { if (replicaHandler != null) {
IOUtils.cleanup(null, replicaHandler); IOUtils.cleanupWithLogger(null, replicaHandler);
replicaHandler = null; replicaHandler = null;
} }
if (measuredFlushTime) { if (measuredFlushTime) {

View File

@ -252,7 +252,7 @@ public class BlockScanner {
if (!success) { if (!success) {
// If we didn't create a new VolumeScanner object, we don't // If we didn't create a new VolumeScanner object, we don't
// need this reference to the volume. // need this reference to the volume.
IOUtils.cleanup(null, ref); IOUtils.cleanupWithLogger(null, ref);
} }
} }
} }

View File

@ -432,7 +432,7 @@ class DataXceiver extends Receiver implements Runnable {
blk.getBlockId(), dnR.getDatanodeUuid(), success)); blk.getBlockId(), dnR.getDatanodeUuid(), success));
} }
if (fis != null) { if (fis != null) {
IOUtils.cleanup(null, fis); IOUtils.cleanupWithLogger(null, fis);
} }
} }
} }
@ -555,7 +555,7 @@ class DataXceiver extends Receiver implements Runnable {
LOG.warn("Failed to shut down socket in error handler", e); LOG.warn("Failed to shut down socket in error handler", e);
} }
} }
IOUtils.cleanup(null, shmInfo); IOUtils.cleanupWithLogger(null, shmInfo);
} }
} }

View File

@ -451,7 +451,7 @@ public class VolumeScanner extends Thread {
} catch (IOException e) { } catch (IOException e) {
resultHandler.handle(block, e); resultHandler.handle(block, e);
} finally { } finally {
IOUtils.cleanup(null, blockSender); IOUtils.cleanupWithLogger(null, blockSender);
} }
metrics.incrBlockVerificationFailures(); metrics.incrBlockVerificationFailures();
return -1; return -1;
@ -674,13 +674,13 @@ public class VolumeScanner extends Thread {
// Save the current position of all block iterators and close them. // Save the current position of all block iterators and close them.
for (BlockIterator iter : blockIters) { for (BlockIterator iter : blockIters) {
saveBlockIterator(iter); saveBlockIterator(iter);
IOUtils.cleanup(null, iter); IOUtils.cleanupWithLogger(null, iter);
} }
} finally { } finally {
VolumeScannerCBInjector.get().terminationCallBack(this); VolumeScannerCBInjector.get().terminationCallBack(this);
// When the VolumeScanner exits, release the reference we were holding // When the VolumeScanner exits, release the reference we were holding
// on the volume. This will allow the volume to be removed later. // 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 class VolumeScanner extends Thread {
if (iter.getBlockPoolId().equals(bpid)) { if (iter.getBlockPoolId().equals(bpid)) {
LOG.trace("{}: disabling scanning on block pool {}", this, bpid); LOG.trace("{}: disabling scanning on block pool {}", this, bpid);
i.remove(); i.remove();
IOUtils.cleanup(null, iter); IOUtils.cleanupWithLogger(null, iter);
if (curBlockIter == iter) { if (curBlockIter == iter) {
curBlockIter = null; curBlockIter = null;
} }

View File

@ -234,7 +234,7 @@ public class DatasetVolumeChecker {
} }
}), MoreExecutors.directExecutor()); }), MoreExecutors.directExecutor());
} else { } else {
IOUtils.cleanup(null, reference); IOUtils.cleanupWithLogger(null, reference);
if (numVolumes.decrementAndGet() == 0) { if (numVolumes.decrementAndGet() == 0) {
latch.countDown(); latch.countDown();
} }
@ -311,7 +311,7 @@ public class DatasetVolumeChecker {
); );
return true; return true;
} else { } else {
IOUtils.cleanup(null, volumeReference); IOUtils.cleanupWithLogger(null, volumeReference);
} }
return false; return false;
} }
@ -404,7 +404,7 @@ public class DatasetVolumeChecker {
} }
private void cleanup() { private void cleanup() {
IOUtils.cleanup(null, reference); IOUtils.cleanupWithLogger(null, reference);
invokeCallback(); invokeCallback();
} }

View File

@ -158,7 +158,7 @@ class StripedBlockReader {
return peer; return peer;
} finally { } finally {
if (!success) { if (!success) {
IOUtils.cleanup(null, peer); IOUtils.cleanupWithLogger(null, peer);
IOUtils.closeSocket(sock); IOUtils.closeSocket(sock);
} }
} }

View File

@ -130,7 +130,7 @@ public class ReplicaInputStreams implements Closeable {
dataInFd = null; dataInFd = null;
} }
if (volumeRef != null) { if (volumeRef != null) {
IOUtils.cleanup(null, volumeRef); IOUtils.cleanupWithLogger(null, volumeRef);
volumeRef = null; volumeRef = null;
} }
// throw IOException if there is any // throw IOException if there is any
@ -146,7 +146,7 @@ public class ReplicaInputStreams implements Closeable {
dataInFd = null; dataInFd = null;
IOUtils.closeStream(checksumIn); IOUtils.closeStream(checksumIn);
checksumIn = null; checksumIn = null;
IOUtils.cleanup(null, volumeRef); IOUtils.cleanupWithLogger(null, volumeRef);
volumeRef = null; volumeRef = null;
} }
} }

View File

@ -335,7 +335,7 @@ class FsDatasetAsyncDiskService {
+ block.getLocalBlock() + " URI " + replicaToDelete.getBlockURI()); + block.getLocalBlock() + " URI " + replicaToDelete.getBlockURI());
} }
updateDeletedBlockId(block); updateDeletedBlockId(block);
IOUtils.cleanup(null, volumeRef); IOUtils.cleanupWithLogger(null, volumeRef);
} }
} }

View File

@ -949,11 +949,11 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
return new ReplicaInputStreams( return new ReplicaInputStreams(
blockInStream, metaInStream, ref, datanode.getFileIoProvider()); blockInStream, metaInStream, ref, datanode.getFileIoProvider());
} catch (IOException e) { } catch (IOException e) {
IOUtils.cleanup(null, blockInStream); IOUtils.cleanupWithLogger(null, blockInStream);
throw e; throw e;
} }
} catch (IOException e) { } catch (IOException e) {
IOUtils.cleanup(null, ref); IOUtils.cleanupWithLogger(null, ref);
throw e; throw e;
} }
} }
@ -1421,7 +1421,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
replica = append(b.getBlockPoolId(), replicaInfo, newGS, replica = append(b.getBlockPoolId(), replicaInfo, newGS,
b.getNumBytes()); b.getNumBytes());
} catch (IOException e) { } catch (IOException e) {
IOUtils.cleanup(null, ref); IOUtils.cleanupWithLogger(null, ref);
throw e; throw e;
} }
return new ReplicaHandler(replica, ref); return new ReplicaHandler(replica, ref);
@ -1553,7 +1553,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
replica = (ReplicaInPipeline) replicaInfo; replica = (ReplicaInPipeline) replicaInfo;
} }
} catch (IOException e) { } catch (IOException e) {
IOUtils.cleanup(null, ref); IOUtils.cleanupWithLogger(null, ref);
throw e; throw e;
} }
return new ReplicaHandler(replica, ref); return new ReplicaHandler(replica, ref);
@ -1648,7 +1648,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
+ " for block " + b.getBlockId()); + " for block " + b.getBlockId());
} }
} catch (IOException e) { } catch (IOException e) {
IOUtils.cleanup(null, ref); IOUtils.cleanupWithLogger(null, ref);
throw e; throw e;
} }
@ -1752,7 +1752,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
// bump the replica's generation stamp to newGS // bump the replica's generation stamp to newGS
rbw.getReplicaInfo().bumpReplicaGS(newGS); rbw.getReplicaInfo().bumpReplicaGS(newGS);
} catch (IOException e) { } catch (IOException e) {
IOUtils.cleanup(null, ref); IOUtils.cleanupWithLogger(null, ref);
throw e; throw e;
} }
return new ReplicaHandler(rbw, ref); return new ReplicaHandler(rbw, ref);
@ -1905,7 +1905,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
try { try {
newReplicaInfo = v.createTemporary(b); newReplicaInfo = v.createTemporary(b);
} catch (IOException e) { } catch (IOException e) {
IOUtils.cleanup(null, ref); IOUtils.cleanupWithLogger(null, ref);
throw e; throw e;
} }

View File

@ -117,7 +117,7 @@ public class FsDatasetUtil {
} }
return raf.getFD(); return raf.getFD();
} catch(IOException ioe) { } catch(IOException ioe) {
IOUtils.cleanup(null, raf); IOUtils.cleanupWithLogger(null, raf);
throw ioe; throw ioe;
} }
} }

View File

@ -357,7 +357,7 @@ class FsVolumeList {
} else { } else {
// If the volume is not put into a volume scanner, it does not need to // If the volume is not put into a volume scanner, it does not need to
// hold the reference. // 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 // If the volume is used to replace a failed volume, it needs to reset the
// volume failure info for this volume. // volume failure info for this volume.

View File

@ -76,7 +76,7 @@ public class EditsDoubleBuffer {
+ " bytes still to be flushed and cannot be closed."); + " bytes still to be flushed and cannot be closed.");
} }
IOUtils.cleanup(null, bufCurrent, bufReady); IOUtils.cleanupWithLogger(null, bufCurrent, bufReady);
bufCurrent = bufReady = null; bufCurrent = bufReady = null;
} }

View File

@ -202,7 +202,7 @@ public class DebugAdmin extends Configured implements Tool {
blockFile); blockFile);
return 0; return 0;
} finally { } finally {
IOUtils.cleanup(null, metaStream, dataStream, checksumStream); IOUtils.cleanupWithLogger(null, metaStream, dataStream, checksumStream);
} }
} }
} }
@ -287,7 +287,7 @@ public class DebugAdmin extends Configured implements Tool {
+ " saved metadata to meta file " + outFile); + " saved metadata to meta file " + outFile);
return 0; return 0;
} finally { } finally {
IOUtils.cleanup(null, metaOut); IOUtils.cleanupWithLogger(null, metaOut);
} }
} }
} }

View File

@ -319,10 +319,10 @@ abstract class PBImageTextWriter implements Closeable {
@Override @Override
public void close() throws IOException { public void close() throws IOException {
if (batch != null) { if (batch != null) {
IOUtils.cleanup(null, batch); IOUtils.cleanupWithLogger(null, batch);
batch = null; batch = null;
} }
IOUtils.cleanup(null, db); IOUtils.cleanupWithLogger(null, db);
db = null; db = null;
} }
@ -388,13 +388,13 @@ abstract class PBImageTextWriter implements Closeable {
dirMap = new LevelDBStore(new File(dbDir, "dirMap")); dirMap = new LevelDBStore(new File(dbDir, "dirMap"));
} catch (IOException e) { } catch (IOException e) {
LOG.error("Failed to open LevelDBs", e); LOG.error("Failed to open LevelDBs", e);
IOUtils.cleanup(null, this); IOUtils.cleanupWithLogger(null, this);
} }
} }
@Override @Override
public void close() throws IOException { public void close() throws IOException {
IOUtils.cleanup(null, dirChildMap, dirMap); IOUtils.cleanupWithLogger(null, dirChildMap, dirMap);
dirChildMap = null; dirChildMap = null;
dirMap = null; dirMap = null;
} }
@ -515,7 +515,7 @@ abstract class PBImageTextWriter implements Closeable {
@Override @Override
public void close() throws IOException { public void close() throws IOException {
out.flush(); out.flush();
IOUtils.cleanup(null, metadataMap); IOUtils.cleanupWithLogger(null, metadataMap);
} }
void append(StringBuffer buffer, int field) { void append(StringBuffer buffer, int field) {

View File

@ -841,7 +841,7 @@ public class TestEnhancedByteBufferAccess {
if (buf2 != null) { if (buf2 != null) {
fsIn2.releaseBuffer(buf2); fsIn2.releaseBuffer(buf2);
} }
IOUtils.cleanup(null, fsIn, fsIn2); IOUtils.cleanupWithLogger(null, fsIn, fsIn2);
if (cluster != null) { if (cluster != null) {
cluster.shutdown(); cluster.shutdown();
} }

View File

@ -87,7 +87,7 @@ public class TestUnbuffer {
Assert.assertEquals(b, b2); Assert.assertEquals(b, b2);
} finally { } finally {
if (stream != null) { if (stream != null) {
IOUtils.cleanup(null, stream); IOUtils.cleanupWithLogger(null, stream);
} }
if (cluster != null) { if (cluster != null) {
cluster.shutdown(); cluster.shutdown();
@ -122,7 +122,7 @@ public class TestUnbuffer {
} }
} finally { } finally {
for (FSDataInputStream stream : streams) { for (FSDataInputStream stream : streams) {
IOUtils.cleanup(null, stream); IOUtils.cleanupWithLogger(null, stream);
} }
if (cluster != null) { if (cluster != null) {
cluster.shutdown(); cluster.shutdown();

View File

@ -96,7 +96,7 @@ public class TestStickyBit {
@AfterClass @AfterClass
public static void shutdown() throws Exception { public static void shutdown() throws Exception {
IOUtils.cleanup(null, hdfs, hdfsAsUser1, hdfsAsUser2); IOUtils.cleanupWithLogger(null, hdfs, hdfsAsUser1, hdfsAsUser2);
if (cluster != null) { if (cluster != null) {
cluster.shutdown(); cluster.shutdown();
} }
@ -121,7 +121,7 @@ public class TestStickyBit {
h.close(); h.close();
h = null; h = null;
} finally { } finally {
IOUtils.cleanup(null, h); IOUtils.cleanupWithLogger(null, h);
} }
} }
@ -500,7 +500,7 @@ public class TestStickyBit {
o.close(); o.close();
o = null; o = null;
} finally { } finally {
IOUtils.cleanup(null, o); IOUtils.cleanupWithLogger(null, o);
} }
} }

View File

@ -227,7 +227,7 @@ public class TestDataTransferKeepalive {
IOUtils.copyBytes(stm, new IOUtils.NullOutputStream(), 1024); IOUtils.copyBytes(stm, new IOUtils.NullOutputStream(), 1024);
} }
} finally { } finally {
IOUtils.cleanup(null, stms); IOUtils.cleanupWithLogger(null, stms);
} }
assertEquals(5, peerCache.size()); assertEquals(5, peerCache.size());

View File

@ -177,7 +177,7 @@ public class TestHFlush {
blocks = fileSystem.dfs.getLocatedBlocks(path.toString(), 0); blocks = fileSystem.dfs.getLocatedBlocks(path.toString(), 0);
assertEquals(3, blocks.getLocatedBlocks().size()); assertEquals(3, blocks.getLocatedBlocks().size());
} finally { } finally {
IOUtils.cleanup(null, stm, fileSystem); IOUtils.cleanupWithLogger(null, stm, fileSystem);
if (cluster != null) { if (cluster != null) {
cluster.shutdown(); cluster.shutdown();
} }

View File

@ -363,8 +363,8 @@ public class TestPread {
assertTrue(false); assertTrue(false);
} finally { } finally {
Mockito.reset(injector); Mockito.reset(injector);
IOUtils.cleanup(null, input); IOUtils.cleanupWithLogger(null, input);
IOUtils.cleanup(null, output); IOUtils.cleanupWithLogger(null, output);
fileSys.close(); fileSys.close();
cluster.shutdown(); cluster.shutdown();
} }

View File

@ -741,7 +741,7 @@ public class TestRollingUpgrade {
// do checkpoint in SNN again // do checkpoint in SNN again
snn.doCheckpoint(); snn.doCheckpoint();
} finally { } finally {
IOUtils.cleanup(null, dfs); IOUtils.cleanupWithLogger(null, dfs);
if (snn != null) { if (snn != null) {
snn.shutdown(); snn.shutdown();
} }

View File

@ -273,7 +273,7 @@ public class TestSecureEncryptionZoneWithKMS {
@After @After
public void shutdown() throws IOException { public void shutdown() throws IOException {
IOUtils.cleanup(null, fs); IOUtils.cleanupWithLogger(null, fs);
if (cluster != null) { if (cluster != null) {
cluster.shutdown(); cluster.shutdown();
cluster = null; cluster = null;

View File

@ -229,7 +229,7 @@ public class TestTrashWithSecureEncryptionZones {
@AfterClass @AfterClass
public static void destroy() { public static void destroy() {
IOUtils.cleanup(null, fs); IOUtils.cleanupWithLogger(null, fs);
if (cluster != null) { if (cluster != null) {
cluster.shutdown(); cluster.shutdown();
cluster = null; cluster = null;

View File

@ -54,7 +54,7 @@ public class TestWriteConfigurationToDFS {
fs.close(); fs.close();
fs = null; fs = null;
} finally { } finally {
IOUtils.cleanup(null, os, fs); IOUtils.cleanupWithLogger(null, os, fs);
cluster.shutdown(); cluster.shutdown();
} }
} }

View File

@ -256,7 +256,7 @@ public class TestSaslDataTransfer extends SaslDataTransferTestCase {
} catch (SocketTimeoutException e) { } catch (SocketTimeoutException e) {
GenericTestUtils.assertExceptionContains("Read timed out", e); GenericTestUtils.assertExceptionContains("Read timed out", e);
} finally { } finally {
IOUtils.cleanup(null, socket, serverSocket); IOUtils.cleanupWithLogger(null, socket, serverSocket);
} }
} }

View File

@ -173,7 +173,7 @@ public abstract class QJMTestUtil {
lastRecoveredTxn = elis.getLastTxId(); lastRecoveredTxn = elis.getLastTxId();
} }
} finally { } finally {
IOUtils.cleanup(null, streams.toArray(new Closeable[0])); IOUtils.cleanupWithLogger(null, streams.toArray(new Closeable[0]));
} }
return lastRecoveredTxn; return lastRecoveredTxn;
} }

View File

@ -163,7 +163,7 @@ public class TestSecureNNWithQJM {
@After @After
public void shutdown() throws IOException { public void shutdown() throws IOException {
IOUtils.cleanup(null, fs); IOUtils.cleanupWithLogger(null, fs);
if (cluster != null) { if (cluster != null) {
cluster.shutdown(); cluster.shutdown();
cluster = null; cluster = null;
@ -213,7 +213,7 @@ public class TestSecureNNWithQJM {
* @throws IOException if there is an I/O error * @throws IOException if there is an I/O error
*/ */
private void restartNameNode() throws IOException { private void restartNameNode() throws IOException {
IOUtils.cleanup(null, fs); IOUtils.cleanupWithLogger(null, fs);
cluster.restartNameNode(); cluster.restartNameNode();
fs = cluster.getFileSystem(); fs = cluster.getFileSystem();
} }

View File

@ -479,7 +479,7 @@ public class TestBlockReplacement {
assertEquals("The block should be only on 1 datanode ", 1, assertEquals("The block should be only on 1 datanode ", 1,
locatedBlocks1.get(0).getLocations().length); locatedBlocks1.get(0).getLocations().length);
} finally { } finally {
IOUtils.cleanup(null, client); IOUtils.cleanupWithLogger(null, client);
cluster.shutdown(); cluster.shutdown();
} }
} }

View File

@ -332,7 +332,7 @@ public class TestCachingStrategy {
stats = tracker.getStats(fadvisedFileName); stats = tracker.getStats(fadvisedFileName);
stats.assertNotDroppedInRange(0, TEST_PATH_LEN - WRITE_PACKET_SIZE); stats.assertNotDroppedInRange(0, TEST_PATH_LEN - WRITE_PACKET_SIZE);
} finally { } finally {
IOUtils.cleanup(null, fis); IOUtils.cleanupWithLogger(null, fis);
if (cluster != null) { if (cluster != null) {
cluster.shutdown(); cluster.shutdown();
} }

View File

@ -230,7 +230,7 @@ public class TestStorageMover {
} }
void shutdownCluster() throws Exception { void shutdownCluster() throws Exception {
IOUtils.cleanup(null, dfs); IOUtils.cleanupWithLogger(null, dfs);
if (cluster != null) { if (cluster != null) {
cluster.shutdown(); cluster.shutdown();
} }

View File

@ -108,7 +108,8 @@ public abstract class FSAclBaseTest {
@After @After
public void destroyFileSystems() { public void destroyFileSystems() {
IOUtils.cleanup(null, fs, fsAsBruce, fsAsDiana, fsAsSupergroupMember); IOUtils.cleanupWithLogger(null, fs, fsAsBruce, fsAsDiana,
fsAsSupergroupMember);
fs = fsAsBruce = fsAsDiana = fsAsSupergroupMember = fsAsBob = null; fs = fsAsBruce = fsAsDiana = fsAsSupergroupMember = fsAsBob = null;
} }

View File

@ -127,7 +127,7 @@ public class FSXAttrBaseTest {
@After @After
public void destroyFileSystems() { public void destroyFileSystems() {
IOUtils.cleanup(null, fs); IOUtils.cleanupWithLogger(null, fs);
fs = null; fs = null;
} }

View File

@ -55,7 +55,7 @@ public class TestAclConfigFlag {
@After @After
public void shutdown() throws Exception { public void shutdown() throws Exception {
IOUtils.cleanup(null, fs); IOUtils.cleanupWithLogger(null, fs);
if (cluster != null) { if (cluster != null) {
cluster.shutdown(); cluster.shutdown();
cluster = null; cluster = null;

View File

@ -206,7 +206,7 @@ public class TestAddStripedBlocks {
assertTrue(blocks[0].isStriped()); assertTrue(blocks[0].isStriped());
checkStripedBlockUC((BlockInfoStriped) fileNode.getLastBlock(), false); checkStripedBlockUC((BlockInfoStriped) fileNode.getLastBlock(), false);
} finally { } finally {
IOUtils.cleanup(null, out); IOUtils.cleanupWithLogger(null, out);
} }
} }
@ -271,7 +271,7 @@ public class TestAddStripedBlocks {
assertArrayEquals(indices, blockIndices); assertArrayEquals(indices, blockIndices);
assertArrayEquals(expectedDNs, datanodes); assertArrayEquals(expectedDNs, datanodes);
} finally { } finally {
IOUtils.cleanup(null, out); IOUtils.cleanupWithLogger(null, out);
} }
} }
@ -327,7 +327,7 @@ public class TestAddStripedBlocks {
assertTrue(storageIDs.contains(newstorage.getStorageID())); assertTrue(storageIDs.contains(newstorage.getStorageID()));
} }
} finally { } finally {
IOUtils.cleanup(null, out); IOUtils.cleanupWithLogger(null, out);
} }
// 3. restart the namenode. mimic the full block reports and check the // 3. restart the namenode. mimic the full block reports and check the

View File

@ -95,7 +95,7 @@ public class TestCommitBlockWithInvalidGenStamp {
dfs.getClient().getClientName(), previous, fileNode.getId()); dfs.getClient().getClientName(), previous, fileNode.getId());
Assert.assertTrue("should complete successfully", complete); Assert.assertTrue("should complete successfully", complete);
} finally { } finally {
IOUtils.cleanup(null, out); IOUtils.cleanupWithLogger(null, out);
} }
} }
} }

View File

@ -1548,7 +1548,8 @@ public class TestEditLog {
LOG.error("edit log failover didn't work", e); LOG.error("edit log failover didn't work", e);
fail("Edit log failover didn't work"); fail("Edit log failover didn't work");
} finally { } finally {
IOUtils.cleanup(null, streams.toArray(new EditLogInputStream[0])); IOUtils.cleanupWithLogger(null,
streams.toArray(new EditLogInputStream[0]));
} }
} }
@ -1598,7 +1599,8 @@ public class TestEditLog {
LOG.error("edit log failover didn't work", e); LOG.error("edit log failover didn't work", e);
fail("Edit log failover didn't work"); fail("Edit log failover didn't work");
} finally { } finally {
IOUtils.cleanup(null, streams.toArray(new EditLogInputStream[0])); IOUtils.cleanupWithLogger(null,
streams.toArray(new EditLogInputStream[0]));
} }
} }

View File

@ -154,7 +154,7 @@ public class TestEditLogFileOutputStream {
editLogStream.abort(); editLogStream.abort();
editLogStream.abort(); editLogStream.abort();
} finally { } finally {
IOUtils.cleanup(null, editLogStream); IOUtils.cleanupWithLogger(null, editLogStream);
} }
} }
} }

View File

@ -568,7 +568,7 @@ public class TestFsck {
} }
} }
} finally { } finally {
IOUtils.cleanup(null, in); IOUtils.cleanupWithLogger(null, in);
} }
} }
} }

View File

@ -212,7 +212,7 @@ public class TestMetaSave {
line = rdr.readLine(); line = rdr.readLine();
} }
} finally { } finally {
IOUtils.cleanup(null, rdr, isr, fis); IOUtils.cleanupWithLogger(null, rdr, isr, fis);
} }
} }
@ -271,7 +271,7 @@ public class TestMetaSave {
line = rdr.readLine(); line = rdr.readLine();
} }
} finally { } finally {
IOUtils.cleanup(null, rdr, isr, fis); IOUtils.cleanupWithLogger(null, rdr, isr, fis);
} }
} }

View File

@ -140,7 +140,7 @@ public class TestQuotaWithStripedBlocks {
Assert.assertEquals(cellSize * groupSize, Assert.assertEquals(cellSize * groupSize,
actualDiskUsed); actualDiskUsed);
} finally { } finally {
IOUtils.cleanup(null, out); IOUtils.cleanupWithLogger(null, out);
} }
} }
} }

View File

@ -48,7 +48,7 @@ public class TestXAttrConfigFlag {
@After @After
public void shutdown() throws Exception { public void shutdown() throws Exception {
IOUtils.cleanup(null, fs); IOUtils.cleanupWithLogger(null, fs);
if (cluster != null) { if (cluster != null) {
cluster.shutdown(); cluster.shutdown();
cluster = null; cluster = null;

View File

@ -1315,7 +1315,7 @@ public class TestRetryCacheWithHA {
} catch (Exception e) { } catch (Exception e) {
LOG.info("Got Exception while calling " + op.name, e); LOG.info("Got Exception while calling " + op.name, e);
} finally { } finally {
IOUtils.cleanup(null, op.client); IOUtils.cleanupWithLogger(null, op.client);
} }
} }
}.start(); }.start();

View File

@ -86,7 +86,7 @@ public class TestAclWithSnapshot {
@AfterClass @AfterClass
public static void shutdown() throws Exception { public static void shutdown() throws Exception {
IOUtils.cleanup(null, hdfs, fsAsBruce, fsAsDiana); IOUtils.cleanupWithLogger(null, hdfs, fsAsBruce, fsAsDiana);
if (cluster != null) { if (cluster != null) {
cluster.shutdown(); cluster.shutdown();
} }

View File

@ -79,7 +79,7 @@ public class TestXAttrWithSnapshot {
@AfterClass @AfterClass
public static void shutdown() throws Exception { public static void shutdown() throws Exception {
IOUtils.cleanup(null, hdfs); IOUtils.cleanupWithLogger(null, hdfs);
if (cluster != null) { if (cluster != null) {
cluster.shutdown(); cluster.shutdown();
} }

View File

@ -142,7 +142,7 @@ public class TestAtomicFileOutputStream {
fos = null; fos = null;
} }
} finally { } finally {
IOUtils.cleanup(null, fos); IOUtils.cleanupWithLogger(null, fos);
FileUtil.setWritable(TEST_DIR, true); FileUtil.setWritable(TEST_DIR, true);
} }
} }

View File

@ -95,7 +95,7 @@ public class TestWebHDFSForHA {
final Path dir2 = new Path("/test2"); final Path dir2 = new Path("/test2");
Assert.assertTrue(fs.mkdirs(dir2)); Assert.assertTrue(fs.mkdirs(dir2));
} finally { } finally {
IOUtils.cleanup(null, fs); IOUtils.cleanupWithLogger(null, fs);
if (cluster != null) { if (cluster != null) {
cluster.shutdown(); cluster.shutdown();
} }
@ -130,7 +130,7 @@ public class TestWebHDFSForHA {
verify(fs).renewDelegationToken(token); verify(fs).renewDelegationToken(token);
verify(fs).cancelDelegationToken(token); verify(fs).cancelDelegationToken(token);
} finally { } finally {
IOUtils.cleanup(null, fs); IOUtils.cleanupWithLogger(null, fs);
if (cluster != null) { if (cluster != null) {
cluster.shutdown(); cluster.shutdown();
} }
@ -199,7 +199,7 @@ public class TestWebHDFSForHA {
Exception unwrapped = re.unwrapRemoteException(StandbyException.class); Exception unwrapped = re.unwrapRemoteException(StandbyException.class);
Assert.assertTrue(unwrapped instanceof StandbyException); Assert.assertTrue(unwrapped instanceof StandbyException);
} finally { } finally {
IOUtils.cleanup(null, fs); IOUtils.cleanupWithLogger(null, fs);
if (cluster != null) { if (cluster != null) {
cluster.shutdown(); cluster.shutdown();
} }
@ -238,7 +238,7 @@ public class TestWebHDFSForHA {
IOUtils.readFully(in, buf, 0, buf.length); IOUtils.readFully(in, buf, 0, buf.length);
Assert.assertArrayEquals(data, buf); Assert.assertArrayEquals(data, buf);
} finally { } finally {
IOUtils.cleanup(null, fs); IOUtils.cleanupWithLogger(null, fs);
if (cluster != null) { if (cluster != null) {
cluster.shutdown(); cluster.shutdown();
} }
@ -264,7 +264,7 @@ public class TestWebHDFSForHA {
fs = (WebHdfsFileSystem)FileSystem.get(WEBHDFS_URI, conf); fs = (WebHdfsFileSystem)FileSystem.get(WEBHDFS_URI, conf);
Assert.assertEquals(2, fs.getResolvedNNAddr().length); Assert.assertEquals(2, fs.getResolvedNNAddr().length);
} finally { } finally {
IOUtils.cleanup(null, fs); IOUtils.cleanupWithLogger(null, fs);
if (cluster != null) { if (cluster != null) {
cluster.shutdown(); cluster.shutdown();
} }
@ -304,7 +304,7 @@ public class TestWebHDFSForHA {
} catch (IOException e) { } catch (IOException e) {
result = false; result = false;
} finally { } finally {
IOUtils.cleanup(null, fs); IOUtils.cleanupWithLogger(null, fs);
} }
synchronized (TestWebHDFSForHA.this) { synchronized (TestWebHDFSForHA.this) {
resultMap.put("mkdirs", result); resultMap.put("mkdirs", result);

View File

@ -83,7 +83,7 @@ public class TestDistCpSync {
@After @After
public void tearDown() throws Exception { public void tearDown() throws Exception {
IOUtils.cleanup(null, dfs); IOUtils.cleanupWithLogger(null, dfs);
if (cluster != null) { if (cluster != null) {
cluster.shutdown(); cluster.shutdown();
} }

View File

@ -153,7 +153,7 @@ public abstract class TestDistCpSyncReverseBase {
@After @After
public void tearDown() throws Exception { public void tearDown() throws Exception {
IOUtils.cleanup(null, dfs); IOUtils.cleanupWithLogger(null, dfs);
if (cluster != null) { if (cluster != null) {
cluster.shutdown(); cluster.shutdown();
} }

View File

@ -96,7 +96,7 @@ public class TestDistCpWithAcls {
@AfterClass @AfterClass
public static void shutdown() { public static void shutdown() {
IOUtils.cleanup(null, fs); IOUtils.cleanupWithLogger(null, fs);
if (cluster != null) { if (cluster != null) {
cluster.shutdown(); cluster.shutdown();
} }

View File

@ -81,7 +81,7 @@ public class TestDistCpWithRawXAttrs {
@AfterClass @AfterClass
public static void shutdown() { public static void shutdown() {
IOUtils.cleanup(null, fs); IOUtils.cleanupWithLogger(null, fs);
if (cluster != null) { if (cluster != null) {
cluster.shutdown(); cluster.shutdown();
} }

View File

@ -114,7 +114,7 @@ public class TestDistCpWithXAttrs {
@AfterClass @AfterClass
public static void shutdown() { public static void shutdown() {
IOUtils.cleanup(null, fs); IOUtils.cleanupWithLogger(null, fs);
if (cluster != null) { if (cluster != null) {
cluster.shutdown(); cluster.shutdown();
} }

View File

@ -72,7 +72,7 @@ public class TestGlobbedCopyListing {
recordInExpectedValues(path); recordInExpectedValues(path);
} }
finally { finally {
IOUtils.cleanup(null, fileSystem); IOUtils.cleanupWithLogger(null, fileSystem);
} }
} }
@ -85,7 +85,7 @@ public class TestGlobbedCopyListing {
recordInExpectedValues(path); recordInExpectedValues(path);
} }
finally { finally {
IOUtils.cleanup(null, fileSystem, outputStream); IOUtils.cleanupWithLogger(null, fileSystem, outputStream);
} }
} }

View File

@ -261,7 +261,7 @@ public class TestCopyMapper {
System.out.println(fileStatus.getReplication()); System.out.println(fileStatus.getReplication());
} }
finally { finally {
IOUtils.cleanup(null, outputStream); IOUtils.cleanupWithLogger(null, outputStream);
} }
} }

View File

@ -91,7 +91,7 @@ public class TestUniformSizeInputFormat {
return size; return size;
} }
finally { finally {
IOUtils.cleanup(null, fileSystem, outputStream); IOUtils.cleanupWithLogger(null, fileSystem, outputStream);
} }
} }

View File

@ -100,7 +100,7 @@ public class TestDynamicInputFormat {
new Path(path))[0].getPath().toString()); new Path(path))[0].getPath().toString());
} }
finally { finally {
IOUtils.cleanup(null, fileSystem, outputStream); IOUtils.cleanupWithLogger(null, fileSystem, outputStream);
} }
} }

View File

@ -79,7 +79,7 @@ class ReadRecordFactory extends RecordFactory {
@Override @Override
public void close() throws IOException { public void close() throws IOException {
IOUtils.cleanup(null, src); IOUtils.cleanupWithLogger(null, src);
factory.close(); factory.close();
} }
} }

View File

@ -112,7 +112,7 @@ import java.util.concurrent.TimeUnit;
} catch (InterruptedException e) { } catch (InterruptedException e) {
// exit thread; ignore any jobs remaining in the trace // exit thread; ignore any jobs remaining in the trace
} finally { } finally {
IOUtils.cleanup(null, jobProducer); IOUtils.cleanupWithLogger(null, jobProducer);
} }
} }
} }

View File

@ -143,7 +143,7 @@ public class SerialJobFactory extends JobFactory<JobStats> {
} catch (InterruptedException e) { } catch (InterruptedException e) {
return; return;
} finally { } finally {
IOUtils.cleanup(null, jobProducer); IOUtils.cleanupWithLogger(null, jobProducer);
} }
} }

View File

@ -247,7 +247,7 @@ public class StressJobFactory extends JobFactory<Statistics.ClusterStats> {
LOG.error("[STRESS] Interrupted in the main block!", e); LOG.error("[STRESS] Interrupted in the main block!", e);
return; return;
} finally { } finally {
IOUtils.cleanup(null, jobProducer); IOUtils.cleanupWithLogger(null, jobProducer);
} }
} }
} }

View File

@ -470,7 +470,7 @@ public class Folder extends Configured implements Tool {
next = heap.poll(); next = heap.poll();
} }
} finally { } finally {
IOUtils.cleanup(null, reader); IOUtils.cleanupWithLogger(null, reader);
if (outGen != null) { if (outGen != null) {
outGen.close(); outGen.close();
} }