You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by um...@apache.org on 2015/05/08 08:57:28 UTC

hadoop git commit: HDFS-8332. DFS client API calls should check filesystem closed. Contributed by Rakesh R.

Repository: hadoop
Updated Branches:
  refs/heads/trunk ef3d66d46 -> e16f4b7f7


HDFS-8332. DFS client API calls should check filesystem closed. Contributed by Rakesh R.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/e16f4b7f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e16f4b7f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e16f4b7f

Branch: refs/heads/trunk
Commit: e16f4b7f70b8675760cf5aaa471dfe29d48041e6
Parents: ef3d66d
Author: Uma Maheswara Rao G <um...@apache.org>
Authored: Fri May 8 12:26:47 2015 +0530
Committer: Uma Maheswara Rao G <um...@apache.org>
Committed: Fri May 8 12:26:47 2015 +0530

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   2 +
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  25 +++
 .../hadoop/hdfs/TestDistributedFileSystem.java  | 167 ++++++++++++++++++-
 .../hadoop/hdfs/TestRollingUpgradeRollback.java |   1 +
 .../server/namenode/TestCacheDirectives.java    |  59 ++++++-
 5 files changed, 247 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e16f4b7f/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 76cd03b..656e005 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -661,6 +661,8 @@ Release 2.8.0 - UNRELEASED
     HDFS-6291. FSImage may be left unclosed in BootstrapStandby#doRun()
     (Sanghyun Yun via vinayakumarb)
 
+    HDFS-8332. DFS client API calls should check filesystem closed (Rakesh R via umamahesh)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e16f4b7f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index c145959..191ebc9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -638,6 +638,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @see ClientProtocol#getPreferredBlockSize(String)
    */
   public long getBlockSize(String f) throws IOException {
+    checkOpen();
     TraceScope scope = getPathTraceScope("getBlockSize", f);
     try {
       return namenode.getPreferredBlockSize(f);
@@ -654,6 +655,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @see ClientProtocol#getServerDefaults()
    */
   public FsServerDefaults getServerDefaults() throws IOException {
+    checkOpen();
     long now = Time.monotonicNow();
     if ((serverDefaults == null) ||
         (now - serverDefaultsLastUpdate > SERVER_DEFAULTS_VALIDITY_PERIOD)) {
@@ -845,6 +847,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @see ClientProtocol#reportBadBlocks(LocatedBlock[])
    */
   public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
+    checkOpen();
     namenode.reportBadBlocks(blocks);
   }
   
@@ -918,6 +921,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public BlockLocation[] getBlockLocations(String src, long start, 
         long length) throws IOException, UnresolvedLinkException {
+    checkOpen();
     TraceScope scope = getPathTraceScope("getBlockLocations", src);
     try {
       LocatedBlocks blocks = getLocatedBlocks(src, start, length);
@@ -952,6 +956,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public BlockStorageLocation[] getBlockStorageLocations(
       List<BlockLocation> blockLocations) throws IOException,
       UnsupportedOperationException, InvalidBlockTokenException {
+    checkOpen();
     if (!getConf().isHdfsBlocksMetadataEnabled()) {
       throw new UnsupportedOperationException("Datanode-side support for " +
           "getVolumeBlockLocations() must also be enabled in the client " +
@@ -1418,6 +1423,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public void createSymlink(String target, String link, boolean createParent)
       throws IOException {
+    checkOpen();
     TraceScope scope = getPathTraceScope("createSymlink", target);
     try {
       final FsPermission dirPerm = applyUMask(null);
@@ -1540,6 +1546,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public boolean setReplication(String src, short replication)
       throws IOException {
+    checkOpen();
     TraceScope scope = getPathTraceScope("setReplication", src);
     try {
       return namenode.setReplication(src, replication);
@@ -1563,6 +1570,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public void setStoragePolicy(String src, String policyName)
       throws IOException {
+    checkOpen();
     TraceScope scope = getPathTraceScope("setStoragePolicy", src);
     try {
       namenode.setStoragePolicy(src, policyName);
@@ -1582,6 +1590,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @return All the existing storage policies
    */
   public BlockStoragePolicy[] getStoragePolicies() throws IOException {
+    checkOpen();
     TraceScope scope = Trace.startSpan("getStoragePolicies", traceSampler);
     try {
       return namenode.getStoragePolicies();
@@ -2232,6 +2241,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @see ClientProtocol#setSafeMode(HdfsConstants.SafeModeAction,boolean)
    */
   public boolean setSafeMode(SafeModeAction action) throws IOException {
+    checkOpen();
     return setSafeMode(action, false);
   }
   
@@ -2434,6 +2444,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   
   public RemoteIterator<CacheDirectiveEntry> listCacheDirectives(
       CacheDirectiveInfo filter) throws IOException {
+    checkOpen();
     return new CacheDirectiveIterator(namenode, filter, traceSampler);
   }
 
@@ -2474,6 +2485,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   }
 
   public RemoteIterator<CachePoolEntry> listCachePools() throws IOException {
+    checkOpen();
     return new CachePoolIterator(namenode, traceSampler);
   }
 
@@ -2483,6 +2495,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @see ClientProtocol#saveNamespace(long, long)
    */
   boolean saveNamespace(long timeWindow, long txGap) throws IOException {
+    checkOpen();
     TraceScope scope = Trace.startSpan("saveNamespace", traceSampler);
     try {
       return namenode.saveNamespace(timeWindow, txGap);
@@ -2500,6 +2513,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @see ClientProtocol#rollEdits()
    */
   long rollEdits() throws AccessControlException, IOException {
+    checkOpen();
     TraceScope scope = Trace.startSpan("rollEdits", traceSampler);
     try {
       return namenode.rollEdits();
@@ -2522,6 +2536,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   boolean restoreFailedStorage(String arg)
       throws AccessControlException, IOException{
+    checkOpen();
     TraceScope scope = Trace.startSpan("restoreFailedStorage", traceSampler);
     try {
       return namenode.restoreFailedStorage(arg);
@@ -2538,6 +2553,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @see ClientProtocol#refreshNodes()
    */
   public void refreshNodes() throws IOException {
+    checkOpen();
     TraceScope scope = Trace.startSpan("refreshNodes", traceSampler);
     try {
       namenode.refreshNodes();
@@ -2552,6 +2568,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @see ClientProtocol#metaSave(String)
    */
   public void metaSave(String pathname) throws IOException {
+    checkOpen();
     TraceScope scope = Trace.startSpan("metaSave", traceSampler);
     try {
       namenode.metaSave(pathname);
@@ -2569,6 +2586,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @see ClientProtocol#setBalancerBandwidth(long)
    */
   public void setBalancerBandwidth(long bandwidth) throws IOException {
+    checkOpen();
     TraceScope scope = Trace.startSpan("setBalancerBandwidth", traceSampler);
     try {
       namenode.setBalancerBandwidth(bandwidth);
@@ -2581,6 +2599,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @see ClientProtocol#finalizeUpgrade()
    */
   public void finalizeUpgrade() throws IOException {
+    checkOpen();
     TraceScope scope = Trace.startSpan("finalizeUpgrade", traceSampler);
     try {
       namenode.finalizeUpgrade();
@@ -2590,6 +2609,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   }
 
   RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action) throws IOException {
+    checkOpen();
     TraceScope scope = Trace.startSpan("rollingUpgrade", traceSampler);
     try {
       return namenode.rollingUpgrade(action);
@@ -2675,6 +2695,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @see ClientProtocol#getContentSummary(String)
    */
   ContentSummary getContentSummary(String src) throws IOException {
+    checkOpen();
     TraceScope scope = getPathTraceScope("getContentSummary", src);
     try {
       return namenode.getContentSummary(src);
@@ -2693,6 +2714,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   void setQuota(String src, long namespaceQuota, long storagespaceQuota)
       throws IOException {
+    checkOpen();
     // sanity check
     if ((namespaceQuota <= 0 && namespaceQuota != HdfsConstants.QUOTA_DONT_SET &&
          namespaceQuota != HdfsConstants.QUOTA_RESET) ||
@@ -2726,6 +2748,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   void setQuotaByStorageType(String src, StorageType type, long quota)
       throws IOException {
+    checkOpen();
     if (quota <= 0 && quota != HdfsConstants.QUOTA_DONT_SET &&
         quota != HdfsConstants.QUOTA_RESET) {
       throw new IllegalArgumentException("Invalid values for quota :" +
@@ -3071,11 +3094,13 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   }
 
   public DFSInotifyEventInputStream getInotifyEventStream() throws IOException {
+    checkOpen();
     return new DFSInotifyEventInputStream(traceSampler, namenode);
   }
 
   public DFSInotifyEventInputStream getInotifyEventStream(long lastReadTxid)
       throws IOException {
+    checkOpen();
     return new DFSInotifyEventInputStream(traceSampler, namenode, lastReadTxid);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e16f4b7f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
index 837665e..f9da472 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
@@ -62,11 +62,15 @@ import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
 import org.apache.hadoop.fs.Options.ChecksumOpt;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.VolumeId;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
-import org.apache.hadoop.hdfs.client.impl.LeaseRenewer;
 import org.apache.hadoop.hdfs.net.Peer;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeFaultInjector;
 import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
 import org.apache.hadoop.hdfs.web.WebHdfsConstants;
@@ -160,25 +164,176 @@ public class TestDistributedFileSystem {
     MiniDFSCluster cluster = null;
     try {
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
-      FileSystem fileSys = cluster.getFileSystem();
-      
+      DistributedFileSystem fileSys = cluster.getFileSystem();
+
       // create two files, leaving them open
       fileSys.create(new Path("/test/dfsclose/file-0"));
       fileSys.create(new Path("/test/dfsclose/file-1"));
-      
+
       // create another file, close it, and read it, so
       // the client gets a socket in its SocketCache
       Path p = new Path("/non-empty-file");
       DFSTestUtil.createFile(fileSys, p, 1L, (short)1, 0L);
       DFSTestUtil.readFile(fileSys, p);
-      
+
       fileSys.close();
-      
+
+      DFSClient dfsClient = fileSys.getClient();
+      verifyOpsUsingClosedClient(dfsClient);
     } finally {
       if (cluster != null) {cluster.shutdown();}
     }
   }
 
+  private void verifyOpsUsingClosedClient(DFSClient dfsClient) {
+    Path p = new Path("/non-empty-file");
+    try {
+      dfsClient.getBlockSize(p.getName());
+      fail("getBlockSize using a closed filesystem!");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("Filesystem closed", ioe);
+    }
+    try {
+      dfsClient.getServerDefaults();
+      fail("getServerDefaults using a closed filesystem!");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("Filesystem closed", ioe);
+    }
+    try {
+      dfsClient.reportBadBlocks(new LocatedBlock[0]);
+      fail("reportBadBlocks using a closed filesystem!");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("Filesystem closed", ioe);
+    }
+    try {
+      dfsClient.getBlockLocations(p.getName(), 0, 1);
+      fail("getBlockLocations using a closed filesystem!");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("Filesystem closed", ioe);
+    }
+    try {
+      dfsClient.getBlockStorageLocations(new ArrayList<BlockLocation>());
+      fail("getBlockStorageLocations using a closed filesystem!");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("Filesystem closed", ioe);
+    }
+    try {
+      dfsClient.createSymlink("target", "link", true);
+      fail("createSymlink using a closed filesystem!");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("Filesystem closed", ioe);
+    }
+    try {
+      dfsClient.getLinkTarget(p.getName());
+      fail("getLinkTarget using a closed filesystem!");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("Filesystem closed", ioe);
+    }
+    try {
+      dfsClient.setReplication(p.getName(), (short) 3);
+      fail("setReplication using a closed filesystem!");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("Filesystem closed", ioe);
+    }
+    try {
+      dfsClient.setStoragePolicy(p.getName(),
+          HdfsConstants.ONESSD_STORAGE_POLICY_NAME);
+      fail("setStoragePolicy using a closed filesystem!");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("Filesystem closed", ioe);
+    }
+    try {
+      dfsClient.getStoragePolicies();
+      fail("getStoragePolicies using a closed filesystem!");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("Filesystem closed", ioe);
+    }
+    try {
+      dfsClient.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+      fail("setSafeMode using a closed filesystem!");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("Filesystem closed", ioe);
+    }
+    try {
+      dfsClient.refreshNodes();
+      fail("refreshNodes using a closed filesystem!");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("Filesystem closed", ioe);
+    }
+    try {
+      dfsClient.metaSave(p.getName());
+      fail("metaSave using a closed filesystem!");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("Filesystem closed", ioe);
+    }
+    try {
+      dfsClient.setBalancerBandwidth(1000L);
+      fail("setBalancerBandwidth using a closed filesystem!");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("Filesystem closed", ioe);
+    }
+    try {
+      dfsClient.finalizeUpgrade();
+      fail("finalizeUpgrade using a closed filesystem!");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("Filesystem closed", ioe);
+    }
+    try {
+      dfsClient.rollingUpgrade(RollingUpgradeAction.QUERY);
+      fail("rollingUpgrade using a closed filesystem!");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("Filesystem closed", ioe);
+    }
+    try {
+      dfsClient.getInotifyEventStream();
+      fail("getInotifyEventStream using a closed filesystem!");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("Filesystem closed", ioe);
+    }
+    try {
+      dfsClient.getInotifyEventStream(100L);
+      fail("getInotifyEventStream using a closed filesystem!");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("Filesystem closed", ioe);
+    }
+    try {
+      dfsClient.saveNamespace(1000L, 200L);
+      fail("saveNamespace using a closed filesystem!");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("Filesystem closed", ioe);
+    }
+    try {
+      dfsClient.rollEdits();
+      fail("rollEdits using a closed filesystem!");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("Filesystem closed", ioe);
+    }
+    try {
+      dfsClient.restoreFailedStorage("");
+      fail("restoreFailedStorage using a closed filesystem!");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("Filesystem closed", ioe);
+    }
+    try {
+      dfsClient.getContentSummary(p.getName());
+      fail("getContentSummary using a closed filesystem!");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("Filesystem closed", ioe);
+    }
+    try {
+      dfsClient.setQuota(p.getName(), 1000L, 500L);
+      fail("setQuota using a closed filesystem!");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("Filesystem closed", ioe);
+    }
+    try {
+      dfsClient.setQuotaByStorageType(p.getName(), StorageType.DISK, 500L);
+      fail("setQuotaByStorageType using a closed filesystem!");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("Filesystem closed", ioe);
+    }
+  }
+
   @Test
   public void testDFSCloseOrdering() throws Exception {
     DistributedFileSystem fs = new MyDistributedFileSystem();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e16f4b7f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgradeRollback.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgradeRollback.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgradeRollback.java
index c2e9d7c..38cfb92 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgradeRollback.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgradeRollback.java
@@ -228,6 +228,7 @@ public class TestRollingUpgradeRollback {
       dfs.mkdirs(bar);
       dfs.close();
 
+      dfs = dfsCluster.getFileSystem(0);
       TestRollingUpgrade.queryForPreparation(dfs);
 
       // If the query returns true, both active and the standby NN should have

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e16f4b7f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java
index 3617ee3..6027934 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java
@@ -59,7 +59,6 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.LogVerificationAppender;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
@@ -296,6 +295,35 @@ public class TestCacheDirectives {
 
     info = new CachePoolInfo("pool2");
     dfs.addCachePool(info);
+
+    // Perform cache pool operations using a closed file system.
+    DistributedFileSystem dfs1 = (DistributedFileSystem) cluster
+        .getNewFileSystemInstance(0);
+    dfs1.close();
+    try {
+      dfs1.listCachePools();
+      fail("listCachePools using a closed filesystem!");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("Filesystem closed", ioe);
+    }
+    try {
+      dfs1.addCachePool(info);
+      fail("addCachePool using a closed filesystem!");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("Filesystem closed", ioe);
+    }
+    try {
+      dfs1.modifyCachePool(info);
+      fail("modifyCachePool using a closed filesystem!");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("Filesystem closed", ioe);
+    }
+    try {
+      dfs1.removeCachePool(poolName);
+      fail("removeCachePool using a closed filesystem!");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("Filesystem closed", ioe);
+    }
   }
 
   @Test(timeout=60000)
@@ -538,6 +566,35 @@ public class TestCacheDirectives {
     dfs.modifyCacheDirective(new CacheDirectiveInfo.Builder(
         directive).setId(id).setReplication((short)2).build());
     dfs.removeCacheDirective(id);
+
+    // Perform cache directive operations using a closed file system.
+    DistributedFileSystem dfs1 = (DistributedFileSystem) cluster
+        .getNewFileSystemInstance(0);
+    dfs1.close();
+    try {
+      dfs1.listCacheDirectives(null);
+      fail("listCacheDirectives using a closed filesystem!");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("Filesystem closed", ioe);
+    }
+    try {
+      dfs1.addCacheDirective(alpha);
+      fail("addCacheDirective using a closed filesystem!");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("Filesystem closed", ioe);
+    }
+    try {
+      dfs1.modifyCacheDirective(alpha);
+      fail("modifyCacheDirective using a closed filesystem!");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("Filesystem closed", ioe);
+    }
+    try {
+      dfs1.removeCacheDirective(alphaId);
+      fail("removeCacheDirective using a closed filesystem!");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("Filesystem closed", ioe);
+    }
   }
 
   @Test(timeout=60000)