HADOOP-17622. Avoid usage of deprecated IOUtils#cleanup API. (#2862)
Signed-off-by: Takanobu Asanuma <tasanuma@apache.org>
(cherry picked from commit 3f2682b92b
)
This commit is contained in:
parent
2b5fd341b9
commit
8b4b3d6fe6
|
@ -312,7 +312,7 @@ public class TestLocalFileSystem {
|
|||
.new LocalFSFileInputStream(path), 1024);
|
||||
assertNotNull(bis.getFileDescriptor());
|
||||
} finally {
|
||||
IOUtils.cleanup(null, bis);
|
||||
IOUtils.cleanupWithLogger(null, bis);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -115,7 +115,7 @@ public class TestFileSink {
|
|||
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 class TestDomainSocket {
|
|||
readerThread.join();
|
||||
Assert.assertFalse(failed.get());
|
||||
Assert.assertEquals(3, bytesRead.get());
|
||||
IOUtils.cleanup(null, socks);
|
||||
IOUtils.cleanupWithLogger(null, socks);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -617,8 +617,8 @@ class BPServiceActor implements Runnable {
|
|||
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);
|
||||
}
|
||||
|
||||
|
@ -989,7 +989,7 @@ class BPServiceActor implements Runnable {
|
|||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
IOUtils.cleanup(null, lifelineNamenode);
|
||||
IOUtils.cleanupWithLogger(null, lifelineNamenode);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -369,7 +369,7 @@ class BlockReceiver implements Closeable {
|
|||
streams.close();
|
||||
}
|
||||
if (replicaHandler != null) {
|
||||
IOUtils.cleanup(null, replicaHandler);
|
||||
IOUtils.cleanupWithLogger(null, replicaHandler);
|
||||
replicaHandler = null;
|
||||
}
|
||||
if (measuredFlushTime) {
|
||||
|
|
|
@ -252,7 +252,7 @@ public class BlockScanner {
|
|||
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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -431,7 +431,7 @@ class DataXceiver extends Receiver implements Runnable {
|
|||
blk.getBlockId(), dnR.getDatanodeUuid(), success));
|
||||
}
|
||||
if (fis != null) {
|
||||
IOUtils.cleanup(null, fis);
|
||||
IOUtils.cleanupWithLogger(null, fis);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -554,7 +554,7 @@ class DataXceiver extends Receiver implements Runnable {
|
|||
LOG.warn("Failed to shut down socket in error handler", e);
|
||||
}
|
||||
}
|
||||
IOUtils.cleanup(null, shmInfo);
|
||||
IOUtils.cleanupWithLogger(null, shmInfo);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -451,7 +451,7 @@ public class VolumeScanner extends Thread {
|
|||
} 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 class VolumeScanner extends Thread {
|
|||
// 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 class VolumeScanner extends Thread {
|
|||
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 class DatasetVolumeChecker {
|
|||
}
|
||||
}), MoreExecutors.directExecutor());
|
||||
} else {
|
||||
IOUtils.cleanup(null, reference);
|
||||
IOUtils.cleanupWithLogger(null, reference);
|
||||
if (numVolumes.decrementAndGet() == 0) {
|
||||
latch.countDown();
|
||||
}
|
||||
|
@ -311,7 +311,7 @@ public class DatasetVolumeChecker {
|
|||
);
|
||||
return true;
|
||||
} else {
|
||||
IOUtils.cleanup(null, volumeReference);
|
||||
IOUtils.cleanupWithLogger(null, volumeReference);
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
@ -404,7 +404,7 @@ public class DatasetVolumeChecker {
|
|||
}
|
||||
|
||||
private void cleanup() {
|
||||
IOUtils.cleanup(null, reference);
|
||||
IOUtils.cleanupWithLogger(null, reference);
|
||||
invokeCallback();
|
||||
}
|
||||
|
||||
|
|
|
@ -158,7 +158,7 @@ class StripedBlockReader {
|
|||
return peer;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.cleanup(null, peer);
|
||||
IOUtils.cleanupWithLogger(null, peer);
|
||||
IOUtils.closeSocket(sock);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -130,7 +130,7 @@ public class ReplicaInputStreams implements Closeable {
|
|||
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 class ReplicaInputStreams implements Closeable {
|
|||
dataInFd = null;
|
||||
IOUtils.closeStream(checksumIn);
|
||||
checksumIn = null;
|
||||
IOUtils.cleanup(null, volumeRef);
|
||||
IOUtils.cleanupWithLogger(null, volumeRef);
|
||||
volumeRef = null;
|
||||
}
|
||||
}
|
|
@ -335,7 +335,7 @@ class FsDatasetAsyncDiskService {
|
|||
+ block.getLocalBlock() + " URI " + replicaToDelete.getBlockURI());
|
||||
}
|
||||
updateDeletedBlockId(block);
|
||||
IOUtils.cleanup(null, volumeRef);
|
||||
IOUtils.cleanupWithLogger(null, volumeRef);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -905,11 +905,11 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|||
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;
|
||||
}
|
||||
}
|
||||
|
@ -1252,7 +1252,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|||
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);
|
||||
|
@ -1384,7 +1384,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|||
replica = (ReplicaInPipeline) replicaInfo;
|
||||
}
|
||||
} catch (IOException e) {
|
||||
IOUtils.cleanup(null, ref);
|
||||
IOUtils.cleanupWithLogger(null, ref);
|
||||
throw e;
|
||||
}
|
||||
return new ReplicaHandler(replica, ref);
|
||||
|
@ -1478,7 +1478,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|||
+ " for block " + b.getBlockId());
|
||||
}
|
||||
} catch (IOException e) {
|
||||
IOUtils.cleanup(null, ref);
|
||||
IOUtils.cleanupWithLogger(null, ref);
|
||||
throw e;
|
||||
}
|
||||
|
||||
|
@ -1570,7 +1570,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|||
// 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);
|
||||
|
@ -1716,7 +1716,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|||
try {
|
||||
newReplicaInfo = v.createTemporary(b);
|
||||
} catch (IOException e) {
|
||||
IOUtils.cleanup(null, ref);
|
||||
IOUtils.cleanupWithLogger(null, ref);
|
||||
throw e;
|
||||
}
|
||||
|
||||
|
|
|
@ -117,7 +117,7 @@ public class FsDatasetUtil {
|
|||
}
|
||||
return raf.getFD();
|
||||
} catch(IOException ioe) {
|
||||
IOUtils.cleanup(null, raf);
|
||||
IOUtils.cleanupWithLogger(null, raf);
|
||||
throw ioe;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -296,7 +296,7 @@ class FsVolumeList {
|
|||
} 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 class EditsDoubleBuffer {
|
|||
+ " 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 @@ public class DebugAdmin extends Configured implements Tool {
|
|||
blockFile);
|
||||
return 0;
|
||||
} 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);
|
||||
return 0;
|
||||
} finally {
|
||||
IOUtils.cleanup(null, metaOut);
|
||||
IOUtils.cleanupWithLogger(null, metaOut);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -319,10 +319,10 @@ abstract class PBImageTextWriter 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 @@ abstract class PBImageTextWriter implements Closeable {
|
|||
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 @@ abstract class PBImageTextWriter implements Closeable {
|
|||
@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 class TestEnhancedByteBufferAccess {
|
|||
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 class TestUnbuffer {
|
|||
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 class TestUnbuffer {
|
|||
}
|
||||
} finally {
|
||||
for (FSDataInputStream stream : streams) {
|
||||
IOUtils.cleanup(null, stream);
|
||||
IOUtils.cleanupWithLogger(null, stream);
|
||||
}
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
|
|
|
@ -96,7 +96,7 @@ public class TestStickyBit {
|
|||
|
||||
@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 @@ public class TestStickyBit {
|
|||
h.close();
|
||||
h = null;
|
||||
} finally {
|
||||
IOUtils.cleanup(null, h);
|
||||
IOUtils.cleanupWithLogger(null, h);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -500,7 +500,7 @@ public class TestStickyBit {
|
|||
o.close();
|
||||
o = null;
|
||||
} finally {
|
||||
IOUtils.cleanup(null, o);
|
||||
IOUtils.cleanupWithLogger(null, o);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -227,7 +227,7 @@ public class TestDataTransferKeepalive {
|
|||
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 class TestHFlush {
|
|||
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 class TestPread {
|
|||
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 class TestRollingUpgrade {
|
|||
// 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 class TestSecureEncryptionZoneWithKMS {
|
|||
|
||||
@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 class TestTrashWithSecureEncryptionZones {
|
|||
|
||||
@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 class TestWriteConfigurationToDFS {
|
|||
fs.close();
|
||||
fs = null;
|
||||
} finally {
|
||||
IOUtils.cleanup(null, os, fs);
|
||||
IOUtils.cleanupWithLogger(null, os, fs);
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -256,7 +256,7 @@ public class TestSaslDataTransfer extends SaslDataTransferTestCase {
|
|||
} 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 abstract class QJMTestUtil {
|
|||
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 class TestSecureNNWithQJM {
|
|||
|
||||
@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 @@ public class TestSecureNNWithQJM {
|
|||
* @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 class TestBlockReplacement {
|
|||
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 class TestCachingStrategy {
|
|||
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 @@ public class TestStorageMover {
|
|||
}
|
||||
|
||||
void shutdownCluster() throws Exception {
|
||||
IOUtils.cleanup(null, dfs);
|
||||
IOUtils.cleanupWithLogger(null, dfs);
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
|
|
|
@ -108,7 +108,8 @@ public abstract class FSAclBaseTest {
|
|||
|
||||
@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 class FSXAttrBaseTest {
|
|||
|
||||
@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 class TestAddStripedBlocks {
|
|||
assertTrue(blocks[0].isStriped());
|
||||
checkStripedBlockUC((BlockInfoStriped) fileNode.getLastBlock(), false);
|
||||
} finally {
|
||||
IOUtils.cleanup(null, out);
|
||||
IOUtils.cleanupWithLogger(null, out);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -271,7 +271,7 @@ public class TestAddStripedBlocks {
|
|||
assertArrayEquals(indices, blockIndices);
|
||||
assertArrayEquals(expectedDNs, datanodes);
|
||||
} finally {
|
||||
IOUtils.cleanup(null, out);
|
||||
IOUtils.cleanupWithLogger(null, out);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -327,7 +327,7 @@ public class TestAddStripedBlocks {
|
|||
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 class TestCommitBlockWithInvalidGenStamp {
|
|||
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 class TestEditLog {
|
|||
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 class TestEditLog {
|
|||
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 class TestEditLogFileOutputStream {
|
|||
editLogStream.abort();
|
||||
editLogStream.abort();
|
||||
} finally {
|
||||
IOUtils.cleanup(null, editLogStream);
|
||||
IOUtils.cleanupWithLogger(null, editLogStream);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -567,7 +567,7 @@ public class TestFsck {
|
|||
}
|
||||
}
|
||||
} finally {
|
||||
IOUtils.cleanup(null, in);
|
||||
IOUtils.cleanupWithLogger(null, in);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -212,7 +212,7 @@ public class TestMetaSave {
|
|||
line = rdr.readLine();
|
||||
}
|
||||
} finally {
|
||||
IOUtils.cleanup(null, rdr, isr, fis);
|
||||
IOUtils.cleanupWithLogger(null, rdr, isr, fis);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -271,7 +271,7 @@ public class TestMetaSave {
|
|||
line = rdr.readLine();
|
||||
}
|
||||
} finally {
|
||||
IOUtils.cleanup(null, rdr, isr, fis);
|
||||
IOUtils.cleanupWithLogger(null, rdr, isr, fis);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -140,7 +140,7 @@ public class TestQuotaWithStripedBlocks {
|
|||
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 class TestRetryCacheWithHA {
|
|||
} 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 class TestAclWithSnapshot {
|
|||
|
||||
@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 class TestXAttrWithSnapshot {
|
|||
|
||||
@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 class TestAtomicFileOutputStream {
|
|||
fos = null;
|
||||
}
|
||||
} finally {
|
||||
IOUtils.cleanup(null, fos);
|
||||
IOUtils.cleanupWithLogger(null, fos);
|
||||
FileUtil.setWritable(TEST_DIR, true);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -95,7 +95,7 @@ public class TestWebHDFSForHA {
|
|||
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 class TestWebHDFSForHA {
|
|||
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 class TestWebHDFSForHA {
|
|||
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 class TestWebHDFSForHA {
|
|||
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 class TestWebHDFSForHA {
|
|||
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 class TestWebHDFSForHA {
|
|||
} 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 class TestDistCpSync {
|
|||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
IOUtils.cleanup(null, dfs);
|
||||
IOUtils.cleanupWithLogger(null, dfs);
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
|
|
|
@ -153,7 +153,7 @@ public abstract class TestDistCpSyncReverseBase {
|
|||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
IOUtils.cleanup(null, dfs);
|
||||
IOUtils.cleanupWithLogger(null, dfs);
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
|
|
|
@ -96,7 +96,7 @@ public class TestDistCpWithAcls {
|
|||
|
||||
@AfterClass
|
||||
public static void shutdown() {
|
||||
IOUtils.cleanup(null, fs);
|
||||
IOUtils.cleanupWithLogger(null, fs);
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
|
|
|
@ -73,7 +73,7 @@ public class TestDistCpWithRawXAttrs {
|
|||
|
||||
@AfterClass
|
||||
public static void shutdown() {
|
||||
IOUtils.cleanup(null, fs);
|
||||
IOUtils.cleanupWithLogger(null, fs);
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
|
|
|
@ -114,7 +114,7 @@ public class TestDistCpWithXAttrs {
|
|||
|
||||
@AfterClass
|
||||
public static void shutdown() {
|
||||
IOUtils.cleanup(null, fs);
|
||||
IOUtils.cleanupWithLogger(null, fs);
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
|
|
|
@ -72,7 +72,7 @@ public class TestGlobbedCopyListing {
|
|||
recordInExpectedValues(path);
|
||||
}
|
||||
finally {
|
||||
IOUtils.cleanup(null, fileSystem);
|
||||
IOUtils.cleanupWithLogger(null, fileSystem);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -85,7 +85,7 @@ public class TestGlobbedCopyListing {
|
|||
recordInExpectedValues(path);
|
||||
}
|
||||
finally {
|
||||
IOUtils.cleanup(null, fileSystem, outputStream);
|
||||
IOUtils.cleanupWithLogger(null, fileSystem, outputStream);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -261,7 +261,7 @@ public class TestCopyMapper {
|
|||
System.out.println(fileStatus.getReplication());
|
||||
}
|
||||
finally {
|
||||
IOUtils.cleanup(null, outputStream);
|
||||
IOUtils.cleanupWithLogger(null, outputStream);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -91,7 +91,7 @@ public class TestUniformSizeInputFormat {
|
|||
return size;
|
||||
}
|
||||
finally {
|
||||
IOUtils.cleanup(null, fileSystem, outputStream);
|
||||
IOUtils.cleanupWithLogger(null, fileSystem, outputStream);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -100,7 +100,7 @@ public class TestDynamicInputFormat {
|
|||
new Path(path))[0].getPath().toString());
|
||||
}
|
||||
finally {
|
||||
IOUtils.cleanup(null, fileSystem, outputStream);
|
||||
IOUtils.cleanupWithLogger(null, fileSystem, outputStream);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -79,7 +79,7 @@ class ReadRecordFactory extends RecordFactory {
|
|||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
IOUtils.cleanup(null, src);
|
||||
IOUtils.cleanupWithLogger(null, src);
|
||||
factory.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -112,7 +112,7 @@ import java.util.concurrent.TimeUnit;
|
|||
} 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 class SerialJobFactory extends JobFactory<JobStats> {
|
|||
} catch (InterruptedException e) {
|
||||
return;
|
||||
} finally {
|
||||
IOUtils.cleanup(null, jobProducer);
|
||||
IOUtils.cleanupWithLogger(null, jobProducer);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -247,7 +247,7 @@ public class StressJobFactory extends JobFactory<Statistics.ClusterStats> {
|
|||
LOG.error("[STRESS] Interrupted in the main block!", e);
|
||||
return;
|
||||
} finally {
|
||||
IOUtils.cleanup(null, jobProducer);
|
||||
IOUtils.cleanupWithLogger(null, jobProducer);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -470,7 +470,7 @@ public class Folder extends Configured implements Tool {
|
|||
next = heap.poll();
|
||||
}
|
||||
} finally {
|
||||
IOUtils.cleanup(null, reader);
|
||||
IOUtils.cleanupWithLogger(null, reader);
|
||||
if (outGen != null) {
|
||||
outGen.close();
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue