public void testRead() throws Exception {
    for (int i = 0; i < TEST_FILE_NUM; ++i) {
      String file = "/tmp" + i + ".txt";
      DFSTestUtil.createFile(fs, new Path(file), FILE_LEN, (short) 5, 1L);

      DFSDataInputStream in = (DFSDataInputStream) fs.open(new Path(file));
      int numOfRead = 0;
      while (in.read() > 0) {
        numOfRead++;
      }
      assertEquals(FILE_LEN * (i + 1), metrics.readSize.getCurrentIntervalValue());
      assertEquals(numOfRead * (i + 1), metrics.readOps.getCurrentIntervalValue());
    }
  }
  public void testBlockSynchronization() throws Exception {
    final long softLease = 1000;
    final long hardLease = 60 * 60 * 1000;
    final short repl = 3;
    final Configuration conf = new Configuration();
    final int bufferSize = conf.getInt("io.file.buffer.size", 4096);
    conf.setLong("dfs.block.size", BLOCK_SIZE);
    conf.setInt("dfs.heartbeat.interval", 1);
    //  conf.setInt("io.bytes.per.checksum", 16);

    MiniDFSCluster cluster = null;
    byte[] actual = new byte[FILE_SIZE];

    try {
      cluster = new MiniDFSCluster(conf, 5, true, null);
      cluster.waitActive();

      // create a file
      DistributedFileSystem dfs = (DistributedFileSystem) cluster.getFileSystem();
      // create a random file name
      String filestr = "/foo" + AppendTestUtil.nextInt();
      System.out.println("filestr=" + filestr);
      Path filepath = new Path(filestr);
      FSDataOutputStream stm = dfs.create(filepath, true, bufferSize, repl, BLOCK_SIZE);
      assertTrue(dfs.dfs.exists(filestr));

      // write random number of bytes into it.
      int size = AppendTestUtil.nextInt(FILE_SIZE);
      System.out.println("size=" + size);
      stm.write(buffer, 0, size);

      // sync file
      AppendTestUtil.LOG.info("sync");
      stm.sync();
      AppendTestUtil.LOG.info("leasechecker.interrupt()");
      dfs.dfs.leaseChecker.interrupt();

      // set the soft limit to be 1 second so that the
      // namenode triggers lease recovery on next attempt to write-for-open.
      cluster.setLeasePeriod(softLease, hardLease);

      // try to re-open the file before closing the previous handle. This
      // should fail but will trigger lease recovery.
      {
        Configuration conf2 = new Configuration(conf);
        String username = UserGroupInformation.getCurrentUGI().getUserName() + "_1";
        UnixUserGroupInformation.saveToConf(
            conf2,
            UnixUserGroupInformation.UGI_PROPERTY_NAME,
            new UnixUserGroupInformation(username, new String[] {"supergroup"}));
        FileSystem dfs2 = FileSystem.get(conf2);

        boolean done = false;
        for (int i = 0; i < 10 && !done; i++) {
          AppendTestUtil.LOG.info("i=" + i);
          try {
            dfs2.create(filepath, false, bufferSize, repl, BLOCK_SIZE);
            fail("Creation of an existing file should never succeed.");
          } catch (IOException ioe) {
            final String message = ioe.getMessage();
            if (message.contains("file exists")) {
              AppendTestUtil.LOG.info("done", ioe);
              done = true;
            } else if (message.contains(AlreadyBeingCreatedException.class.getSimpleName())) {
              AppendTestUtil.LOG.info("GOOD! got " + message);
            } else {
              AppendTestUtil.LOG.warn("UNEXPECTED IOException", ioe);
            }
          }

          if (!done) {
            AppendTestUtil.LOG.info("sleep " + 5000 + "ms");
            try {
              Thread.sleep(5000);
            } catch (InterruptedException e) {
            }
          }
        }
        assertTrue(done);
      }

      AppendTestUtil.LOG.info(
          "Lease for file " + filepath + " is recovered. " + "Validating its contents now...");

      // verify that file-size matches
      assertTrue(
          "File should be "
              + size
              + " bytes, but is actually "
              + " found to be "
              + dfs.getFileStatus(filepath).getLen()
              + " bytes",
          dfs.getFileStatus(filepath).getLen() == size);

      // verify that there is enough data to read.
      System.out.println("File size is good. Now validating sizes from datanodes...");
      FSDataInputStream stmin = dfs.open(filepath);
      stmin.readFully(0, actual, 0, size);
      stmin.close();
    } finally {
      try {
        if (cluster != null) {
          cluster.shutdown();
        }
      } catch (Exception e) {
        // ignore
      }
    }
  }
  /** Test replace datanode on failure. */
  @Test
  public void testReplaceDatanodeOnFailure() throws Exception {
    final Configuration conf = new HdfsConfiguration();
    // do not consider load factor when selecting a data node
    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, false);
    // always replace a datanode
    ReplaceDatanodeOnFailure.write(Policy.ALWAYS, true, conf);

    final String[] racks = new String[REPLICATION];
    Arrays.fill(racks, RACK0);
    final MiniDFSCluster cluster =
        new MiniDFSCluster.Builder(conf).racks(racks).numDataNodes(REPLICATION).build();

    try {
      cluster.waitActive();
      final DistributedFileSystem fs = cluster.getFileSystem();
      final Path dir = new Path(DIR);
      final int NUM_WRITERS = 10;
      final int FIRST_BATCH = 5;
      final SlowWriter[] slowwriters = new SlowWriter[NUM_WRITERS];
      for (int i = 1; i <= slowwriters.length; i++) {
        // create slow writers in different speed
        slowwriters[i - 1] = new SlowWriter(fs, new Path(dir, "file" + i), i * 200L);
      }

      for (int i = 0; i < FIRST_BATCH; i++) {
        slowwriters[i].start();
      }

      // Let slow writers write something.
      // Some of them are too slow and will be not yet started.
      sleepSeconds(3);

      // start new datanodes
      cluster.startDataNodes(conf, 2, true, null, new String[] {RACK1, RACK1});
      cluster.waitActive();
      // wait for first block reports for up to 10 seconds
      cluster.waitFirstBRCompleted(0, 10000);

      // stop an old datanode
      MiniDFSCluster.DataNodeProperties dnprop =
          cluster.stopDataNode(AppendTestUtil.nextInt(REPLICATION));

      for (int i = FIRST_BATCH; i < slowwriters.length; i++) {
        slowwriters[i].start();
      }

      waitForBlockReplication(slowwriters);

      // check replication and interrupt.
      for (SlowWriter s : slowwriters) {
        s.checkReplication();
        s.interruptRunning();
      }

      // close files
      for (SlowWriter s : slowwriters) {
        s.joinAndClose();
      }

      // Verify the file
      LOG.info("Verify the file");
      for (int i = 0; i < slowwriters.length; i++) {
        LOG.info(
            slowwriters[i].filepath
                + ": length="
                + fs.getFileStatus(slowwriters[i].filepath).getLen());
        FSDataInputStream in = null;
        try {
          in = fs.open(slowwriters[i].filepath);
          for (int j = 0, x; (x = in.read()) != -1; j++) {
            Assert.assertEquals(j, x);
          }
        } finally {
          IOUtils.closeStream(in);
        }
      }
    } finally {
      if (cluster != null) {
        cluster.shutdown();
      }
    }
  }