/**
   * Append to a partial CRC chunk and the first write does not fill up the partial CRC trunk *
   *
   * @throws IOException
   */
  public void testAppendToPartialChunk() throws IOException {
    final Path p = new Path("/partialChunk/foo");
    final int fileLen = 513;
    System.out.println("p=" + p);

    byte[] fileContents = AppendTestUtil.initBuffer(fileLen);

    // create a new file.
    FSDataOutputStream stm = AppendTestUtil.createFile(fs, p, 1);

    // create 1 byte file
    stm.write(fileContents, 0, 1);
    stm.close();
    System.out.println("Wrote 1 byte and closed the file " + p);

    // append to file
    stm = fs.append(p);
    // Append to a partial CRC trunk
    stm.write(fileContents, 1, 1);
    stm.hflush();
    // The partial CRC trunk is not full yet and close the file
    stm.close();
    System.out.println("Append 1 byte and closed the file " + p);

    // write the remainder of the file
    stm = fs.append(p);

    // ensure getPos is set to reflect existing size of the file
    assertEquals(2, stm.getPos());

    // append to a partial CRC trunk
    stm.write(fileContents, 2, 1);
    // The partial chunk is not full yet, force to send a packet to DN
    stm.hflush();
    System.out.println("Append and flush 1 byte");
    // The partial chunk is not full yet, force to send another packet to DN
    stm.write(fileContents, 3, 2);
    stm.hflush();
    System.out.println("Append and flush 2 byte");

    // fill up the partial chunk and close the file
    stm.write(fileContents, 5, fileLen - 5);
    stm.close();
    System.out.println("Flush 508 byte and closed the file " + p);

    // verify that entire file is good
    AppendTestUtil.checkFullFile(
        fs, p, fileLen, fileContents, "Failed to append to a partial chunk");
  }
  @Test
  public void testAppend() throws Exception {
    final Configuration conf = new HdfsConfiguration();
    final short REPLICATION = (short) 3;

    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();

    try {
      final DistributedFileSystem fs = cluster.getFileSystem();
      final Path f = new Path(DIR, "testAppend");

      {
        LOG.info("create an empty file " + f);
        fs.create(f, REPLICATION).close();
        final FileStatus status = fs.getFileStatus(f);
        Assert.assertEquals(REPLICATION, status.getReplication());
        Assert.assertEquals(0L, status.getLen());
      }

      final byte[] bytes = new byte[1000];
      {
        LOG.info("append " + bytes.length + " bytes to " + f);
        final FSDataOutputStream out = fs.append(f);
        out.write(bytes);
        out.close();

        final FileStatus status = fs.getFileStatus(f);
        Assert.assertEquals(REPLICATION, status.getReplication());
        Assert.assertEquals(bytes.length, status.getLen());
      }

      {
        LOG.info("append another " + bytes.length + " bytes to " + f);
        try {
          final FSDataOutputStream out = fs.append(f);
          out.write(bytes);
          out.close();

          Assert.fail();
        } catch (IOException ioe) {
          LOG.info("This exception is expected", ioe);
        }
      }
    } finally {
      if (cluster != null) {
        cluster.shutdown();
      }
    }
  }
  /**
   * TC12: Append to partial CRC chunk
   *
   * @throws IOException an exception might be thrown
   */
  public void testTC12() throws Exception {
    final Path p = new Path("/TC12/foo");
    System.out.println("p=" + p);

    // a. Create file with a block size of 64KB
    //   and a default io.bytes.per.checksum of 512 bytes.
    //   Write 25687 bytes of data. Close file.
    final int len1 = 25687;
    {
      FSDataOutputStream out = fs.create(p, false, buffersize, REPLICATION, BLOCK_SIZE);
      AppendTestUtil.write(out, 0, len1);
      out.close();
    }

    // b. Reopen file in "append" mode. Append another 5877 bytes of data. Close file.
    final int len2 = 5877;
    {
      FSDataOutputStream out = fs.append(p);
      AppendTestUtil.write(out, len1, len2);
      out.close();
    }

    // c. Reopen file and read 25687+5877 bytes of data from file. Close file.
    AppendTestUtil.check(fs, p, len1 + len2);
  }
  /**
   * TC5: Only one simultaneous append.
   *
   * @throws IOException an exception might be thrown
   */
  public void testTC5() throws Exception {
    final Path p = new Path("/TC5/foo");
    System.out.println("p=" + p);

    // a. Create file on Machine M1. Write half block to it. Close file.
    {
      FSDataOutputStream out = fs.create(p, false, buffersize, REPLICATION, BLOCK_SIZE);
      AppendTestUtil.write(out, 0, (int) (BLOCK_SIZE / 2));
      out.close();
    }

    // b. Reopen file in "append" mode on Machine M1.
    FSDataOutputStream out = fs.append(p);

    // c. On Machine M2, reopen file in "append" mode. This should fail.
    try {
      AppendTestUtil.createHdfsWithDifferentUsername(conf).append(p);
      fail("This should fail.");
    } catch (IOException ioe) {
      AppendTestUtil.LOG.info("GOOD: got an exception", ioe);
    }

    // d. On Machine M1, close file.
    out.close();
  }
  /**
   * TC2: Append on non-block boundary.
   *
   * @throws IOException an exception might be thrown
   */
  public void testTC2() throws Exception {
    final Path p = new Path("/TC2/foo");
    System.out.println("p=" + p);

    // a. Create file with one and a half block of data. Close file.
    final int len1 = (int) (BLOCK_SIZE + BLOCK_SIZE / 2);
    {
      FSDataOutputStream out = fs.create(p, false, buffersize, REPLICATION, BLOCK_SIZE);
      AppendTestUtil.write(out, 0, len1);
      out.close();
    }

    AppendTestUtil.check(fs, p, len1);

    //   Reopen file to append quarter block of data. Close file.
    final int len2 = (int) BLOCK_SIZE / 4;
    {
      FSDataOutputStream out = fs.append(p);
      AppendTestUtil.write(out, len1, len2);
      out.close();
    }

    // b. Reopen file and read 1.75 blocks of data. Close file.
    AppendTestUtil.check(fs, p, len1 + len2);
  }
  /** Test the updation of NeededReplications for the Appended Block */
  @Test(timeout = 60000)
  public void testUpdateNeededReplicationsForAppendedFile() throws Exception {
    Configuration conf = new Configuration();
    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
    DistributedFileSystem fileSystem = null;
    try {
      // create a file.
      fileSystem = cluster.getFileSystem();
      Path f = new Path("/testAppend");
      FSDataOutputStream create = fileSystem.create(f, (short) 2);
      create.write("/testAppend".getBytes());
      create.close();

      // Append to the file.
      FSDataOutputStream append = fileSystem.append(f);
      append.write("/testAppend".getBytes());
      append.close();

      // Start a new datanode
      cluster.startDataNodes(conf, 1, true, null, null);

      // Check for replications
      DFSTestUtil.waitReplication(fileSystem, f, (short) 2);
    } finally {
      if (null != fileSystem) {
        fileSystem.close();
      }
      cluster.shutdown();
    }
  }
  /**
   * 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();
    }
  }
  /**
   * TC11: Racing rename
   *
   * @throws IOException an exception might be thrown
   */
  public void testTC11() throws Exception {
    final Path p = new Path("/TC11/foo");
    System.out.println("p=" + p);

    // a. Create file and write one block of data. Close file.
    final int len1 = (int) BLOCK_SIZE;
    {
      FSDataOutputStream out = fs.create(p, false, buffersize, REPLICATION, BLOCK_SIZE);
      AppendTestUtil.write(out, 0, len1);
      out.close();
    }

    // b. Reopen file in "append" mode. Append half block of data.
    FSDataOutputStream out = fs.append(p);
    final int len2 = (int) BLOCK_SIZE / 2;
    AppendTestUtil.write(out, len1, len2);
    out.hflush();

    // c. Rename file to file.new.
    final Path pnew = new Path(p + ".new");
    assertTrue(fs.rename(p, pnew));

    // d. Close file handle that was opened in (b).
    try {
      out.close();
      fail("close() should throw an exception");
    } catch (Exception e) {
      AppendTestUtil.LOG.info("GOOD!", e);
    }

    // wait for the lease recovery
    cluster.setLeasePeriod(1000, 1000);
    AppendTestUtil.sleep(5000);

    // check block sizes
    final long len = fs.getFileStatus(pnew).getLen();
    final LocatedBlocks locatedblocks =
        fs.dfs.getNamenode().getBlockLocations(pnew.toString(), 0L, len);
    final int numblock = locatedblocks.locatedBlockCount();
    for (int i = 0; i < numblock; i++) {
      final LocatedBlock lb = locatedblocks.get(i);
      final Block blk = lb.getBlock();
      final long size = lb.getBlockSize();
      if (i < numblock - 1) {
        assertEquals(BLOCK_SIZE, size);
      }
      for (DatanodeInfo datanodeinfo : lb.getLocations()) {
        final DataNode dn = cluster.getDataNode(datanodeinfo.getIpcPort());
        final Block metainfo = dn.data.getStoredBlock(blk.getBlockId());
        assertEquals(size, metainfo.getNumBytes());
      }
    }
  }
  /**
   * TC7: Corrupted replicas are present.
   *
   * @throws IOException an exception might be thrown
   */
  public void testTC7() throws Exception {
    final short repl = 2;
    final Path p = new Path("/TC7/foo");
    System.out.println("p=" + p);

    // a. Create file with replication factor of 2. Write half block of data. Close file.
    final int len1 = (int) (BLOCK_SIZE / 2);
    {
      FSDataOutputStream out = fs.create(p, false, buffersize, repl, BLOCK_SIZE);
      AppendTestUtil.write(out, 0, len1);
      out.close();
    }
    DFSTestUtil.waitReplication(fs, p, repl);

    // b. Log into one datanode that has one replica of this block.
    //   Find the block file on this datanode and truncate it to zero size.
    final LocatedBlocks locatedblocks =
        fs.dfs.getNamenode().getBlockLocations(p.toString(), 0L, len1);
    assertEquals(1, locatedblocks.locatedBlockCount());
    final LocatedBlock lb = locatedblocks.get(0);
    final Block blk = lb.getBlock();
    assertEquals(len1, lb.getBlockSize());

    DatanodeInfo[] datanodeinfos = lb.getLocations();
    assertEquals(repl, datanodeinfos.length);
    final DataNode dn = cluster.getDataNode(datanodeinfos[0].getIpcPort());
    final FSDataset data = (FSDataset) dn.getFSDataset();
    final RandomAccessFile raf = new RandomAccessFile(data.getBlockFile(blk), "rw");
    AppendTestUtil.LOG.info("dn=" + dn + ", blk=" + blk + " (length=" + blk.getNumBytes() + ")");
    assertEquals(len1, raf.length());
    raf.setLength(0);
    raf.close();

    // c. Open file in "append mode".  Append a new block worth of data. Close file.
    final int len2 = (int) BLOCK_SIZE;
    {
      FSDataOutputStream out = fs.append(p);
      AppendTestUtil.write(out, len1, len2);
      out.close();
    }

    // d. Reopen file and read two blocks worth of data.
    AppendTestUtil.check(fs, p, len1 + len2);
  }
  @Test
  public void testBestEffort() throws Exception {
    final Configuration conf = new HdfsConfiguration();

    // always replace a datanode but do not throw exception
    ReplaceDatanodeOnFailure.write(Policy.ALWAYS, true, conf);

    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();

    try {
      final DistributedFileSystem fs = cluster.getFileSystem();
      final Path f = new Path(DIR, "testIgnoreReplaceFailure");

      final byte[] bytes = new byte[1000];
      {
        LOG.info("write " + bytes.length + " bytes to " + f);
        final FSDataOutputStream out = fs.create(f, REPLICATION);
        out.write(bytes);
        out.close();

        final FileStatus status = fs.getFileStatus(f);
        Assert.assertEquals(REPLICATION, status.getReplication());
        Assert.assertEquals(bytes.length, status.getLen());
      }

      {
        LOG.info("append another " + bytes.length + " bytes to " + f);
        final FSDataOutputStream out = fs.append(f);
        out.write(bytes);
        out.close();
      }
    } finally {
      if (cluster != null) {
        cluster.shutdown();
      }
    }
  }