/**
   * Test whether we can delay the deletion of unknown blocks in DataNode's first several block
   * reports.
   */
  @Test
  public void testPendingDeleteUnknownBlocks() throws Exception {
    final int fileNum = 5; // 5 files
    final Path[] files = new Path[fileNum];
    final DataNodeProperties[] dnprops = new DataNodeProperties[REPLICATION];
    // create a group of files, each file contains 1 block
    for (int i = 0; i < fileNum; i++) {
      files[i] = new Path("/file" + i);
      DFSTestUtil.createFile(dfs, files[i], BLOCKSIZE, REPLICATION, i);
    }
    // wait until all DataNodes have replicas
    waitForReplication();
    for (int i = REPLICATION - 1; i >= 0; i--) {
      dnprops[i] = cluster.stopDataNode(i);
    }
    Thread.sleep(2000);
    // delete 2 files, we still have 3 files remaining so that we can cover
    // every DN storage
    for (int i = 0; i < 2; i++) {
      dfs.delete(files[i], true);
    }

    // restart NameNode
    cluster.restartNameNode(false);
    InvalidateBlocks invalidateBlocks =
        (InvalidateBlocks)
            Whitebox.getInternalState(
                cluster.getNamesystem().getBlockManager(), "invalidateBlocks");
    InvalidateBlocks mockIb = Mockito.spy(invalidateBlocks);
    Mockito.doReturn(1L).when(mockIb).getInvalidationDelay();
    Whitebox.setInternalState(
        cluster.getNamesystem().getBlockManager(), "invalidateBlocks", mockIb);

    Assert.assertEquals(0L, cluster.getNamesystem().getPendingDeletionBlocks());
    // restart DataNodes
    for (int i = 0; i < REPLICATION; i++) {
      cluster.restartDataNode(dnprops[i], true);
    }
    cluster.waitActive();

    for (int i = 0; i < REPLICATION; i++) {
      DataNodeTestUtils.triggerBlockReport(cluster.getDataNodes().get(i));
    }
    Thread.sleep(2000);
    // make sure we have received block reports by checking the total block #
    Assert.assertEquals(3, cluster.getNamesystem().getBlocksTotal());
    Assert.assertEquals(4, cluster.getNamesystem().getPendingDeletionBlocks());

    cluster.restartNameNode(true);
    Thread.sleep(6000);
    Assert.assertEquals(3, cluster.getNamesystem().getBlocksTotal());
    Assert.assertEquals(0, cluster.getNamesystem().getPendingDeletionBlocks());
  }
示例#2
0
  /*
   * Since NameNode will not persist any locations of the block, addBlock()
   * retry call after restart NN should re-select the locations and return to
   * client. refer HDFS-5257
   */
  @Test
  public void testAddBlockRetryShouldReturnBlockWithLocations() throws Exception {
    final String src = "/testAddBlockRetryShouldReturnBlockWithLocations";
    NamenodeProtocols nameNodeRpc = cluster.getNameNodeRpc();
    // create file
    nameNodeRpc.create(
        src,
        FsPermission.getFileDefault(),
        "clientName",
        new EnumSetWritable<CreateFlag>(EnumSet.of(CreateFlag.CREATE)),
        true,
        (short) 3,
        1024,
        null);
    // start first addBlock()
    LOG.info("Starting first addBlock for " + src);
    LocatedBlock lb1 =
        nameNodeRpc.addBlock(src, "clientName", null, null, INodeId.GRANDFATHER_INODE_ID, null);
    assertTrue("Block locations should be present", lb1.getLocations().length > 0);

    cluster.restartNameNode();
    nameNodeRpc = cluster.getNameNodeRpc();
    LocatedBlock lb2 =
        nameNodeRpc.addBlock(src, "clientName", null, null, INodeId.GRANDFATHER_INODE_ID, null);
    assertEquals("Blocks are not equal", lb1.getBlock(), lb2.getBlock());
    assertTrue("Wrong locations with retry", lb2.getLocations().length > 0);
  }
  /** Test that the NN re-learns of volume failures after restart. */
  @Test
  public void testVolFailureStatsPreservedOnNNRestart() throws Exception {
    assumeTrue(!System.getProperty("os.name").startsWith("Windows"));

    // Bring up two more datanodes that can tolerate 1 failure
    cluster.startDataNodes(conf, 2, true, null, null);
    cluster.waitActive();

    final DatanodeManager dm = cluster.getNamesystem().getBlockManager().getDatanodeManager();
    long origCapacity = DFSTestUtil.getLiveDatanodeCapacity(dm);
    long dnCapacity = DFSTestUtil.getDatanodeCapacity(dm, 0);

    // Fail the first volume on both datanodes (we have to keep the
    // third healthy so one node in the pipeline will not fail).
    File dn1Vol1 = new File(dataDir, "data" + (2 * 0 + 1));
    File dn2Vol1 = new File(dataDir, "data" + (2 * 1 + 1));
    assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn1Vol1, false));
    assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn2Vol1, false));

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

    // The NN reports two volumes failures
    DFSTestUtil.waitForDatanodeStatus(
        dm, 3, 0, 2, origCapacity - (1 * dnCapacity), WAIT_FOR_HEARTBEATS);

    // After restarting the NN it still see the two failures
    cluster.restartNameNode(0);
    cluster.waitActive();
    DFSTestUtil.waitForDatanodeStatus(
        dm, 3, 0, 2, origCapacity - (1 * dnCapacity), WAIT_FOR_HEARTBEATS);
  }
  @Test
  public void testOpenFilesWithRename() throws Exception {
    Path path = new Path("/test");
    doWriteAndAbort(fs, path);

    // check for zero sized blocks
    Path fileWithEmptyBlock = new Path("/test/test/test4");
    fs.create(fileWithEmptyBlock);
    NamenodeProtocols nameNodeRpc = cluster.getNameNodeRpc();
    String clientName = fs.getClient().getClientName();
    // create one empty block
    nameNodeRpc.addBlock(
        fileWithEmptyBlock.toString(),
        clientName,
        null,
        null,
        HdfsConstants.GRANDFATHER_INODE_ID,
        null);
    fs.createSnapshot(path, "s2");

    fs.rename(new Path("/test/test"), new Path("/test/test-renamed"));
    fs.delete(new Path("/test/test-renamed"), true);
    NameNode nameNode = cluster.getNameNode();
    NameNodeAdapter.enterSafeMode(nameNode, false);
    NameNodeAdapter.saveNamespace(nameNode);
    NameNodeAdapter.leaveSafeMode(nameNode);
    cluster.restartNameNode(true);
  }
  @Test
  public void testParentDirWithUCFileDeleteWithSnapShot() throws Exception {
    Path path = new Path("/test");
    doWriteAndAbort(fs, path);

    // delete parent directory
    fs.delete(new Path("/test/test"), true);
    cluster.restartNameNode();
  }
  @Test
  public void testUCFileDeleteWithSnapShot() throws Exception {
    Path path = new Path("/test");
    doWriteAndAbort(fs, path);

    // delete files separately
    fs.delete(new Path("/test/test/test2"), true);
    fs.delete(new Path("/test/test/test3"), true);
    cluster.restartNameNode();
  }
 @Test
 public void testPendingDeletion() throws Exception {
   final Path foo = new Path("/foo");
   DFSTestUtil.createFile(dfs, foo, BLOCKSIZE, REPLICATION, 0);
   // restart NN
   cluster.restartNameNode(true);
   dfs.delete(foo, true);
   Assert.assertEquals(0, cluster.getNamesystem().getBlocksTotal());
   Assert.assertEquals(REPLICATION, cluster.getNamesystem().getPendingDeletionBlocks());
   Thread.sleep(6000);
   Assert.assertEquals(0, cluster.getNamesystem().getBlocksTotal());
   Assert.assertEquals(0, cluster.getNamesystem().getPendingDeletionBlocks());
 }
 private void doTestMultipleSnapshots(boolean saveNamespace) throws IOException {
   Path path = new Path("/test");
   doWriteAndAbort(fs, path);
   fs.createSnapshot(path, "s2");
   fs.delete(new Path("/test/test"), true);
   fs.deleteSnapshot(path, "s2");
   cluster.triggerBlockReports();
   if (saveNamespace) {
     NameNode nameNode = cluster.getNameNode();
     NameNodeAdapter.enterSafeMode(nameNode, false);
     NameNodeAdapter.saveNamespace(nameNode);
     NameNodeAdapter.leaveSafeMode(nameNode);
   }
   cluster.restartNameNode(true);
 }
  @Test
  public void testWithCheckpoint() throws Exception {
    Path path = new Path("/test");
    doWriteAndAbort(fs, path);
    fs.delete(new Path("/test/test"), true);
    NameNode nameNode = cluster.getNameNode();
    NameNodeAdapter.enterSafeMode(nameNode, false);
    NameNodeAdapter.saveNamespace(nameNode);
    NameNodeAdapter.leaveSafeMode(nameNode);
    cluster.restartNameNode(true);

    // read snapshot file after restart
    String test2snapshotPath = Snapshot.getSnapshotPath(path.toString(), "s1/test/test2");
    DFSTestUtil.readFile(fs, new Path(test2snapshotPath));
    String test3snapshotPath = Snapshot.getSnapshotPath(path.toString(), "s1/test/test3");
    DFSTestUtil.readFile(fs, new Path(test3snapshotPath));
  }
示例#10
0
  /**
   * Test to simulate interleaved checkpointing by 2 2NNs after a storage directory has been taken
   * offline. The first will cause the directory to come back online, but it won't have any valid
   * contents. The second 2NN will then try to perform a checkpoint. The NN should not serve up the
   * image or edits from the restored (empty) dir.
   */
  @Test
  public void testCheckpointWithRestoredDirectory() throws IOException {
    SecondaryNameNode secondary = null;
    try {
      cluster = new MiniDFSCluster(0, config, 1, true, false, true, null, null, null, null);
      cluster.waitActive();

      secondary = new SecondaryNameNode(config);
      FSImage fsImage = cluster.getNameNode().getFSImage();

      FileSystem fs = cluster.getFileSystem();
      Path path1 = new Path("/", "test");
      writeFile(fs, path1, 2);

      // Take name3 offline
      fsImage.getEditLog().removeEditsAndStorageDir(2);

      // Simulate a 2NN beginning a checkpoint, but not finishing. This will
      // cause name3 to be restored.
      cluster.getNameNode().rollEditLog();

      // Now another 2NN comes along to do a full checkpoint.
      secondary.doCheckpoint();

      // The created file should still exist in the in-memory FS state after the
      // checkpoint.
      assertTrue("File missing after checkpoint", fs.exists(path1));

      secondary.shutdown();

      // Restart the NN so it reloads the edits from on-disk.
      cluster.restartNameNode();

      // The created file should still exist after the restart.
      assertTrue("path should still exist after restart", fs.exists(path1));
    } finally {
      if (cluster != null) {
        cluster.shutdown();
      }
      if (secondary != null) {
        secondary.shutdown();
      }
    }
  }
  /**
   * 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);
  }
  protected void doTestRead(Configuration conf, MiniDFSCluster cluster, boolean isStriped)
      throws Exception {
    final int numDataNodes = cluster.getDataNodes().size();
    final NameNode nn = cluster.getNameNode();
    final NamenodeProtocols nnProto = nn.getRpcServer();
    final BlockManager bm = nn.getNamesystem().getBlockManager();
    final BlockTokenSecretManager sm = bm.getBlockTokenSecretManager();

    // set a short token lifetime (1 second) initially
    SecurityTestUtil.setBlockTokenLifetime(sm, 1000L);

    Path fileToRead = new Path(FILE_TO_READ);
    FileSystem fs = cluster.getFileSystem();
    byte[] expected = generateBytes(FILE_SIZE);
    createFile(fs, fileToRead, expected);

    /*
     * setup for testing expiration handling of cached tokens
     */

    // read using blockSeekTo(). Acquired tokens are cached in in1
    FSDataInputStream in1 = fs.open(fileToRead);
    assertTrue(checkFile1(in1, expected));
    // read using blockSeekTo(). Acquired tokens are cached in in2
    FSDataInputStream in2 = fs.open(fileToRead);
    assertTrue(checkFile1(in2, expected));
    // read using fetchBlockByteRange(). Acquired tokens are cached in in3
    FSDataInputStream in3 = fs.open(fileToRead);
    assertTrue(checkFile2(in3, expected));

    /*
     * testing READ interface on DN using a BlockReader
     */
    DFSClient client = null;
    try {
      client = new DFSClient(new InetSocketAddress("localhost", cluster.getNameNodePort()), conf);
    } finally {
      if (client != null) client.close();
    }
    List<LocatedBlock> locatedBlocks =
        nnProto.getBlockLocations(FILE_TO_READ, 0, FILE_SIZE).getLocatedBlocks();
    LocatedBlock lblock = locatedBlocks.get(0); // first block
    // verify token is not expired
    assertFalse(isBlockTokenExpired(lblock));
    // read with valid token, should succeed
    tryRead(conf, lblock, true);

    /*
     * wait till myToken and all cached tokens in in1, in2 and in3 expire
     */

    while (!isBlockTokenExpired(lblock)) {
      try {
        Thread.sleep(10);
      } catch (InterruptedException ignored) {
      }
    }

    /*
     * continue testing READ interface on DN using a BlockReader
     */

    // verify token is expired
    assertTrue(isBlockTokenExpired(lblock));
    // read should fail
    tryRead(conf, lblock, false);
    // use a valid new token
    bm.setBlockToken(lblock, BlockTokenIdentifier.AccessMode.READ);
    // read should succeed
    tryRead(conf, lblock, true);
    // use a token with wrong blockID
    long rightId = lblock.getBlock().getBlockId();
    long wrongId = rightId + 1;
    lblock.getBlock().setBlockId(wrongId);
    bm.setBlockToken(lblock, BlockTokenIdentifier.AccessMode.READ);
    lblock.getBlock().setBlockId(rightId);
    // read should fail
    tryRead(conf, lblock, false);
    // use a token with wrong access modes
    bm.setBlockToken(lblock, BlockTokenIdentifier.AccessMode.WRITE);
    // read should fail
    tryRead(conf, lblock, false);

    // set a long token lifetime for future tokens
    SecurityTestUtil.setBlockTokenLifetime(sm, 600 * 1000L);

    /*
     * testing that when cached tokens are expired, DFSClient will re-fetch
     * tokens transparently for READ.
     */

    // confirm all tokens cached in in1 are expired by now
    List<LocatedBlock> lblocks = DFSTestUtil.getAllBlocks(in1);
    for (LocatedBlock blk : lblocks) {
      assertTrue(isBlockTokenExpired(blk));
    }
    // verify blockSeekTo() is able to re-fetch token transparently
    in1.seek(0);
    assertTrue(checkFile1(in1, expected));

    // confirm all tokens cached in in2 are expired by now
    List<LocatedBlock> lblocks2 = DFSTestUtil.getAllBlocks(in2);
    for (LocatedBlock blk : lblocks2) {
      assertTrue(isBlockTokenExpired(blk));
    }
    // verify blockSeekTo() is able to re-fetch token transparently (testing
    // via another interface method)
    if (isStriped) {
      // striped block doesn't support seekToNewSource
      in2.seek(0);
    } else {
      assertTrue(in2.seekToNewSource(0));
    }
    assertTrue(checkFile1(in2, expected));

    // confirm all tokens cached in in3 are expired by now
    List<LocatedBlock> lblocks3 = DFSTestUtil.getAllBlocks(in3);
    for (LocatedBlock blk : lblocks3) {
      assertTrue(isBlockTokenExpired(blk));
    }
    // verify fetchBlockByteRange() is able to re-fetch token transparently
    assertTrue(checkFile2(in3, expected));

    /*
     * testing that after datanodes are restarted on the same ports, cached
     * tokens should still work and there is no need to fetch new tokens from
     * namenode. This test should run while namenode is down (to make sure no
     * new tokens can be fetched from namenode).
     */

    // restart datanodes on the same ports that they currently use
    assertTrue(cluster.restartDataNodes(true));
    cluster.waitActive();
    assertEquals(numDataNodes, cluster.getDataNodes().size());
    cluster.shutdownNameNode(0);

    // confirm tokens cached in in1 are still valid
    lblocks = DFSTestUtil.getAllBlocks(in1);
    for (LocatedBlock blk : lblocks) {
      assertFalse(isBlockTokenExpired(blk));
    }
    // verify blockSeekTo() still works (forced to use cached tokens)
    in1.seek(0);
    assertTrue(checkFile1(in1, expected));

    // confirm tokens cached in in2 are still valid
    lblocks2 = DFSTestUtil.getAllBlocks(in2);
    for (LocatedBlock blk : lblocks2) {
      assertFalse(isBlockTokenExpired(blk));
    }

    // verify blockSeekTo() still works (forced to use cached tokens)
    if (isStriped) {
      in2.seek(0);
    } else {
      in2.seekToNewSource(0);
    }
    assertTrue(checkFile1(in2, expected));

    // confirm tokens cached in in3 are still valid
    lblocks3 = DFSTestUtil.getAllBlocks(in3);
    for (LocatedBlock blk : lblocks3) {
      assertFalse(isBlockTokenExpired(blk));
    }
    // verify fetchBlockByteRange() still works (forced to use cached tokens)
    assertTrue(checkFile2(in3, expected));

    /*
     * testing that when namenode is restarted, cached tokens should still
     * work and there is no need to fetch new tokens from namenode. Like the
     * previous test, this test should also run while namenode is down. The
     * setup for this test depends on the previous test.
     */

    // restart the namenode and then shut it down for test
    cluster.restartNameNode(0);
    cluster.shutdownNameNode(0);

    // verify blockSeekTo() still works (forced to use cached tokens)
    in1.seek(0);
    assertTrue(checkFile1(in1, expected));
    // verify again blockSeekTo() still works (forced to use cached tokens)
    if (isStriped) {
      in2.seek(0);
    } else {
      in2.seekToNewSource(0);
    }
    assertTrue(checkFile1(in2, expected));

    // verify fetchBlockByteRange() still works (forced to use cached tokens)
    assertTrue(checkFile2(in3, expected));

    /*
     * testing that after both namenode and datanodes got restarted (namenode
     * first, followed by datanodes), DFSClient can't access DN without
     * re-fetching tokens and is able to re-fetch tokens transparently. The
     * setup of this test depends on the previous test.
     */

    // restore the cluster and restart the datanodes for test
    cluster.restartNameNode(0);
    assertTrue(cluster.restartDataNodes(true));
    cluster.waitActive();
    assertEquals(numDataNodes, cluster.getDataNodes().size());

    // shutdown namenode so that DFSClient can't get new tokens from namenode
    cluster.shutdownNameNode(0);

    // verify blockSeekTo() fails (cached tokens become invalid)
    in1.seek(0);
    assertFalse(checkFile1(in1, expected));
    // verify fetchBlockByteRange() fails (cached tokens become invalid)
    assertFalse(checkFile2(in3, expected));

    // restart the namenode to allow DFSClient to re-fetch tokens
    cluster.restartNameNode(0);
    // verify blockSeekTo() works again (by transparently re-fetching
    // tokens from namenode)
    in1.seek(0);
    assertTrue(checkFile1(in1, expected));
    if (isStriped) {
      in2.seek(0);
    } else {
      in2.seekToNewSource(0);
    }
    assertTrue(checkFile1(in2, expected));
    // verify fetchBlockByteRange() works again (by transparently
    // re-fetching tokens from namenode)
    assertTrue(checkFile2(in3, expected));

    /*
     * testing that when datanodes are restarted on different ports, DFSClient
     * is able to re-fetch tokens transparently to connect to them
     */

    // restart datanodes on newly assigned ports
    assertTrue(cluster.restartDataNodes(false));
    cluster.waitActive();
    assertEquals(numDataNodes, cluster.getDataNodes().size());
    // verify blockSeekTo() is able to re-fetch token transparently
    in1.seek(0);
    assertTrue(checkFile1(in1, expected));
    // verify blockSeekTo() is able to re-fetch token transparently
    if (isStriped) {
      in2.seek(0);
    } else {
      in2.seekToNewSource(0);
    }
    assertTrue(checkFile1(in2, expected));
    // verify fetchBlockByteRange() is able to re-fetch token transparently
    assertTrue(checkFile2(in3, expected));
  }