|
@@ -51,16 +51,13 @@ import java.io.File;
|
|
|
import java.io.FileWriter;
|
|
|
import java.io.IOException;
|
|
|
import java.io.PrintWriter;
|
|
|
-import java.io.RandomAccessFile;
|
|
|
import java.net.InetSocketAddress;
|
|
|
import java.net.URI;
|
|
|
import java.net.URISyntaxException;
|
|
|
-import java.nio.channels.FileChannel;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Collection;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
-import java.util.Random;
|
|
|
import java.util.Set;
|
|
|
import java.util.concurrent.TimeoutException;
|
|
|
|
|
@@ -97,6 +94,9 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.DataStorage;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.FsDatasetTestUtils;
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.FsDatasetTestUtils.MaterializedReplica;
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter.SecureResources;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
|
|
@@ -122,7 +122,6 @@ import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.apache.hadoop.util.ExitUtil;
|
|
|
import org.apache.hadoop.util.ShutdownHookManager;
|
|
|
import org.apache.hadoop.util.StringUtils;
|
|
|
-import org.apache.hadoop.util.ShutdownHookManager;
|
|
|
import org.apache.hadoop.util.ToolRunner;
|
|
|
|
|
|
import com.google.common.base.Joiner;
|
|
@@ -1823,7 +1822,29 @@ public class MiniDFSCluster {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Gets the rpc port used by the NameNode, because the caller
|
|
|
+ * Returns the corresponding FsDatasetTestUtils for a DataNode.
|
|
|
+ * @param dnIdx the index of DataNode.
|
|
|
+ * @return a FsDatasetTestUtils for the given DataNode.
|
|
|
+ */
|
|
|
+ public FsDatasetTestUtils getFsDatasetTestUtils(int dnIdx) {
|
|
|
+ Preconditions.checkArgument(dnIdx < dataNodes.size());
|
|
|
+ return FsDatasetTestUtils.Factory.getFactory(conf)
|
|
|
+ .newInstance(dataNodes.get(dnIdx).datanode);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Returns the corresponding FsDatasetTestUtils for a DataNode.
|
|
|
+ * @param dn a DataNode
|
|
|
+ * @return a FsDatasetTestUtils for the given DataNode.
|
|
|
+ */
|
|
|
+ public FsDatasetTestUtils getFsDatasetTestUtils(DataNode dn) {
|
|
|
+ Preconditions.checkArgument(dn != null);
|
|
|
+ return FsDatasetTestUtils.Factory.getFactory(conf)
|
|
|
+ .newInstance(dn);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Gets the rpc port used by the NameNode, because the caller
|
|
|
* supplied port is not necessarily the actual port used.
|
|
|
* Assumption: cluster has a single namenode
|
|
|
*/
|
|
@@ -2006,11 +2027,18 @@ public class MiniDFSCluster {
|
|
|
private int corruptBlockOnDataNodesHelper(ExtendedBlock block,
|
|
|
boolean deleteBlockFile) throws IOException {
|
|
|
int blocksCorrupted = 0;
|
|
|
- File[] blockFiles = getAllBlockFiles(block);
|
|
|
- for (File f : blockFiles) {
|
|
|
- if ((deleteBlockFile && corruptBlockByDeletingBlockFile(f)) ||
|
|
|
- (!deleteBlockFile && corruptBlock(f))) {
|
|
|
+ for (DataNode dn : getDataNodes()) {
|
|
|
+ try {
|
|
|
+ MaterializedReplica replica =
|
|
|
+ getFsDatasetTestUtils(dn).getMaterializedReplica(block);
|
|
|
+ if (deleteBlockFile) {
|
|
|
+ replica.deleteData();
|
|
|
+ } else {
|
|
|
+ replica.corruptData();
|
|
|
+ }
|
|
|
blocksCorrupted++;
|
|
|
+ } catch (ReplicaNotFoundException e) {
|
|
|
+ // Ignore.
|
|
|
}
|
|
|
}
|
|
|
return blocksCorrupted;
|
|
@@ -2062,46 +2090,33 @@ public class MiniDFSCluster {
|
|
|
*
|
|
|
* @param i index of the datanode
|
|
|
* @param blk name of the block
|
|
|
- * @throws IOException on error accessing the given block or if
|
|
|
- * the contents of the block (on the same datanode) differ.
|
|
|
- * @return true if a replica was corrupted, false otherwise
|
|
|
- * Types: delete, write bad data, truncate
|
|
|
+ * @throws IOException on error accessing the given block file.
|
|
|
*/
|
|
|
- public boolean corruptReplica(int i, ExtendedBlock blk)
|
|
|
+ public void corruptReplica(int i, ExtendedBlock blk)
|
|
|
throws IOException {
|
|
|
- File blockFile = getBlockFile(i, blk);
|
|
|
- return corruptBlock(blockFile);
|
|
|
+ getMaterializedReplica(i, blk).corruptData();
|
|
|
}
|
|
|
|
|
|
- /*
|
|
|
- * Corrupt a block on a particular datanode
|
|
|
+ /**
|
|
|
+ * Corrupt a block on a particular datanode.
|
|
|
+ *
|
|
|
+ * @param dn the datanode
|
|
|
+ * @param blk name of the block
|
|
|
+ * @throws IOException on error accessing the given block file.
|
|
|
*/
|
|
|
- public static boolean corruptBlock(File blockFile) throws IOException {
|
|
|
- if (blockFile == null || !blockFile.exists()) {
|
|
|
- return false;
|
|
|
- }
|
|
|
- // Corrupt replica by writing random bytes into replica
|
|
|
- Random random = new Random();
|
|
|
- RandomAccessFile raFile = new RandomAccessFile(blockFile, "rw");
|
|
|
- FileChannel channel = raFile.getChannel();
|
|
|
- String badString = "BADBAD";
|
|
|
- int rand = random.nextInt((int)channel.size()/2);
|
|
|
- raFile.seek(rand);
|
|
|
- raFile.write(badString.getBytes());
|
|
|
- raFile.close();
|
|
|
- LOG.warn("Corrupting the block " + blockFile);
|
|
|
- return true;
|
|
|
+ public void corruptReplica(DataNode dn, ExtendedBlock blk)
|
|
|
+ throws IOException {
|
|
|
+ getMaterializedReplica(dn, blk).corruptData();
|
|
|
}
|
|
|
|
|
|
- /*
|
|
|
- * Corrupt a block on a particular datanode by deleting the block file
|
|
|
+ /**
|
|
|
+ * Corrupt the metadata of a block on a datanode.
|
|
|
+ * @param i the index of the datanode
|
|
|
+ * @param blk name of the block
|
|
|
+ * @throws IOException on error accessing the given metadata file.
|
|
|
*/
|
|
|
- public static boolean corruptBlockByDeletingBlockFile(File blockFile)
|
|
|
- throws IOException {
|
|
|
- if (blockFile == null || !blockFile.exists()) {
|
|
|
- return false;
|
|
|
- }
|
|
|
- return blockFile.delete();
|
|
|
+ public void corruptMeta(int i, ExtendedBlock blk) throws IOException {
|
|
|
+ getMaterializedReplica(i, blk).corruptMeta();
|
|
|
}
|
|
|
|
|
|
public boolean changeGenStampOfBlock(int dnIndex, ExtendedBlock blk,
|
|
@@ -2757,7 +2772,33 @@ public class MiniDFSCluster {
|
|
|
return new File(getBPDir(storageDir, bpid, Storage.STORAGE_DIR_CURRENT)
|
|
|
+ DataStorage.STORAGE_DIR_FINALIZED );
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Get materialized replica that can be corrupted later.
|
|
|
+ * @param i the index of DataNode.
|
|
|
+ * @param blk name of the block.
|
|
|
+ * @return a materialized replica.
|
|
|
+ * @throws ReplicaNotFoundException if the replica does not exist on the
|
|
|
+ * DataNode.
|
|
|
+ */
|
|
|
+ public MaterializedReplica getMaterializedReplica(
|
|
|
+ int i, ExtendedBlock blk) throws ReplicaNotFoundException {
|
|
|
+ return getFsDatasetTestUtils(i).getMaterializedReplica(blk);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Get materialized replica that can be corrupted later.
|
|
|
+ * @param dn the index of DataNode.
|
|
|
+ * @param blk name of the block.
|
|
|
+ * @return a materialized replica.
|
|
|
+ * @throws ReplicaNotFoundException if the replica does not exist on the
|
|
|
+ * DataNode.
|
|
|
+ */
|
|
|
+ public MaterializedReplica getMaterializedReplica(
|
|
|
+ DataNode dn, ExtendedBlock blk) throws ReplicaNotFoundException {
|
|
|
+ return getFsDatasetTestUtils(dn).getMaterializedReplica(blk);
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Get file correpsonding to a block
|
|
|
* @param storageDir storage directory
|