@@ -2510,6 +2510,9 @@ Release 2.8.0 - UNRELEASED
HDFS-9570. Minor typos, grammar, and case sensitivity cleanup in
HdfsPermissionsGuide.md's (Travis Campbell via aw)
+ HDFS-9515. NPE when MiniDFSCluster#shutdown is invoked on uninitialized
+ reference. (Wei-Chiu Chuang via Arpit Agarwal)
+
Release 2.7.3 - UNRELEASED
INCOMPATIBLE CHANGES
@@ -63,6 +63,7 @@ public class TestBootstrapStandbyWithBKJM {
public void teardown() {
if (cluster != null) {
cluster.shutdown();
+ cluster = null;
}
@@ -86,6 +86,7 @@ public class TestRefreshCallQueue {
public void tearDown() throws Exception {
if(cluster!=null) {
@@ -49,9 +49,11 @@ public class TestAclCLI extends CLITestHelperDFS {
super.tearDown();
if (fs != null) {
fs.close();
+ fs = null;
@@ -76,9 +76,11 @@ public class TestCacheAdminCLI extends CLITestHelper {
if (dfsCluster != null) {
dfsCluster.shutdown();
+ dfsCluster = null;
Thread.sleep(2000);
@@ -86,9 +86,11 @@ public class TestCryptoAdminCLI extends CLITestHelperDFS {
@@ -58,9 +58,11 @@ public class TestDeleteCLI extends CLITestHelperDFS {
@@ -61,9 +61,11 @@ public class TestErasureCodingCLI extends CLITestHelper {
@@ -77,9 +77,11 @@ public class TestHDFSCLI extends CLITestHelperDFS {
@@ -67,9 +67,11 @@ public class TestXAttrCLI extends CLITestHelperDFS {
@@ -58,7 +58,9 @@ public class TestFcHdfsCreateMkdir extends
@AfterClass
public static void ClusterShutdownAtEnd() throws Exception {
- cluster.shutdown();
+ if (cluster != null) {
+ cluster.shutdown();
+ }
@Override
@@ -65,7 +65,9 @@ public class TestFcHdfsPermission extends FileContextPermissionBase {
@@ -96,7 +96,9 @@ public class TestFcHdfsSetUMask {
{
@@ -96,7 +96,9 @@ abstract public class TestSymlinkHdfs extends SymlinkBaseTest {
public static void afterClassTeardown() throws Exception {
@Test(timeout=10000)
@@ -62,6 +62,7 @@ public class HDFSContract extends AbstractFSContract {
public static void destroyCluster() throws IOException {
@@ -60,9 +60,11 @@ public class TestHdfsTextCommand {
public void tearDown() throws IOException{
if(fs != null){
if(cluster != null){
@@ -88,7 +88,9 @@ public class TestViewFileSystemHdfs extends ViewFileSystemBaseTest {
@@ -76,7 +76,9 @@ public class TestViewFileSystemWithAcls {
@Before
@@ -71,7 +71,9 @@ public class TestViewFileSystemWithXAttrs {
@@ -64,7 +64,9 @@ public class TestViewFsAtHdfsRoot extends ViewFsBaseTest {
@@ -64,7 +64,9 @@ public class TestViewFsHdfs extends ViewFsBaseTest {
@@ -76,7 +76,9 @@ public class TestViewFsWithAcls {
@@ -70,7 +70,9 @@ public class TestViewFsWithXAttrs {
@@ -68,7 +68,9 @@ public class FileAppendTest4 {
public static void tearDown() {
/**
@@ -56,8 +56,14 @@ public class TestAbandonBlock {
@After
- fs.close();
+ if (fs != null) {
+ fs.close();
@Test
@@ -43,9 +43,11 @@ public class TestBlocksScheduledCounter {
public void tearDown() throws IOException {
if(cluster!=null){
@@ -82,8 +82,14 @@ public class TestClientReportBadBlock {
public void shutDownCluster() throws IOException {
- dfs.close();
+ if (dfs != null) {
+ dfs.close();
+ dfs = null;
/*
@@ -54,6 +54,7 @@ public class TestDFSClientExcludedNodes {
public void tearDown() {
@@ -88,6 +88,7 @@ public class TestDFSClientFailover {
public void tearDownCluster() throws IOException {
@@ -82,6 +82,7 @@ public class TestDFSClientSocketSize {
LOG.info("Shutting down MiniDFSCluster.");
@@ -188,7 +188,10 @@ public class TestDFSFinalize {
LOG.info("Shutting down MiniDFSCluster");
- if (cluster != null) cluster.shutdown();
public static void main(String[] args) throws Exception {
@@ -150,6 +150,8 @@ public class TestDFSOutputStream {
@@ -123,6 +123,7 @@ public class TestDFSPermission {
@@ -346,7 +346,10 @@ public class TestDFSRollback {
@@ -282,7 +282,10 @@ public class TestDFSStartupVersions {
@@ -452,6 +452,9 @@ public class TestDFSStorageStateRecovery {
@@ -86,6 +86,7 @@ public class TestDFSStripedInputStream {
@@ -69,6 +69,7 @@ public class TestDFSStripedOutputStream {
@@ -69,7 +69,10 @@ public class TestDataTransferKeepalive {
@@ -126,6 +126,7 @@ public class TestDecommission {
cleanupFile(localFileSys, dir);
@@ -170,6 +170,7 @@ public class TestEncryptionZones {
EncryptionFaultInjector.instance = new EncryptionFaultInjector();
@@ -87,6 +87,7 @@ public class TestEncryptionZonesWithHA {
public void shutdownCluster() throws IOException {
@@ -61,6 +61,7 @@ public class TestErasureCodingPolicies {
@@ -57,6 +57,7 @@ public class TestErasureCodingPolicyWithSnapshot {
@@ -84,8 +84,10 @@ public class TestFileConcurrentReader {
- cluster = null;
private void init(Configuration conf) throws IOException {
@@ -77,8 +77,12 @@ public class TestFileStatus {
public static void testTearDown() throws Exception {
private void checkFile(FileSystem fileSys, Path name, int repl)
@@ -50,6 +50,7 @@ public class TestFileStatusWithECPolicy {
public void after() {
@@ -52,6 +52,7 @@ public class TestGetFileChecksum {
@@ -45,8 +45,10 @@ public class TestHDFSFileSystemContract extends FileSystemContractBaseTest {
protected void tearDown() throws Exception {
@@ -47,6 +47,7 @@ public class TestHdfsAdmin {
public void shutDownCluster() {
@@ -57,6 +57,7 @@ public class TestLeaseRecovery {
public void shutdown() throws IOException {
@@ -101,8 +101,10 @@ public class TestLeaseRecovery2 {
*/
public static void tearDown() throws IOException {
- IOUtils.closeStream(dfs);
- if (cluster != null) {cluster.shutdown();}
+ IOUtils.closeStream(dfs);
@@ -46,8 +46,10 @@ public class TestListFilesInDFS extends TestListFiles {
public static void testShutdown() throws Exception {
protected static Path getTestDir() {
@@ -83,7 +83,9 @@ public class TestListFilesInFileContext {
/** Test when input path is a file */
@@ -65,11 +65,14 @@ public class TestPipelines {
- if (fs != null)
cluster.shutdownDataNodes();
@@ -77,6 +77,7 @@ public class TestReadStripedFileWithDecoding {
@@ -61,6 +61,7 @@ public class TestReadStripedFileWithMissingBlocks {
@@ -97,6 +97,7 @@ public class TestRecoverStripedFile {
@@ -91,6 +91,7 @@ public class TestReservedRawPaths {
@@ -90,9 +90,11 @@ public class TestSafeMode {
@@ -63,6 +63,7 @@ public class TestSafeModeWithStripedFile {
@@ -86,7 +86,10 @@ public class TestWriteRead {
public void shutdown() {
// Equivalence of @Before for cluster mode testing.
@@ -76,6 +76,7 @@ public class TestWriteReadStripedFile {
@@ -67,6 +67,7 @@ public class TestSaslDataTransfer extends SaslDataTransferTestCase {
IOUtils.cleanup(null, fs);
@@ -154,9 +154,11 @@ public class TestSecureNNWithQJM {
if (mjc != null) {
mjc.shutdown();
+ mjc = null;
@@ -114,6 +114,7 @@ public class TestQuorumJournalManager {
@@ -61,6 +61,7 @@ public class TestJournalNodeMXBean {
public void cleanup() throws IOException {
if (jCluster != null) {
jCluster.shutdown();
+ jCluster = null;
@@ -84,6 +84,7 @@ public class TestDelegationToken {
@@ -67,6 +67,7 @@ public class TestBlockStatsMXBean {
@@ -70,6 +70,7 @@ public class TestComputeInvalidateWork {
public void teardown() throws Exception {
@@ -74,6 +74,7 @@ public class TestPendingInvalidateBlock {
@@ -93,6 +93,7 @@ public class TestSequentialBlockGroupId {
@@ -115,10 +115,12 @@ public abstract class BlockReportTestBase {
@@ -59,8 +59,10 @@ public class TestDataNodeExit {
- if (cluster != null)
private void stopBPServiceThreads(int numStopThreads, DataNode dn)
@@ -116,12 +116,15 @@ public class TestDataNodeVolumeFailure {
if(data_fail != null) {
FileUtil.setWritable(data_fail, true);
+ data_fail = null;
if(failedDir != null) {
FileUtil.setWritable(failedDir, true);
+ failedDir = null;
if(cluster != null) {
@@ -89,6 +89,7 @@ public class TestDataNodeVolumeFailureReporting {
IOUtils.cleanup(LOG, fs);
@@ -78,6 +78,7 @@ public class TestDataNodeVolumeFailureToleration {
@@ -71,7 +71,10 @@ public class TestDiskError {
@@ -149,9 +149,11 @@ public class TestFsDatasetCache {
DFSTestUtil.verifyExpectedCacheUsage(0, 0, fsd);
// Restore the original CacheManipulator
NativeIO.POSIX.setCacheManipulator(prevCacheManipulator);
@@ -101,10 +101,13 @@ public class TestIncrementalBrVariations {
- client.close();
- cluster.shutdownDataNodes();
+ client.close();
+ cluster.shutdownDataNodes();
@@ -58,6 +58,7 @@ public class TestAclConfigFlag {
@@ -60,6 +60,7 @@ public class TestAddBlock {
@@ -66,6 +66,7 @@ public class TestAddBlockRetry {
@@ -75,6 +75,7 @@ public class TestAddOverReplicatedStripedBlocks {
@@ -82,6 +82,7 @@ public class TestAddStripedBlocks {
@@ -137,8 +137,14 @@ public class TestAuditLogs {
public void teardownCluster() throws Exception {
util.cleanup(fs, "/srcdat");
/** test that allowed operation puts proper entry in audit log */
@@ -494,7 +494,9 @@ public class TestBackupNode {
assertTrue(e.getLocalizedMessage(), false);
} finally {
fileSys.close();
@@ -83,6 +83,7 @@ public class TestBlockPlacementPolicyRackFaultTolerant {
@@ -158,6 +158,7 @@ public class TestCacheDirectives {
waitForCachedBlocks(namenode, 0, 0, "teardown");
@@ -58,6 +58,7 @@ public class TestCommitBlockWithInvalidGenStamp {
@@ -62,7 +62,10 @@ public class TestDeadDatanode {
public void cleanup() {
@@ -71,6 +71,7 @@ public class TestDefaultBlockPlacementPolicy {
@@ -71,6 +71,7 @@ public class TestDiskspaceQuotaUpdate {
@@ -97,9 +97,11 @@ public class TestEditLogAutoroll {
@@ -70,11 +70,14 @@ public class TestEditLogJournalFailures {
public void shutDownMiniCluster() throws IOException {
try {
} catch (ExitException ee) {
// Ignore ExitExceptions as the tests may result in the
// NameNode doing an immediate shutdown.
@@ -108,6 +108,7 @@ public class TestFSDirectory {
@@ -55,7 +55,9 @@ public class TestFSImageWithAcl {
private void testAcl(boolean persistNamespace) throws IOException {
@@ -89,6 +89,7 @@ public class TestFSImageWithSnapshot {
@@ -62,7 +62,9 @@ public class TestFSImageWithXAttr {
private void testXAttr(boolean persistNamespace) throws IOException {
@@ -110,8 +110,14 @@ public class TestFileTruncate {
- if(fs != null) fs.close();
- if(cluster != null) cluster.shutdown();
+ if(fs != null) {
+ if(cluster != null) {