/**
   * Verify that the NameNode is able to still use <tt>READ_ONLY_SHARED</tt> replicas even when the
   * single NORMAL replica is offline (and the effective replication count is 0).
   */
  @Test
  public void testNormalReplicaOffline() throws Exception {
    // Stop the datanode hosting the NORMAL replica
    cluster.stopDataNode(normalDataNode.getXferAddr());

    // Force NameNode to detect that the datanode is down
    BlockManagerTestUtil.noticeDeadDatanode(cluster.getNameNode(), normalDataNode.getXferAddr());

    // The live replica count should now be zero (since the NORMAL replica is offline)
    NumberReplicas numberReplicas = blockManager.countNodes(block);
    assertThat(numberReplicas.liveReplicas(), is(0));

    // The block should be reported as under-replicated
    BlockManagerTestUtil.updateState(blockManager);
    assertThat(blockManager.getUnderReplicatedBlocksCount(), is(1L));

    // The BlockManager should be able to heal the replication count back to 1
    // by triggering an inter-datanode replication from one of the READ_ONLY_SHARED replicas
    BlockManagerTestUtil.computeAllPendingWork(blockManager);

    DFSTestUtil.waitForReplication(cluster, extendedBlock, 1, 1, 0);

    // There should now be 2 *locations* for the block, and 1 *replica*
    assertThat(getLocatedBlock().getLocations().length, is(2));
    validateNumberReplicas(1);
  }
  /**
   * Regression test for HDFS-2742. The issue in this bug was: - DN does a block report while file
   * is open. This BR contains the block in RBW state. - Standby queues the RBW state in
   * PendingDatanodeMessages - Standby processes edit logs during failover. Before fixing this bug,
   * it was mistakenly applying the RBW reported state after the block had been completed, causing
   * the block to get marked corrupt. Instead, we should now be applying the RBW message on OP_ADD,
   * and then the FINALIZED message on OP_CLOSE.
   */
  @Test
  public void testBlockReportsWhileFileBeingWritten() throws Exception {
    FSDataOutputStream out = fs.create(TEST_FILE_PATH);
    try {
      AppendTestUtil.write(out, 0, 10);
      out.hflush();

      // Block report will include the RBW replica, but will be
      // queued on the StandbyNode.
      cluster.triggerBlockReports();

    } finally {
      IOUtils.closeStream(out);
    }

    cluster.transitionToStandby(0);
    cluster.transitionToActive(1);

    // Verify that no replicas are marked corrupt, and that the
    // file is readable from the failed-over standby.
    BlockManagerTestUtil.updateState(nn1.getNamesystem().getBlockManager());
    BlockManagerTestUtil.updateState(nn2.getNamesystem().getBlockManager());
    assertEquals(0, nn1.getNamesystem().getCorruptReplicaBlocks());
    assertEquals(0, nn2.getNamesystem().getCorruptReplicaBlocks());

    DFSTestUtil.readFile(fs, TEST_FILE_PATH);
  }
  @Test
  public void testDnFencing() throws Exception {
    // Create a file with replication level 3.
    DFSTestUtil.createFile(fs, TEST_FILE_PATH, 30 * SMALL_BLOCK, (short) 3, 1L);
    ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, TEST_FILE_PATH);

    // Drop its replication count to 1, so it becomes over-replicated.
    // Then compute the invalidation of the extra blocks and trigger
    // heartbeats so the invalidations are flushed to the DNs.
    nn1.getRpcServer().setReplication(TEST_FILE, (short) 1);
    BlockManagerTestUtil.computeInvalidationWork(nn1.getNamesystem().getBlockManager());
    cluster.triggerHeartbeats();

    // Transition nn2 to active even though nn1 still thinks it's active.
    banner("Failing to NN2 but let NN1 continue to think it's active");
    NameNodeAdapter.abortEditLogs(nn1);
    NameNodeAdapter.enterSafeMode(nn1, false);
    cluster.transitionToActive(1);

    // Check that the standby picked up the replication change.
    assertEquals(1, nn2.getRpcServer().getFileInfo(TEST_FILE).getReplication());

    // Dump some info for debugging purposes.
    banner("NN2 Metadata immediately after failover");
    doMetasave(nn2);

    // Even though NN2 considers the blocks over-replicated, it should
    // post-pone the block invalidation because the DNs are still "stale".
    assertEquals(30, nn2.getNamesystem().getPostponedMisreplicatedBlocks());

    banner("Triggering heartbeats and block reports so that fencing is completed");
    cluster.triggerHeartbeats();
    cluster.triggerBlockReports();

    banner("Metadata after nodes have all block-reported");
    doMetasave(nn2);

    // The blocks should no longer be postponed.
    assertEquals(0, nn2.getNamesystem().getPostponedMisreplicatedBlocks());

    // Wait for NN2 to enact its deletions (replication monitor has to run, etc)
    BlockManagerTestUtil.computeInvalidationWork(nn2.getNamesystem().getBlockManager());
    cluster.triggerHeartbeats();
    HATestUtil.waitForDNDeletions(cluster);
    cluster.triggerDeletionReports();
    assertEquals(0, nn2.getNamesystem().getUnderReplicatedBlocks());
    assertEquals(0, nn2.getNamesystem().getPendingReplicationBlocks());

    banner("Making sure the file is still readable");
    FileSystem fs2 = cluster.getFileSystem(1);
    DFSTestUtil.readFile(fs2, TEST_FILE_PATH);

    banner("Waiting for the actual block files to get deleted from DNs.");
    waitForTrueReplication(cluster, block, 1);
  }
  /**
   * Another regression test for HDFS-2742. This tests the following sequence: - DN does a block
   * report while file is open. This BR contains the block in RBW state. - The block report is
   * delayed in reaching the standby. - The file is closed. - The standby processes the OP_ADD and
   * OP_CLOSE operations before the RBW block report arrives. - The standby should not mark the
   * block as corrupt.
   */
  @Test
  public void testRBWReportArrivesAfterEdits() throws Exception {
    final CountDownLatch brFinished = new CountDownLatch(1);
    DelayAnswer delayer =
        new GenericTestUtils.DelayAnswer(LOG) {
          @Override
          protected Object passThrough(InvocationOnMock invocation) throws Throwable {
            try {
              return super.passThrough(invocation);
            } finally {
              // inform the test that our block report went through.
              brFinished.countDown();
            }
          }
        };

    FSDataOutputStream out = fs.create(TEST_FILE_PATH);
    try {
      AppendTestUtil.write(out, 0, 10);
      out.hflush();

      DataNode dn = cluster.getDataNodes().get(0);
      DatanodeProtocolClientSideTranslatorPB spy = DataNodeTestUtils.spyOnBposToNN(dn, nn2);

      Mockito.doAnswer(delayer)
          .when(spy)
          .blockReport(
              Mockito.<DatanodeRegistration>anyObject(),
              Mockito.anyString(),
              Mockito.<StorageBlockReport[]>anyObject());
      dn.scheduleAllBlockReport(0);
      delayer.waitForCall();

    } finally {
      IOUtils.closeStream(out);
    }

    cluster.transitionToStandby(0);
    cluster.transitionToActive(1);

    delayer.proceed();
    brFinished.await();

    // Verify that no replicas are marked corrupt, and that the
    // file is readable from the failed-over standby.
    BlockManagerTestUtil.updateState(nn1.getNamesystem().getBlockManager());
    BlockManagerTestUtil.updateState(nn2.getNamesystem().getBlockManager());
    assertEquals(0, nn1.getNamesystem().getCorruptReplicaBlocks());
    assertEquals(0, nn2.getNamesystem().getCorruptReplicaBlocks());

    DFSTestUtil.readFile(fs, TEST_FILE_PATH);
  }
  /**
   * Test that, when a block is re-opened for append, the related datanode messages are correctly
   * queued by the SBN because they have future states and genstamps.
   */
  @Test
  public void testQueueingWithAppend() throws Exception {
    int numQueued = 0;
    int numDN = cluster.getDataNodes().size();

    FSDataOutputStream out = fs.create(TEST_FILE_PATH);
    try {
      AppendTestUtil.write(out, 0, 10);
      out.hflush();

      // Opening the file will report RBW replicas, but will be
      // queued on the StandbyNode.
      numQueued += numDN; // RBW messages
    } finally {
      IOUtils.closeStream(out);
      numQueued += numDN; // blockReceived messages
    }

    cluster.triggerBlockReports();
    numQueued += numDN;

    try {
      out = fs.append(TEST_FILE_PATH);
      AppendTestUtil.write(out, 10, 10);
      // RBW replicas once it's opened for append
      numQueued += numDN;

    } finally {
      IOUtils.closeStream(out);
      numQueued += numDN; // blockReceived
    }

    cluster.triggerBlockReports();
    numQueued += numDN;

    assertEquals(
        numQueued, cluster.getNameNode(1).getNamesystem().getPendingDataNodeMessageCount());

    cluster.transitionToStandby(0);
    cluster.transitionToActive(1);

    // Verify that no replicas are marked corrupt, and that the
    // file is readable from the failed-over standby.
    BlockManagerTestUtil.updateState(nn1.getNamesystem().getBlockManager());
    BlockManagerTestUtil.updateState(nn2.getNamesystem().getBlockManager());
    assertEquals(0, nn1.getNamesystem().getCorruptReplicaBlocks());
    assertEquals(0, nn2.getNamesystem().getCorruptReplicaBlocks());

    AppendTestUtil.check(fs, TEST_FILE_PATH, 20);
  }
  /** Test that there are under replication blocks after vol failures */
  @Test
  public void testUnderReplicationAfterVolFailure() throws Exception {
    // The test uses DataNodeTestUtils#injectDataDirFailure() to simulate
    // volume failures which is currently not supported on Windows.
    assumeTrue(!Path.WINDOWS);

    // Bring up one more datanode
    cluster.startDataNodes(conf, 1, true, null, null);
    cluster.waitActive();

    final BlockManager bm = cluster.getNamesystem().getBlockManager();

    Path file1 = new Path("/test1");
    DFSTestUtil.createFile(fs, file1, 1024, (short) 3, 1L);
    DFSTestUtil.waitReplication(fs, file1, (short) 3);

    // Fail the first volume on both datanodes
    File dn1Vol1 = new File(dataDir, "data" + (2 * 0 + 1));
    File dn2Vol1 = new File(dataDir, "data" + (2 * 1 + 1));
    DataNodeTestUtils.injectDataDirFailure(dn1Vol1, dn2Vol1);

    Path file2 = new Path("/test2");
    DFSTestUtil.createFile(fs, file2, 1024, (short) 3, 1L);
    DFSTestUtil.waitReplication(fs, file2, (short) 3);

    // underReplicatedBlocks are due to failed volumes
    int underReplicatedBlocks =
        BlockManagerTestUtil.checkHeartbeatAndGetUnderReplicatedBlocksCount(
            cluster.getNamesystem(), bm);
    assertTrue(
        "There is no under replicated block after volume failure", underReplicatedBlocks > 0);
  }
  /**
   * Setup a {@link MiniDFSCluster}. Create a block with both {@link State#NORMAL} and {@link
   * State#READ_ONLY_SHARED} replicas.
   */
  @Before
  public void setup() throws IOException, InterruptedException {
    conf = new HdfsConfiguration();
    SimulatedFSDataset.setFactory(conf);

    Configuration[] overlays = new Configuration[NUM_DATANODES];
    for (int i = 0; i < overlays.length; i++) {
      overlays[i] = new Configuration();
      if (i == RO_NODE_INDEX) {
        overlays[i].setEnum(
            SimulatedFSDataset.CONFIG_PROPERTY_STATE,
            i == RO_NODE_INDEX ? READ_ONLY_SHARED : NORMAL);
      }
    }

    cluster =
        new MiniDFSCluster.Builder(conf)
            .numDataNodes(NUM_DATANODES)
            .dataNodeConfOverlays(overlays)
            .build();
    fs = cluster.getFileSystem();
    blockManager = cluster.getNameNode().getNamesystem().getBlockManager();
    datanodeManager = blockManager.getDatanodeManager();
    client =
        new DFSClient(
            new InetSocketAddress("localhost", cluster.getNameNodePort()),
            cluster.getConfiguration(0));

    for (int i = 0; i < NUM_DATANODES; i++) {
      DataNode dataNode = cluster.getDataNodes().get(i);
      validateStorageState(
          BlockManagerTestUtil.getStorageReportsForDatanode(
              datanodeManager.getDatanode(dataNode.getDatanodeId())),
          i == RO_NODE_INDEX ? READ_ONLY_SHARED : NORMAL);
    }

    // Create a 1 block file
    DFSTestUtil.createFile(fs, PATH, BLOCK_SIZE, BLOCK_SIZE, BLOCK_SIZE, (short) 1, seed);

    LocatedBlock locatedBlock = getLocatedBlock();
    extendedBlock = locatedBlock.getBlock();
    block = extendedBlock.getLocalBlock();

    assertThat(locatedBlock.getLocations().length, is(1));
    normalDataNode = locatedBlock.getLocations()[0];
    readOnlyDataNode =
        datanodeManager.getDatanode(cluster.getDataNodes().get(RO_NODE_INDEX).getDatanodeId());
    assertThat(normalDataNode, is(not(readOnlyDataNode)));

    validateNumberReplicas(1);

    // Inject the block into the datanode with READ_ONLY_SHARED storage
    cluster.injectBlocks(0, RO_NODE_INDEX, Collections.singleton(block));

    // There should now be 2 *locations* for the block
    // Must wait until the NameNode has processed the block report for the injected blocks
    waitForLocations(2);
  }
  private void validateNumberReplicas(int expectedReplicas) throws IOException {
    NumberReplicas numberReplicas = blockManager.countNodes(block);
    assertThat(numberReplicas.liveReplicas(), is(expectedReplicas));
    assertThat(numberReplicas.excessReplicas(), is(0));
    assertThat(numberReplicas.corruptReplicas(), is(0));
    assertThat(numberReplicas.decommissionedReplicas(), is(0));
    assertThat(numberReplicas.replicasOnStaleNodes(), is(0));

    BlockManagerTestUtil.updateState(blockManager);
    assertThat(blockManager.getUnderReplicatedBlocksCount(), is(0L));
    assertThat(blockManager.getExcessBlocksCount(), is(0L));
  }
  /**
   * verifies two things: 1. number of locations of each block in the name node matches number of
   * actual files 2. block files + pending block equals to total number of blocks that a file has
   * including the replication (HDFS file has 30 blocks, repl=2 - total 60
   *
   * @param fn - file name
   * @param fs - file size
   * @throws IOException
   */
  private void verify(String fn, int fs) throws IOException {
    // now count how many physical blocks are there
    int totalReal = countRealBlocks(block_map);
    System.out.println("countRealBlocks counted " + totalReal + " blocks");

    // count how many blocks store in NN structures.
    int totalNN = countNNBlocks(block_map, fn, fs);
    System.out.println("countNNBlocks counted " + totalNN + " blocks");

    for (String bid : block_map.keySet()) {
      BlockLocs bl = block_map.get(bid);
      // System.out.println(bid + "->" + bl.num_files + "vs." + bl.num_locs);
      // number of physical files (1 or 2) should be same as number of datanodes
      // in the list of the block locations
      assertEquals("Num files should match num locations", bl.num_files, bl.num_locs);
    }
    assertEquals("Num physical blocks should match num stored in the NN", totalReal, totalNN);

    // now check the number of under-replicated blocks
    FSNamesystem fsn = cluster.getNamesystem();
    // force update of all the metric counts by calling computeDatanodeWork
    BlockManagerTestUtil.getComputedDatanodeWork(fsn.getBlockManager());
    // get all the counts
    long underRepl = fsn.getUnderReplicatedBlocks();
    long pendRepl = fsn.getPendingReplicationBlocks();
    long totalRepl = underRepl + pendRepl;
    System.out.println(
        "underreplicated after = "
            + underRepl
            + " and pending repl ="
            + pendRepl
            + "; total underRepl = "
            + totalRepl);

    System.out.println(
        "total blocks (real and replicating):"
            + (totalReal + totalRepl)
            + " vs. all files blocks "
            + blocks_num * 2);

    // together all the blocks should be equal to all real + all underreplicated
    assertEquals("Incorrect total block count", totalReal + totalRepl, blocks_num * repl);
  }
  @BeforeClass
  public static void setupCluster() throws IOException {
    Configuration conf = new HdfsConfiguration();
    final String[] racks = {"/rack1", "/rack1", "/rack1", "/rack2", "/rack2", "/rack2"};
    storages = DFSTestUtil.createDatanodeStorageInfos(racks);
    dataNodes = DFSTestUtil.toDatanodeDescriptor(storages);
    FileSystem.setDefaultUri(conf, "hdfs://localhost:0");
    conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "0.0.0.0:0");
    File baseDir = PathUtils.getTestDir(TestReplicationPolicy.class);
    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, new File(baseDir, "name").getPath());
    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_READ_KEY, true);
    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_WRITE_KEY, true);
    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, true);
    DFSTestUtil.formatNameNode(conf);
    namenode = new NameNode(conf);
    int blockSize = 1024;

    dnrList = new ArrayList<DatanodeRegistration>();
    dnManager = namenode.getNamesystem().getBlockManager().getDatanodeManager();

    // Register DNs
    for (int i = 0; i < 6; i++) {
      DatanodeRegistration dnr =
          new DatanodeRegistration(
              dataNodes[i],
              new StorageInfo(NodeType.DATA_NODE),
              new ExportedBlockKeys(),
              VersionInfo.getVersion());
      dnrList.add(dnr);
      dnManager.registerDatanode(dnr);
      dataNodes[i].getStorageInfos()[0].setUtilizationForTesting(
          2 * HdfsServerConstants.MIN_BLOCKS_FOR_WRITE * blockSize, 0L,
          2 * HdfsServerConstants.MIN_BLOCKS_FOR_WRITE * blockSize, 0L);
      dataNodes[i].updateHeartbeat(
          BlockManagerTestUtil.getStorageReportsForDatanode(dataNodes[i]), 0L, 0L, 0, 0, null);
    }
  }
  /*
   * Wait up to 20s for the given block to be replicated across
   * the requested number of racks, with the requested number of
   * replicas, and the requested number of replicas still needed.
   */
  public static void waitForReplication(
      MiniDFSCluster cluster, ExtendedBlock b, int racks, int replicas, int neededReplicas)
      throws IOException, TimeoutException, InterruptedException {
    int curRacks = 0;
    int curReplicas = 0;
    int curNeededReplicas = 0;
    int count = 0;
    final int ATTEMPTS = 20;

    do {
      Thread.sleep(1000);
      int[] r = BlockManagerTestUtil.getReplicaInfo(cluster.getNamesystem(), b.getLocalBlock());
      curRacks = r[0];
      curReplicas = r[1];
      curNeededReplicas = r[2];
      count++;
    } while ((curRacks != racks || curReplicas != replicas || curNeededReplicas != neededReplicas)
        && count < ATTEMPTS);

    if (count == ATTEMPTS) {
      throw new TimeoutException(
          "Timed out waiting for replication."
              + " Needed replicas = "
              + neededReplicas
              + " Cur needed replicas = "
              + curNeededReplicas
              + " Replicas = "
              + replicas
              + " Cur replicas = "
              + curReplicas
              + " Racks = "
              + racks
              + " Cur racks = "
              + curRacks);
    }
  }
  /**
   * Test case that reduces replication of a file with a lot of blocks and then fails over right
   * after those blocks enter the DN invalidation queues on the active. Ensures that fencing is
   * correct and no replicas are lost.
   */
  @Test
  public void testNNClearsCommandsOnFailoverWithReplChanges() throws Exception {
    // Make lots of blocks to increase chances of triggering a bug.
    DFSTestUtil.createFile(fs, TEST_FILE_PATH, 30 * SMALL_BLOCK, (short) 1, 1L);

    banner("rolling NN1's edit log, forcing catch-up");
    HATestUtil.waitForStandbyToCatchUp(nn1, nn2);

    // Get some new replicas reported so that NN2 now considers
    // them over-replicated and schedules some more deletions
    nn1.getRpcServer().setReplication(TEST_FILE, (short) 2);
    while (BlockManagerTestUtil.getComputedDatanodeWork(nn1.getNamesystem().getBlockManager())
        > 0) {
      LOG.info("Getting more replication work computed");
    }
    BlockManager bm1 = nn1.getNamesystem().getBlockManager();
    while (bm1.getPendingReplicationBlocksCount() > 0) {
      BlockManagerTestUtil.updateState(bm1);
      cluster.triggerHeartbeats();
      Thread.sleep(1000);
    }

    banner("triggering BRs");
    cluster.triggerBlockReports();

    nn1.getRpcServer().setReplication(TEST_FILE, (short) 1);

    banner("computing invalidation on nn1");

    BlockManagerTestUtil.computeInvalidationWork(nn1.getNamesystem().getBlockManager());
    doMetasave(nn1);

    banner("computing invalidation on nn2");
    BlockManagerTestUtil.computeInvalidationWork(nn2.getNamesystem().getBlockManager());
    doMetasave(nn2);

    // Dump some info for debugging purposes.
    banner("Metadata immediately before failover");
    doMetasave(nn2);

    // Transition nn2 to active even though nn1 still thinks it's active
    banner("Failing to NN2 but let NN1 continue to think it's active");
    NameNodeAdapter.abortEditLogs(nn1);
    NameNodeAdapter.enterSafeMode(nn1, false);

    BlockManagerTestUtil.computeInvalidationWork(nn2.getNamesystem().getBlockManager());
    cluster.transitionToActive(1);

    // Check that the standby picked up the replication change.
    assertEquals(1, nn2.getRpcServer().getFileInfo(TEST_FILE).getReplication());

    // Dump some info for debugging purposes.
    banner("Metadata immediately after failover");
    doMetasave(nn2);

    banner("Triggering heartbeats and block reports so that fencing is completed");
    cluster.triggerHeartbeats();
    cluster.triggerBlockReports();

    banner("Metadata after nodes have all block-reported");
    doMetasave(nn2);

    // The block should no longer be postponed.
    assertEquals(0, nn2.getNamesystem().getPostponedMisreplicatedBlocks());

    // Wait for NN2 to enact its deletions (replication monitor has to run, etc)
    BlockManagerTestUtil.computeInvalidationWork(nn2.getNamesystem().getBlockManager());

    HATestUtil.waitForNNToIssueDeletions(nn2);
    cluster.triggerHeartbeats();
    HATestUtil.waitForDNDeletions(cluster);
    cluster.triggerDeletionReports();
    assertEquals(0, nn2.getNamesystem().getUnderReplicatedBlocks());
    assertEquals(0, nn2.getNamesystem().getPendingReplicationBlocks());

    banner("Making sure the file is still readable");
    FileSystem fs2 = cluster.getFileSystem(1);
    DFSTestUtil.readFile(fs2, TEST_FILE_PATH);
  }
  /**
   * Test case which restarts the standby node in such a way that, when it exits safemode, it will
   * want to invalidate a bunch of over-replicated block replicas. Ensures that if we failover at
   * this point it won't lose data.
   */
  @Test
  public void testNNClearsCommandsOnFailoverAfterStartup() throws Exception {
    // Make lots of blocks to increase chances of triggering a bug.
    DFSTestUtil.createFile(fs, TEST_FILE_PATH, 30 * SMALL_BLOCK, (short) 3, 1L);

    banner("Shutting down NN2");
    cluster.shutdownNameNode(1);

    banner("Setting replication to 1, rolling edit log.");
    nn1.getRpcServer().setReplication(TEST_FILE, (short) 1);
    nn1.getRpcServer().rollEditLog();

    // Start NN2 again. When it starts up, it will see all of the
    // blocks as over-replicated, since it has the metadata for
    // replication=1, but the DNs haven't yet processed the deletions.
    banner("Starting NN2 again.");
    cluster.restartNameNode(1);
    nn2 = cluster.getNameNode(1);

    banner("triggering BRs");
    cluster.triggerBlockReports();

    // We expect that both NN1 and NN2 will have some number of
    // deletions queued up for the DNs.
    banner("computing invalidation on nn1");
    BlockManagerTestUtil.computeInvalidationWork(nn1.getNamesystem().getBlockManager());

    banner("computing invalidation on nn2");
    BlockManagerTestUtil.computeInvalidationWork(nn2.getNamesystem().getBlockManager());

    // Dump some info for debugging purposes.
    banner("Metadata immediately before failover");
    doMetasave(nn2);

    // Transition nn2 to active even though nn1 still thinks it's active
    banner("Failing to NN2 but let NN1 continue to think it's active");
    NameNodeAdapter.abortEditLogs(nn1);
    NameNodeAdapter.enterSafeMode(nn1, false);

    cluster.transitionToActive(1);

    // Check that the standby picked up the replication change.
    assertEquals(1, nn2.getRpcServer().getFileInfo(TEST_FILE).getReplication());

    // Dump some info for debugging purposes.
    banner("Metadata immediately after failover");
    doMetasave(nn2);

    banner("Triggering heartbeats and block reports so that fencing is completed");
    cluster.triggerHeartbeats();
    cluster.triggerBlockReports();

    banner("Metadata after nodes have all block-reported");
    doMetasave(nn2);

    // The block should no longer be postponed.
    assertEquals(0, nn2.getNamesystem().getPostponedMisreplicatedBlocks());

    // Wait for NN2 to enact its deletions (replication monitor has to run, etc)
    BlockManagerTestUtil.computeInvalidationWork(nn2.getNamesystem().getBlockManager());

    HATestUtil.waitForNNToIssueDeletions(nn2);
    cluster.triggerHeartbeats();
    HATestUtil.waitForDNDeletions(cluster);
    cluster.triggerDeletionReports();
    assertEquals(0, nn2.getNamesystem().getUnderReplicatedBlocks());
    assertEquals(0, nn2.getNamesystem().getPendingReplicationBlocks());

    banner("Making sure the file is still readable");
    FileSystem fs2 = cluster.getFileSystem(1);
    DFSTestUtil.readFile(fs2, TEST_FILE_PATH);
  }
示例#14
0
  @Test
  public void testSortNodeByFields() throws Exception {
    DatanodeID dnId1 =
        new DatanodeID("127.0.0.1", "localhost1", "datanode1", 1234, 2345, 3456, 4567);
    DatanodeID dnId2 =
        new DatanodeID("127.0.0.2", "localhost2", "datanode2", 1235, 2346, 3457, 4568);

    // Setup DatanodeDescriptors with one storage each.
    DatanodeDescriptor dnDesc1 = new DatanodeDescriptor(dnId1, "rack1");
    DatanodeDescriptor dnDesc2 = new DatanodeDescriptor(dnId2, "rack2");

    // Update the DatanodeDescriptors with their attached storages.
    BlockManagerTestUtil.updateStorage(dnDesc1, new DatanodeStorage("dnStorage1"));
    BlockManagerTestUtil.updateStorage(dnDesc2, new DatanodeStorage("dnStorage2"));

    DatanodeStorage dns1 = new DatanodeStorage("dnStorage1");
    DatanodeStorage dns2 = new DatanodeStorage("dnStorage2");

    StorageReport[] report1 =
        new StorageReport[] {new StorageReport(dns1, false, 1024, 100, 924, 100)};
    StorageReport[] report2 =
        new StorageReport[] {new StorageReport(dns2, false, 2500, 200, 1848, 200)};
    dnDesc1.updateHeartbeat(report1, 5L, 3L, 10, 2);
    dnDesc2.updateHeartbeat(report2, 10L, 2L, 20, 1);

    ArrayList<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
    live.add(dnDesc1);
    live.add(dnDesc2);

    JspHelper.sortNodeList(live, "unexists", "ASC");
    Assert.assertEquals(dnDesc1, live.get(0));
    Assert.assertEquals(dnDesc2, live.get(1));
    JspHelper.sortNodeList(live, "unexists", "DSC");
    Assert.assertEquals(dnDesc2, live.get(0));
    Assert.assertEquals(dnDesc1, live.get(1));

    // test sorting by capacity
    JspHelper.sortNodeList(live, "capacity", "ASC");
    Assert.assertEquals(dnDesc1, live.get(0));
    Assert.assertEquals(dnDesc2, live.get(1));
    JspHelper.sortNodeList(live, "capacity", "DSC");
    Assert.assertEquals(dnDesc2, live.get(0));
    Assert.assertEquals(dnDesc1, live.get(1));

    // test sorting by used
    JspHelper.sortNodeList(live, "used", "ASC");
    Assert.assertEquals(dnDesc1, live.get(0));
    Assert.assertEquals(dnDesc2, live.get(1));
    JspHelper.sortNodeList(live, "used", "DSC");
    Assert.assertEquals(dnDesc2, live.get(0));
    Assert.assertEquals(dnDesc1, live.get(1));

    // test sorting by nondfsused
    JspHelper.sortNodeList(live, "nondfsused", "ASC");
    Assert.assertEquals(dnDesc1, live.get(0));
    Assert.assertEquals(dnDesc2, live.get(1));

    JspHelper.sortNodeList(live, "nondfsused", "DSC");
    Assert.assertEquals(dnDesc2, live.get(0));
    Assert.assertEquals(dnDesc1, live.get(1));

    // test sorting by remaining
    JspHelper.sortNodeList(live, "remaining", "ASC");
    Assert.assertEquals(dnDesc1, live.get(0));
    Assert.assertEquals(dnDesc2, live.get(1));

    JspHelper.sortNodeList(live, "remaining", "DSC");
    Assert.assertEquals(dnDesc2, live.get(0));
    Assert.assertEquals(dnDesc1, live.get(1));
  }
  /**
   * Tests that chooseTarget with considerLoad set to true correctly calculates load with
   * decommissioned nodes.
   */
  @Test
  public void testChooseTargetWithDecomNodes() throws IOException {
    namenode.getNamesystem().writeLock();
    try {
      String blockPoolId = namenode.getNamesystem().getBlockPoolId();
      dnManager.handleHeartbeat(
          dnrList.get(3),
          BlockManagerTestUtil.getStorageReportsForDatanode(dataNodes[3]),
          blockPoolId,
          dataNodes[3].getCacheCapacity(),
          dataNodes[3].getCacheRemaining(),
          2,
          0,
          0,
          null);
      dnManager.handleHeartbeat(
          dnrList.get(4),
          BlockManagerTestUtil.getStorageReportsForDatanode(dataNodes[4]),
          blockPoolId,
          dataNodes[4].getCacheCapacity(),
          dataNodes[4].getCacheRemaining(),
          4,
          0,
          0,
          null);
      dnManager.handleHeartbeat(
          dnrList.get(5),
          BlockManagerTestUtil.getStorageReportsForDatanode(dataNodes[5]),
          blockPoolId,
          dataNodes[5].getCacheCapacity(),
          dataNodes[5].getCacheRemaining(),
          4,
          0,
          0,
          null);
      // value in the above heartbeats
      final int load = 2 + 4 + 4;

      FSNamesystem fsn = namenode.getNamesystem();
      assertEquals(
          (double) load / 6, dnManager.getFSClusterStats().getInServiceXceiverAverage(), EPSILON);

      // Decommission DNs so BlockPlacementPolicyDefault.isGoodTarget()
      // returns false
      for (int i = 0; i < 3; i++) {
        DatanodeDescriptor d = dnManager.getDatanode(dnrList.get(i));
        dnManager.getDecomManager().startDecommission(d);
        d.setDecommissioned();
      }
      assertEquals(
          (double) load / 3, dnManager.getFSClusterStats().getInServiceXceiverAverage(), EPSILON);

      // update references of writer DN to update the de-commissioned state
      List<DatanodeDescriptor> liveNodes = new ArrayList<DatanodeDescriptor>();
      dnManager.fetchDatanodes(liveNodes, null, false);
      DatanodeDescriptor writerDn = null;
      if (liveNodes.contains(dataNodes[0])) {
        writerDn = liveNodes.get(liveNodes.indexOf(dataNodes[0]));
      }

      // Call chooseTarget()
      DatanodeStorageInfo[] targets =
          namenode
              .getNamesystem()
              .getBlockManager()
              .getBlockPlacementPolicy()
              .chooseTarget(
                  "testFile.txt",
                  3,
                  writerDn,
                  new ArrayList<DatanodeStorageInfo>(),
                  false,
                  null,
                  1024,
                  TestBlockStoragePolicy.DEFAULT_STORAGE_POLICY);

      assertEquals(3, targets.length);
      Set<DatanodeStorageInfo> targetSet = new HashSet<DatanodeStorageInfo>(Arrays.asList(targets));
      for (int i = 3; i < storages.length; i++) {
        assertTrue(targetSet.contains(storages[i]));
      }
    } finally {
      dataNodes[0].stopDecommission();
      dataNodes[1].stopDecommission();
      dataNodes[2].stopDecommission();
      namenode.getNamesystem().writeUnlock();
    }
    NameNode.LOG.info("Done working on it");
  }