protected final boolean verifyDeletedBlocks(LocatedBlocks locatedBlocks)
      throws IOException, InterruptedException {

    LOG.info("Verifying replica has no saved copy after deletion.");
    triggerBlockReport();

    while (DataNodeTestUtils.getPendingAsyncDeletions(cluster.getDataNodes().get(0)) > 0L) {
      Thread.sleep(1000);
    }

    final String bpid = cluster.getNamesystem().getBlockPoolId();
    List<? extends FsVolumeSpi> volumes = cluster.getDataNodes().get(0).getFSDataset().getVolumes();

    // Make sure deleted replica does not have a copy on either finalized dir of
    // transient volume or finalized dir of non-transient volume
    for (FsVolumeSpi v : volumes) {
      FsVolumeImpl volume = (FsVolumeImpl) v;
      File targetDir =
          (v.isTransientStorage())
              ? volume.getBlockPoolSlice(bpid).getFinalizedDir()
              : volume.getBlockPoolSlice(bpid).getLazypersistDir();
      if (verifyBlockDeletedFromDir(targetDir, locatedBlocks) == false) {
        return false;
      }
    }
    return true;
  }
  /**
   * Make sure at least one non-transient volume has a saved copy of the replica. An infinite loop
   * is used to ensure the async lazy persist tasks are completely done before verification. Caller
   * of ensureLazyPersistBlocksAreSaved expects either a successful pass or timeout failure.
   */
  protected final void ensureLazyPersistBlocksAreSaved(LocatedBlocks locatedBlocks)
      throws IOException, InterruptedException {
    final String bpid = cluster.getNamesystem().getBlockPoolId();
    List<? extends FsVolumeSpi> volumes = cluster.getDataNodes().get(0).getFSDataset().getVolumes();
    final Set<Long> persistedBlockIds = new HashSet<Long>();

    while (persistedBlockIds.size() < locatedBlocks.getLocatedBlocks().size()) {
      // Take 1 second sleep before each verification iteration
      Thread.sleep(1000);

      for (LocatedBlock lb : locatedBlocks.getLocatedBlocks()) {
        for (FsVolumeSpi v : volumes) {
          if (v.isTransientStorage()) {
            continue;
          }

          FsVolumeImpl volume = (FsVolumeImpl) v;
          File lazyPersistDir = volume.getBlockPoolSlice(bpid).getLazypersistDir();

          long blockId = lb.getBlock().getBlockId();
          File targetDir = DatanodeUtil.idToBlockDir(lazyPersistDir, blockId);
          File blockFile = new File(targetDir, lb.getBlock().getBlockName());
          if (blockFile.exists()) {
            // Found a persisted copy for this block and added to the Set
            persistedBlockIds.add(blockId);
          }
        }
      }
    }

    // We should have found a persisted copy for each located block.
    assertThat(persistedBlockIds.size(), is(locatedBlocks.getLocatedBlocks().size()));
  }
  /**
   * If ramDiskStorageLimit is >=0, then RAM_DISK capacity is artificially capped. If
   * ramDiskStorageLimit < 0 then it is ignored.
   */
  protected final void startUpCluster(
      final int numDataNodes,
      final StorageType[] storageTypes,
      final long ramDiskStorageLimit,
      final boolean useSCR)
      throws IOException {

    Configuration conf = new Configuration();
    conf.setLong(DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
    conf.setInt(
        DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC, LAZY_WRITE_FILE_SCRUBBER_INTERVAL_SEC);
    conf.setLong(DFS_HEARTBEAT_INTERVAL_KEY, HEARTBEAT_INTERVAL_SEC);
    conf.setInt(DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, HEARTBEAT_RECHECK_INTERVAL_MSEC);
    conf.setInt(DFS_DATANODE_LAZY_WRITER_INTERVAL_SEC, LAZY_WRITER_INTERVAL_SEC);

    if (useSCR) {
      conf.setBoolean(DFS_CLIENT_READ_SHORTCIRCUIT_KEY, useSCR);
      conf.set(DFS_CLIENT_CONTEXT, UUID.randomUUID().toString());
      sockDir = new TemporarySocketDirectory();
      conf.set(
          DFS_DOMAIN_SOCKET_PATH_KEY,
          new File(sockDir.getDir(), this.getClass().getSimpleName() + "._PORT.sock")
              .getAbsolutePath());
      conf.set(
          DFS_BLOCK_LOCAL_PATH_ACCESS_USER_KEY,
          UserGroupInformation.getCurrentUser().getShortUserName());
    }

    cluster =
        new MiniDFSCluster.Builder(conf)
            .numDataNodes(numDataNodes)
            .storageTypes(
                storageTypes != null ? storageTypes : new StorageType[] {DEFAULT, DEFAULT})
            .build();
    fs = cluster.getFileSystem();
    client = fs.getClient();

    // Artificially cap the storage capacity of the RAM_DISK volume.
    if (ramDiskStorageLimit >= 0) {
      List<? extends FsVolumeSpi> volumes =
          cluster.getDataNodes().get(0).getFSDataset().getVolumes();

      for (FsVolumeSpi volume : volumes) {
        if (volume.getStorageType() == RAM_DISK) {
          ((FsVolumeImpl) volume).setCapacityForTesting(ramDiskStorageLimit);
        }
      }
    }

    LOG.info("Cluster startup complete");
  }