HDFS-9188. Make block corruption related tests FsDataset-agnostic. (lei)
This commit is contained in:
parent
be7a0add8b
commit
c80b3a804f
|
@ -1529,6 +1529,8 @@ Release 2.8.0 - UNRELEASED
|
|||
|
||||
HDFS-9223. Code cleanup for DatanodeDescriptor and HeartbeatManager. (jing9)
|
||||
|
||||
HDFS-9188. Make block corruption related tests FsDataset-agnostic. (lei)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -18,11 +18,9 @@
|
|||
package org.apache.hadoop.hdfs;
|
||||
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.File;
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
import java.io.RandomAccessFile;
|
||||
import java.util.Random;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
|
||||
|
@ -39,7 +37,6 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|||
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
||||
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
||||
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
||||
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
|
||||
import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck;
|
||||
import org.apache.hadoop.hdfs.tools.DFSck;
|
||||
import org.apache.hadoop.security.AccessControlException;
|
||||
|
@ -217,7 +214,7 @@ public class TestClientReportBadBlock {
|
|||
for (int i = 0; i < corruptBlockCount; i++) {
|
||||
DatanodeInfo dninfo = datanodeinfos[i];
|
||||
final DataNode dn = cluster.getDataNode(dninfo.getIpcPort());
|
||||
corruptBlock(block, dn);
|
||||
cluster.corruptReplica(dn, block);
|
||||
LOG.debug("Corrupted block " + block.getBlockName() + " on data node "
|
||||
+ dninfo);
|
||||
|
||||
|
@ -292,30 +289,6 @@ public class TestClientReportBadBlock {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Corrupt a block on a data node. Replace the block file content with content
|
||||
* of 1, 2, ...BLOCK_SIZE.
|
||||
*
|
||||
* @param block
|
||||
* the ExtendedBlock to be corrupted
|
||||
* @param dn
|
||||
* the data node where the block needs to be corrupted
|
||||
* @throws FileNotFoundException
|
||||
* @throws IOException
|
||||
*/
|
||||
private static void corruptBlock(final ExtendedBlock block, final DataNode dn)
|
||||
throws FileNotFoundException, IOException {
|
||||
final File f = DataNodeTestUtils.getBlockFile(
|
||||
dn, block.getBlockPoolId(), block.getLocalBlock());
|
||||
final RandomAccessFile raFile = new RandomAccessFile(f, "rw");
|
||||
final byte[] bytes = new byte[(int) BLOCK_SIZE];
|
||||
for (int i = 0; i < BLOCK_SIZE; i++) {
|
||||
bytes[i] = (byte) (i);
|
||||
}
|
||||
raFile.write(bytes);
|
||||
raFile.close();
|
||||
}
|
||||
|
||||
private static void verifyFsckHealth(String expected) throws Exception {
|
||||
// Fsck health has error code 0.
|
||||
// Make sure filesystem is in healthy state
|
||||
|
|
|
@ -39,8 +39,8 @@ import org.apache.hadoop.fs.permission.FsPermission;
|
|||
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
|
||||
import org.apache.hadoop.hdfs.protocol.Block;
|
||||
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
|
||||
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
||||
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.hdfs.server.datanode.FsDatasetTestUtils.MaterializedReplica;
|
||||
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
|
||||
import org.apache.hadoop.hdfs.tools.DFSAdmin;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
|
@ -1507,32 +1507,31 @@ public class TestDFSShell {
|
|||
}
|
||||
}
|
||||
|
||||
static List<File> getBlockFiles(MiniDFSCluster cluster) throws IOException {
|
||||
List<File> files = new ArrayList<File>();
|
||||
List<DataNode> datanodes = cluster.getDataNodes();
|
||||
private static List<MaterializedReplica> getMaterializedReplicas(
|
||||
MiniDFSCluster cluster) throws IOException {
|
||||
List<MaterializedReplica> replicas = new ArrayList<>();
|
||||
String poolId = cluster.getNamesystem().getBlockPoolId();
|
||||
List<Map<DatanodeStorage, BlockListAsLongs>> blocks = cluster.getAllBlockReports(poolId);
|
||||
List<Map<DatanodeStorage, BlockListAsLongs>> blocks =
|
||||
cluster.getAllBlockReports(poolId);
|
||||
for(int i = 0; i < blocks.size(); i++) {
|
||||
DataNode dn = datanodes.get(i);
|
||||
Map<DatanodeStorage, BlockListAsLongs> map = blocks.get(i);
|
||||
for(Map.Entry<DatanodeStorage, BlockListAsLongs> e : map.entrySet()) {
|
||||
for(Block b : e.getValue()) {
|
||||
files.add(DataNodeTestUtils.getFile(dn, poolId, b.getBlockId()));
|
||||
replicas.add(cluster.getMaterializedReplica(i,
|
||||
new ExtendedBlock(poolId, b)));
|
||||
}
|
||||
}
|
||||
}
|
||||
return files;
|
||||
return replicas;
|
||||
}
|
||||
|
||||
static void corrupt(List<File> files) throws IOException {
|
||||
for(File f : files) {
|
||||
StringBuilder content = new StringBuilder(DFSTestUtil.readFile(f));
|
||||
char c = content.charAt(0);
|
||||
content.setCharAt(0, ++c);
|
||||
PrintWriter out = new PrintWriter(f);
|
||||
out.print(content);
|
||||
out.flush();
|
||||
out.close();
|
||||
private static void corrupt(
|
||||
List<MaterializedReplica> replicas, String content) throws IOException {
|
||||
StringBuilder sb = new StringBuilder(content);
|
||||
char c = content.charAt(0);
|
||||
sb.setCharAt(0, ++c);
|
||||
for(MaterializedReplica replica : replicas) {
|
||||
replica.corruptData(sb.toString().getBytes("UTF8"));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1636,7 +1635,7 @@ public class TestDFSShell {
|
|||
assertEquals(localfcontent, runner.run(0, "-ignoreCrc"));
|
||||
|
||||
// find block files to modify later
|
||||
List<File> files = getBlockFiles(cluster);
|
||||
List<MaterializedReplica> replicas = getMaterializedReplicas(cluster);
|
||||
|
||||
// Shut down cluster and then corrupt the block files by overwriting a
|
||||
// portion with junk data. We must shut down the cluster so that threads
|
||||
|
@ -1649,8 +1648,8 @@ public class TestDFSShell {
|
|||
dfs.close();
|
||||
cluster.shutdown();
|
||||
|
||||
show("files=" + files);
|
||||
corrupt(files);
|
||||
show("replicas=" + replicas);
|
||||
corrupt(replicas, localfcontent);
|
||||
|
||||
// Start the cluster again, but do not reformat, so prior files remain.
|
||||
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).format(false)
|
||||
|
|
|
@ -21,20 +21,18 @@ import static org.junit.Assert.assertEquals;
|
|||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.RandomAccessFile;
|
||||
import java.util.EnumSet;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.fs.CreateFlag;
|
||||
import org.apache.hadoop.hdfs.server.datanode.FsDatasetTestUtils.MaterializedReplica;
|
||||
import org.apache.hadoop.test.GenericTestUtils;
|
||||
import org.mockito.invocation.InvocationOnMock;
|
||||
import static org.mockito.Mockito.spy;
|
||||
import static org.mockito.Mockito.when;
|
||||
import org.mockito.stubbing.Answer;
|
||||
|
||||
import org.apache.commons.logging.impl.Log4JLogger;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
|
@ -315,13 +313,7 @@ public class TestFileAppend3 {
|
|||
DatanodeInfo[] datanodeinfos = lb.getLocations();
|
||||
assertEquals(repl, datanodeinfos.length);
|
||||
final DataNode dn = cluster.getDataNode(datanodeinfos[0].getIpcPort());
|
||||
final File f = DataNodeTestUtils.getBlockFile(
|
||||
dn, blk.getBlockPoolId(), blk.getLocalBlock());
|
||||
final RandomAccessFile raf = new RandomAccessFile(f, "rw");
|
||||
AppendTestUtil.LOG.info("dn=" + dn + ", blk=" + blk + " (length=" + blk.getNumBytes() + ")");
|
||||
assertEquals(len1, raf.length());
|
||||
raf.setLength(0);
|
||||
raf.close();
|
||||
cluster.getMaterializedReplica(dn, blk).truncateData(0);
|
||||
|
||||
//c. Open file in "append mode". Append a new block worth of data. Close file.
|
||||
final int len2 = (int)BLOCK_SIZE;
|
||||
|
|
|
@ -81,7 +81,7 @@ public class TestMissingBlocksAlert {
|
|||
|
||||
// Corrupt the block
|
||||
ExtendedBlock block = DFSTestUtil.getFirstBlock(dfs, corruptFile);
|
||||
assertTrue(cluster.corruptReplica(0, block));
|
||||
cluster.corruptReplica(0, block);
|
||||
|
||||
// read the file so that the corrupt block is reported to NN
|
||||
FSDataInputStream in = dfs.open(corruptFile);
|
||||
|
@ -126,7 +126,7 @@ public class TestMissingBlocksAlert {
|
|||
DFSTestUtil.createFile(dfs, replOneFile, fileLen, (short)1, 0);
|
||||
ExtendedBlock replOneBlock = DFSTestUtil.getFirstBlock(
|
||||
dfs, replOneFile);
|
||||
assertTrue(cluster.corruptReplica(0, replOneBlock));
|
||||
cluster.corruptReplica(0, replOneBlock);
|
||||
|
||||
// read the file so that the corrupt block is reported to NN
|
||||
in = dfs.open(replOneFile);
|
||||
|
|
|
@ -210,7 +210,7 @@ public class TestBlocksWithNotEnoughRacks {
|
|||
|
||||
// Corrupt a replica of the block
|
||||
int dnToCorrupt = DFSTestUtil.firstDnWithBlock(cluster, b);
|
||||
assertTrue(cluster.corruptReplica(dnToCorrupt, b));
|
||||
cluster.corruptReplica(dnToCorrupt, b);
|
||||
|
||||
// Restart the datanode so blocks are re-scanned, and the corrupt
|
||||
// block is detected.
|
||||
|
|
|
@ -67,7 +67,7 @@ public class TestOverReplicatedBlocks {
|
|||
|
||||
// corrupt the block on datanode 0
|
||||
ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, fileName);
|
||||
assertTrue(cluster.corruptReplica(0, block));
|
||||
cluster.corruptReplica(0, block);
|
||||
DataNodeProperties dnProps = cluster.stopDataNode(0);
|
||||
// remove block scanner log to trigger block scanning
|
||||
File scanCursor = new File(new File(MiniDFSCluster.getFinalizedDir(
|
||||
|
|
|
@ -22,7 +22,6 @@ import static org.junit.Assert.assertTrue;
|
|||
import static org.junit.Assume.assumeTrue;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
|
@ -37,10 +36,8 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
|
|||
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
|
||||
import org.apache.hadoop.hdfs.protocol.Block;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
||||
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
|
||||
import org.apache.hadoop.hdfs.server.datanode.FsDatasetTestUtils.MaterializedReplica;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
||||
import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
|
||||
import org.apache.hadoop.hdfs.server.namenode.ha.TestDNFencing.RandomDeleterPolicy;
|
||||
|
@ -91,19 +88,14 @@ public class TestRBWBlockInvalidation {
|
|||
out.writeBytes("HDFS-3157: " + testPath);
|
||||
out.hsync();
|
||||
cluster.startDataNodes(conf, 1, true, null, null, null);
|
||||
String bpid = namesystem.getBlockPoolId();
|
||||
ExtendedBlock blk = DFSTestUtil.getFirstBlock(fs, testPath);
|
||||
Block block = blk.getLocalBlock();
|
||||
DataNode dn = cluster.getDataNodes().get(0);
|
||||
|
||||
// Delete partial block and its meta information from the RBW folder
|
||||
// of first datanode.
|
||||
File blockFile = DataNodeTestUtils.getBlockFile(dn, bpid, block);
|
||||
File metaFile = DataNodeTestUtils.getMetaFile(dn, bpid, block);
|
||||
assertTrue("Could not delete the block file from the RBW folder",
|
||||
blockFile.delete());
|
||||
assertTrue("Could not delete the block meta file from the RBW folder",
|
||||
metaFile.delete());
|
||||
MaterializedReplica replica = cluster.getMaterializedReplica(0, blk);
|
||||
|
||||
replica.deleteData();
|
||||
replica.deleteMeta();
|
||||
|
||||
out.close();
|
||||
|
||||
|
|
|
@ -158,20 +158,6 @@ public class DataNodeTestUtils {
|
|||
return dn.getFSDataset();
|
||||
}
|
||||
|
||||
public static File getFile(DataNode dn, String bpid, long bid) {
|
||||
return FsDatasetTestUtil.getFile(dn.getFSDataset(), bpid, bid);
|
||||
}
|
||||
|
||||
public static File getBlockFile(DataNode dn, String bpid, Block b
|
||||
) throws IOException {
|
||||
return FsDatasetTestUtil.getBlockFile(dn.getFSDataset(), bpid, b);
|
||||
}
|
||||
|
||||
public static File getMetaFile(DataNode dn, String bpid, Block b)
|
||||
throws IOException {
|
||||
return FsDatasetTestUtil.getMetaFile(dn.getFSDataset(), bpid, b);
|
||||
}
|
||||
|
||||
public static long getPendingAsyncDeletions(DataNode dn) {
|
||||
return FsDatasetTestUtil.getPendingAsyncDeletions(dn.getFSDataset());
|
||||
}
|
||||
|
|
|
@ -0,0 +1,32 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.hdfs.server.datanode;
|
||||
|
||||
import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetImplTestUtils;
|
||||
|
||||
/**
|
||||
* A factory for creating {@link FsDatasetImplTestUtils} objects.
|
||||
*/
|
||||
public final class FsDatasetImplTestUtilsFactory
|
||||
extends FsDatasetTestUtils.Factory<FsDatasetTestUtils> {
|
||||
@Override
|
||||
public FsDatasetTestUtils newInstance(DataNode datanode) {
|
||||
return new FsDatasetImplTestUtils(datanode);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,140 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hdfs.server.datanode;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetFactory;
|
||||
import org.apache.hadoop.util.ReflectionUtils;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Provide block access for FsDataset white box tests.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Unstable
|
||||
public interface FsDatasetTestUtils {
|
||||
|
||||
abstract class Factory<D extends FsDatasetTestUtils> {
|
||||
/**
|
||||
* By default, it returns FsDatasetImplTestUtilsFactory.
|
||||
*
|
||||
* @return The configured Factory.
|
||||
*/
|
||||
public static Factory<?> getFactory(Configuration conf) {
|
||||
String className = conf.get(
|
||||
DFSConfigKeys.DFS_DATANODE_FSDATASET_FACTORY_KEY,
|
||||
FsDatasetFactory.class.getName());
|
||||
Preconditions.checkState(className.contains("Factory"));
|
||||
className = className.replaceFirst("(\\$)?Factory$", "TestUtilsFactory");
|
||||
final Class<? extends Factory> clazz = conf.getClass(
|
||||
className,
|
||||
FsDatasetImplTestUtilsFactory.class,
|
||||
Factory.class);
|
||||
return ReflectionUtils.newInstance(clazz, conf);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a new instance of FsDatasetTestUtils.
|
||||
*/
|
||||
public abstract D newInstance(DataNode datanode);
|
||||
|
||||
/**
|
||||
* @return True for SimulatedFsDataset
|
||||
*/
|
||||
public boolean isSimulated() {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A replica to be corrupted.
|
||||
*
|
||||
* It is safe to corrupt this replica even if the MiniDFSCluster is shutdown.
|
||||
*/
|
||||
interface MaterializedReplica {
|
||||
|
||||
/**
|
||||
* Corrupt the block file of the replica.
|
||||
* @throws FileNotFoundException if the block file does not exist.
|
||||
* @throws IOException if I/O error.
|
||||
*/
|
||||
void corruptData() throws IOException;
|
||||
|
||||
/**
|
||||
* Corrupt the block file with the given content.
|
||||
* @param newContent the new content written to the block file.
|
||||
* @throws FileNotFoundException if the block file does not exist.
|
||||
* @throws IOException if I/O error.
|
||||
*/
|
||||
void corruptData(byte[] newContent) throws IOException;
|
||||
|
||||
/**
|
||||
* Truncate the block file of the replica to the newSize.
|
||||
* @param newSize the new size of the block file.
|
||||
* @throws FileNotFoundException if the block file does not exist.
|
||||
* @throws IOException if I/O error.
|
||||
*/
|
||||
void truncateData(long newSize) throws IOException;
|
||||
|
||||
/**
|
||||
* Delete the block file of the replica.
|
||||
* @throws FileNotFoundException if the block file does not exist.
|
||||
* @throws IOException if I/O error.
|
||||
*/
|
||||
void deleteData() throws IOException;
|
||||
|
||||
/**
|
||||
* Corrupt the metadata file of the replica.
|
||||
* @throws FileNotFoundException if the block file does not exist.
|
||||
* @throws IOException if I/O error.
|
||||
*/
|
||||
void corruptMeta() throws IOException;
|
||||
|
||||
/**
|
||||
* Delete the metadata file of the replcia.
|
||||
* @throws FileNotFoundException if the block file does not exist.
|
||||
* @throws IOException I/O error.
|
||||
*/
|
||||
void deleteMeta() throws IOException;
|
||||
|
||||
/**
|
||||
* Truncate the metadata file of the replica to the newSize.
|
||||
* @throws FileNotFoundException if the block file does not exist.
|
||||
* @throws IOException I/O error.
|
||||
*/
|
||||
void truncateMeta(long newSize) throws IOException;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get a materialized replica to corrupt its block / crc later.
|
||||
* @param block the block of this replica begone to.
|
||||
* @return a replica to corrupt. Return null if the replica does not exist
|
||||
* in this dataset.
|
||||
* @throws ReplicaNotFoundException if the replica does not exists on the
|
||||
* dataset.
|
||||
*/
|
||||
MaterializedReplica getMaterializedReplica(ExtendedBlock block)
|
||||
throws ReplicaNotFoundException;
|
||||
}
|
|
@ -0,0 +1,179 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
||||
import org.apache.hadoop.hdfs.server.datanode.FsDatasetTestUtils;
|
||||
import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.io.RandomAccessFile;
|
||||
import java.nio.channels.FileChannel;
|
||||
import java.nio.file.Files;
|
||||
import java.util.Random;
|
||||
|
||||
/**
|
||||
* Test-related utilities to access blocks in {@link FsDatasetImpl}.
|
||||
*/
|
||||
@InterfaceStability.Unstable
|
||||
@InterfaceAudience.Private
|
||||
public class FsDatasetImplTestUtils implements FsDatasetTestUtils {
|
||||
private static final Log LOG =
|
||||
LogFactory.getLog(FsDatasetImplTestUtils.class);
|
||||
private final FsDatasetImpl dataset;
|
||||
|
||||
/**
|
||||
* A reference to the replica that is used to corrupt block / meta later.
|
||||
*/
|
||||
private static class FsDatasetImplMaterializedReplica
|
||||
implements MaterializedReplica {
|
||||
/** Block file of the replica. */
|
||||
private final File blockFile;
|
||||
private final File metaFile;
|
||||
|
||||
/** Check the existence of the file. */
|
||||
private static void checkFile(File file) throws FileNotFoundException {
|
||||
if (file == null || !file.exists()) {
|
||||
throw new FileNotFoundException(
|
||||
"The block file or metadata file " + file + " does not exist.");
|
||||
}
|
||||
}
|
||||
|
||||
/** Corrupt a block / crc file by truncating it to a newSize */
|
||||
private static void truncate(File file, long newSize)
|
||||
throws IOException {
|
||||
Preconditions.checkArgument(newSize >= 0);
|
||||
checkFile(file);
|
||||
try (RandomAccessFile raf = new RandomAccessFile(file, "rw")) {
|
||||
raf.setLength(newSize);
|
||||
}
|
||||
}
|
||||
|
||||
/** Corrupt a block / crc file by deleting it. */
|
||||
private static void delete(File file) throws IOException {
|
||||
checkFile(file);
|
||||
Files.delete(file.toPath());
|
||||
}
|
||||
|
||||
FsDatasetImplMaterializedReplica(File blockFile, File metaFile) {
|
||||
this.blockFile = blockFile;
|
||||
this.metaFile = metaFile;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void corruptData() throws IOException {
|
||||
checkFile(blockFile);
|
||||
LOG.info("Corrupting block file: " + blockFile);
|
||||
final int BUF_SIZE = 32;
|
||||
byte[] buf = new byte[BUF_SIZE];
|
||||
try (RandomAccessFile raf = new RandomAccessFile(blockFile, "rw")) {
|
||||
int nread = raf.read(buf);
|
||||
for (int i = 0; i < nread; i++) {
|
||||
buf[i]++;
|
||||
}
|
||||
raf.seek(0);
|
||||
raf.write(buf);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void corruptData(byte[] newContent) throws IOException {
|
||||
checkFile(blockFile);
|
||||
LOG.info("Corrupting block file with new content: " + blockFile);
|
||||
try (RandomAccessFile raf = new RandomAccessFile(blockFile, "rw")) {
|
||||
raf.write(newContent);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void truncateData(long newSize) throws IOException {
|
||||
LOG.info("Truncating block file: " + blockFile);
|
||||
truncate(blockFile, newSize);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void deleteData() throws IOException {
|
||||
LOG.info("Deleting block file: " + blockFile);
|
||||
delete(blockFile);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void corruptMeta() throws IOException {
|
||||
checkFile(metaFile);
|
||||
LOG.info("Corrupting meta file: " + metaFile);
|
||||
Random random = new Random();
|
||||
try (RandomAccessFile raf = new RandomAccessFile(metaFile, "rw")) {
|
||||
FileChannel channel = raf.getChannel();
|
||||
int offset = random.nextInt((int)channel.size() / 2);
|
||||
raf.seek(offset);
|
||||
raf.write("BADBAD".getBytes());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void deleteMeta() throws IOException {
|
||||
LOG.info("Deleting metadata file: " + metaFile);
|
||||
delete(metaFile);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void truncateMeta(long newSize) throws IOException {
|
||||
LOG.info("Truncating metadata file: " + metaFile);
|
||||
truncate(metaFile, newSize);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return String.format("MaterializedReplica: file=%s", blockFile);
|
||||
}
|
||||
}
|
||||
|
||||
public FsDatasetImplTestUtils(DataNode datanode) {
|
||||
Preconditions.checkArgument(
|
||||
datanode.getFSDataset() instanceof FsDatasetImpl);
|
||||
dataset = (FsDatasetImpl) datanode.getFSDataset();
|
||||
}
|
||||
|
||||
/**
|
||||
* Return a materialized replica from the FsDatasetImpl.
|
||||
*/
|
||||
@Override
|
||||
public MaterializedReplica getMaterializedReplica(ExtendedBlock block)
|
||||
throws ReplicaNotFoundException {
|
||||
File blockFile;
|
||||
try {
|
||||
blockFile = dataset.getBlockFile(
|
||||
block.getBlockPoolId(), block.getBlockId());
|
||||
} catch (IOException e) {
|
||||
LOG.error("Block file for " + block + " does not existed:", e);
|
||||
throw new ReplicaNotFoundException(block);
|
||||
}
|
||||
File metaFile = FsDatasetUtil.getMetaFile(
|
||||
blockFile, block.getGenerationStamp());
|
||||
return new FsDatasetImplMaterializedReplica(blockFile, metaFile);
|
||||
}
|
||||
}
|
|
@ -19,12 +19,9 @@ package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
|
|||
import com.google.common.base.Preconditions;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.fs.ChecksumException;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.StorageType;
|
||||
import org.apache.hadoop.hdfs.ClientContext;
|
||||
import org.apache.hadoop.hdfs.DFSTestUtil;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
|
||||
import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
|
||||
import org.apache.hadoop.io.nativeio.NativeIO;
|
||||
|
@ -264,9 +261,7 @@ public class TestScrLazyPersistFiles extends LazyPersistTestCase {
|
|||
// Corrupt the lazy-persisted checksum file, and verify that checksum
|
||||
// verification catches it.
|
||||
ensureFileReplicasOnStorageType(path1, DEFAULT);
|
||||
File metaFile = cluster.getBlockMetadataFile(0,
|
||||
DFSTestUtil.getFirstBlock(fs, path1));
|
||||
MiniDFSCluster.corruptBlock(metaFile);
|
||||
cluster.corruptMeta(0, DFSTestUtil.getFirstBlock(fs, path1));
|
||||
exception.expect(ChecksumException.class);
|
||||
DFSTestUtil.readFileBuffer(fs, path1);
|
||||
}
|
||||
|
|
|
@ -268,10 +268,12 @@ public class TestProcessCorruptBlocks {
|
|||
|
||||
private void corruptBlock(MiniDFSCluster cluster, FileSystem fs, final Path fileName,
|
||||
int dnIndex, ExtendedBlock block) throws IOException {
|
||||
// corrupt the block on datanode dnIndex
|
||||
// Truncate the block on the first datanode that has not been corrupted,
|
||||
// so that directory scanner can discover the corruption from file size
|
||||
// change.
|
||||
// the indexes change once the nodes are restarted.
|
||||
// But the datadirectory will not change
|
||||
assertTrue(cluster.corruptReplica(dnIndex, block));
|
||||
cluster.getMaterializedReplica(0, block).truncateData(10);
|
||||
|
||||
// Run directory scanner to update the DN's volume map
|
||||
DataNodeTestUtils.runDirectoryScanner(cluster.getDataNodes().get(0));
|
||||
|
|
Loading…
Reference in New Issue