/**
   * Test that an append with no locations fails with an exception showing insufficient locations.
   */
  @Test(timeout = 60000)
  public void testAppendInsufficientLocations() throws Exception {
    Configuration conf = new Configuration();

    // lower heartbeat interval for fast recognition of DN
    conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000);
    conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
    conf.setInt(HdfsClientConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 3000);

    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(4).build();
    DistributedFileSystem fileSystem = null;
    try {
      // create a file with replication 3
      fileSystem = cluster.getFileSystem();
      Path f = new Path("/testAppend");
      FSDataOutputStream create = fileSystem.create(f, (short) 2);
      create.write("/testAppend".getBytes());
      create.close();

      // Check for replications
      DFSTestUtil.waitReplication(fileSystem, f, (short) 2);

      // Shut down all DNs that have the last block location for the file
      LocatedBlocks lbs =
          fileSystem.dfs.getNamenode().getBlockLocations("/testAppend", 0, Long.MAX_VALUE);
      List<DataNode> dnsOfCluster = cluster.getDataNodes();
      DatanodeInfo[] dnsWithLocations = lbs.getLastLocatedBlock().getLocations();
      for (DataNode dn : dnsOfCluster) {
        for (DatanodeInfo loc : dnsWithLocations) {
          if (dn.getDatanodeId().equals(loc)) {
            dn.shutdown();
            DFSTestUtil.waitForDatanodeDeath(dn);
          }
        }
      }

      // Wait till 0 replication is recognized
      DFSTestUtil.waitReplication(fileSystem, f, (short) 0);

      // Append to the file, at this state there are 3 live DNs but none of them
      // have the block.
      try {
        fileSystem.append(f);
        fail("Append should fail because insufficient locations");
      } catch (IOException e) {
        LOG.info("Expected exception: ", e);
      }
      FSDirectory dir = cluster.getNamesystem().getFSDirectory();
      final INodeFile inode = INodeFile.valueOf(dir.getINode("/testAppend"), "/testAppend");
      assertTrue("File should remain closed", !inode.isUnderConstruction());
    } finally {
      if (null != fileSystem) {
        fileSystem.close();
      }
      cluster.shutdown();
    }
  }
 /** Check whether the data-node can be started. */
 private boolean canStartDataNode(Configuration conf) throws IOException {
   DataNode dn = null;
   try {
     dn = DataNode.createDataNode(new String[] {}, conf);
   } catch (IOException e) {
     if (e instanceof java.net.BindException) return false;
     throw e;
   }
   dn.shutdown();
   return true;
 }
 /** Stop the datanode. */
 public void stopDataNode(DataNode dn) {
   if (dn != null) {
     dn.shutdown();
   }
 }