/**
   * 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);
  }
  @Before
  public void setupCluster() throws Exception {
    conf = new Configuration();
    conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, SMALL_BLOCK);
    // Bump up replication interval so that we only run replication
    // checks explicitly.
    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 600);
    // Increase max streams so that we re-replicate quickly.
    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 1000);
    // See RandomDeleterPolicy javadoc.
    conf.setClass(
        DFSConfigKeys.DFS_BLOCK_REPLICATOR_CLASSNAME_KEY,
        RandomDeleterPolicy.class,
        BlockPlacementPolicy.class);
    conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
    cluster =
        new MiniDFSCluster.Builder(conf)
            .nnTopology(MiniDFSNNTopology.simpleHATopology())
            .numDataNodes(3)
            .build();
    nn1 = cluster.getNameNode(0);
    nn2 = cluster.getNameNode(1);

    cluster.waitActive();
    cluster.transitionToActive(0);
    // Trigger block reports so that the first NN trusts all
    // of the DNs, and will issue deletions
    cluster.triggerBlockReports();
    fs = HATestUtil.configureFailoverFs(cluster, conf);
  }
  @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);
  }
  // Don't move this code to the parent class. There's a binary
  // incompatibility between hadoop 1 and 2 wrt MiniDFSCluster and we
  // need to have two different shim classes even though they are
  // exactly the same.
  @Override
  public HadoopShims.MiniDFSShim getMiniDfs(
      Configuration conf, int numDataNodes, boolean format, String[] racks, boolean isHA)
      throws IOException {
    configureImpersonation(conf);
    MiniDFSCluster miniDFSCluster;
    if (isHA) {
      MiniDFSNNTopology topo =
          new MiniDFSNNTopology()
              .addNameservice(
                  new MiniDFSNNTopology.NSConf("minidfs")
                      .addNN(new MiniDFSNNTopology.NNConf("nn1"))
                      .addNN(new MiniDFSNNTopology.NNConf("nn2")));
      miniDFSCluster =
          new MiniDFSCluster.Builder(conf)
              .numDataNodes(numDataNodes)
              .format(format)
              .racks(racks)
              .nnTopology(topo)
              .build();
      miniDFSCluster.waitActive();
      miniDFSCluster.transitionToActive(0);
    } else {
      miniDFSCluster =
          new MiniDFSCluster.Builder(conf)
              .numDataNodes(numDataNodes)
              .format(format)
              .racks(racks)
              .build();
    }

    // Need to set the client's KeyProvider to the NN's for JKS,
    // else the updates do not get flushed properly
    KeyProviderCryptoExtension keyProvider =
        miniDFSCluster.getNameNode(0).getNamesystem().getProvider();
    if (keyProvider != null) {
      try {
        setKeyProvider(miniDFSCluster.getFileSystem(0).getClient(), keyProvider);
      } catch (Exception err) {
        throw new IOException(err);
      }
    }

    cluster = new MiniDFSShim(miniDFSCluster);
    return cluster;
  }
  @Test
  public void testFencingStress() throws Exception {
    HAStressTestHarness harness = new HAStressTestHarness();
    harness.setNumberOfNameNodes(3);
    harness.conf.setInt(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 1000);
    harness.conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1);
    harness.conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1);

    final MiniDFSCluster cluster = harness.startCluster();
    try {
      cluster.waitActive();
      cluster.transitionToActive(0);

      FileSystem fs = harness.getFailoverFs();
      TestContext togglers = new TestContext();
      for (int i = 0; i < NUM_THREADS; i++) {
        Path p = new Path("/test-" + i);
        DFSTestUtil.createFile(fs, p, BLOCK_SIZE * 10, (short) 3, (long) i);
        togglers.addThread(new ReplicationToggler(togglers, fs, p));
      }

      // Start a separate thread which will make sure that replication
      // happens quickly by triggering deletion reports and replication
      // work calculation frequently.
      harness.addReplicationTriggerThread(500);
      harness.addFailoverThread(5000);
      harness.startThreads();
      togglers.startThreads();

      togglers.waitFor(RUNTIME);
      togglers.stop();
      harness.stopThreads();

      // CHeck that the files can be read without throwing
      for (int i = 0; i < NUM_THREADS; i++) {
        Path p = new Path("/test-" + i);
        DFSTestUtil.readFile(fs, p);
      }
    } finally {
      System.err.println("===========================\n\n\n\n");
      harness.shutdown();
    }
  }
  /**
   * 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);
  }