HDFS-17332 DFSInputStream: avoid logging stacktrace until when we really need to fail a read request with a MissingBlockException (#6446)
Print a warn log message for read retries and only print the full stack trace for a read request failure. Contributed by: Xing Lin
This commit is contained in:
parent
cc4c4be1b7
commit
27ecc23ae7
@ -771,7 +771,7 @@ public synchronized int read() throws IOException {
|
||||
* ChecksumFileSystem
|
||||
*/
|
||||
private synchronized int readBuffer(ReaderStrategy reader, int len,
|
||||
CorruptedBlocks corruptedBlocks)
|
||||
CorruptedBlocks corruptedBlocks, final Map<InetSocketAddress, List<IOException>> exceptionMap)
|
||||
throws IOException {
|
||||
IOException ioe;
|
||||
|
||||
@ -786,6 +786,7 @@ private synchronized int readBuffer(ReaderStrategy reader, int len,
|
||||
while (true) {
|
||||
// retry as many times as seekToNewSource allows.
|
||||
try {
|
||||
DFSClientFaultInjector.get().fetchFromDatanodeException();
|
||||
return reader.readFromBlock(blockReader, len);
|
||||
} catch (ChecksumException ce) {
|
||||
DFSClient.LOG.warn("Found Checksum error for "
|
||||
@ -796,11 +797,18 @@ private synchronized int readBuffer(ReaderStrategy reader, int len,
|
||||
// we want to remember which block replicas we have tried
|
||||
corruptedBlocks.addCorruptedBlock(getCurrentBlock(), currentNode);
|
||||
} catch (IOException e) {
|
||||
if (!retryCurrentNode) {
|
||||
DFSClient.LOG.warn("Exception while reading from "
|
||||
+ getCurrentBlock() + " of " + src + " from "
|
||||
+ currentNode, e);
|
||||
String msg = String.format("Failed to read block %s for file %s from datanode %s. "
|
||||
+ "Exception is %s. Retry with the current or next available datanode.",
|
||||
getCurrentBlock().getBlockName(), src, currentNode.getXferAddr(), e);
|
||||
DFSClient.LOG.warn(msg);
|
||||
|
||||
// Add the exception to exceptionMap for this datanode.
|
||||
InetSocketAddress datanode = currentNode.getResolvedAddress();
|
||||
if (!exceptionMap.containsKey(datanode)) {
|
||||
exceptionMap.put(datanode, new LinkedList<IOException>());
|
||||
}
|
||||
exceptionMap.get(datanode).add(e);
|
||||
|
||||
ioe = e;
|
||||
}
|
||||
boolean sourceFound;
|
||||
@ -822,6 +830,29 @@ private synchronized int readBuffer(ReaderStrategy reader, int len,
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Send IOExceptions happened at each individual datanode to DFSClient.LOG for a failed read
|
||||
* request. Used in both readWithStrategy() and pread(), to record the exceptions when a read
|
||||
* request failed to be served.
|
||||
* @param position offset in the file where we fail to read
|
||||
* @param exceptionMap a map which stores the list of IOExceptions for each datanode
|
||||
*/
|
||||
private void logDataNodeExceptionsOnReadError(long position, final Map<InetSocketAddress,
|
||||
List<IOException>> exceptionMap) {
|
||||
String msg = String.format("Failed to read from all available datanodes for file %s "
|
||||
+ "at position=%d after retrying.", src, position);
|
||||
DFSClient.LOG.error(msg);
|
||||
for (Map.Entry<InetSocketAddress, List<IOException>> dataNodeExceptions :
|
||||
exceptionMap.entrySet()) {
|
||||
List<IOException> exceptions = dataNodeExceptions.getValue();
|
||||
for (IOException ex : exceptions) {
|
||||
msg = String.format("Exception when fetching file %s at position=%d at datanode %s:", src,
|
||||
position, dataNodeExceptions.getKey());
|
||||
DFSClient.LOG.error(msg, ex);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
protected synchronized int readWithStrategy(ReaderStrategy strategy)
|
||||
throws IOException {
|
||||
dfsClient.checkOpen();
|
||||
@ -831,6 +862,9 @@ protected synchronized int readWithStrategy(ReaderStrategy strategy)
|
||||
|
||||
int len = strategy.getTargetLength();
|
||||
CorruptedBlocks corruptedBlocks = new CorruptedBlocks();
|
||||
// A map to record IOExceptions when fetching from each datanode. Key is the socketAddress of
|
||||
// a datanode.
|
||||
Map<InetSocketAddress, List<IOException>> exceptionMap = new HashMap<>();
|
||||
failures = 0;
|
||||
|
||||
maybeRegisterBlockRefresh();
|
||||
@ -852,7 +886,7 @@ protected synchronized int readWithStrategy(ReaderStrategy strategy)
|
||||
}
|
||||
}
|
||||
long beginReadMS = Time.monotonicNow();
|
||||
int result = readBuffer(strategy, realLen, corruptedBlocks);
|
||||
int result = readBuffer(strategy, realLen, corruptedBlocks, exceptionMap);
|
||||
long readTimeMS = Time.monotonicNow() - beginReadMS;
|
||||
if (result >= 0) {
|
||||
pos += result;
|
||||
@ -880,6 +914,8 @@ protected synchronized int readWithStrategy(ReaderStrategy strategy)
|
||||
dfsClient.addNodeToDeadNodeDetector(this, currentNode);
|
||||
}
|
||||
if (--retries == 0) {
|
||||
// Fail the request and log all exceptions
|
||||
logDataNodeExceptionsOnReadError(pos, exceptionMap);
|
||||
throw e;
|
||||
}
|
||||
} finally {
|
||||
@ -1122,8 +1158,8 @@ private static String getBestNodeDNAddrPairErrorString(
|
||||
return errMsgr.toString();
|
||||
}
|
||||
|
||||
protected void fetchBlockByteRange(LocatedBlock block, long start, long end,
|
||||
ByteBuffer buf, CorruptedBlocks corruptedBlocks)
|
||||
protected void fetchBlockByteRange(LocatedBlock block, long start, long end, ByteBuffer buf,
|
||||
CorruptedBlocks corruptedBlocks, final Map<InetSocketAddress, List<IOException>> exceptionMap)
|
||||
throws IOException {
|
||||
while (true) {
|
||||
DNAddrPair addressPair = chooseDataNode(block, null);
|
||||
@ -1131,7 +1167,7 @@ protected void fetchBlockByteRange(LocatedBlock block, long start, long end,
|
||||
block = addressPair.block;
|
||||
try {
|
||||
actualGetFromOneDataNode(addressPair, start, end, buf,
|
||||
corruptedBlocks);
|
||||
corruptedBlocks, exceptionMap);
|
||||
return;
|
||||
} catch (IOException e) {
|
||||
checkInterrupted(e); // check if the read has been interrupted
|
||||
@ -1142,15 +1178,15 @@ protected void fetchBlockByteRange(LocatedBlock block, long start, long end,
|
||||
}
|
||||
|
||||
private Callable<ByteBuffer> getFromOneDataNode(final DNAddrPair datanode,
|
||||
final LocatedBlock block, final long start, final long end,
|
||||
final long start, final long end,
|
||||
final ByteBuffer bb,
|
||||
final CorruptedBlocks corruptedBlocks,
|
||||
final int hedgedReadId) {
|
||||
final Map<InetSocketAddress, List<IOException>> exceptionMap) {
|
||||
return new Callable<ByteBuffer>() {
|
||||
@Override
|
||||
public ByteBuffer call() throws Exception {
|
||||
DFSClientFaultInjector.get().sleepBeforeHedgedGet();
|
||||
actualGetFromOneDataNode(datanode, start, end, bb, corruptedBlocks);
|
||||
actualGetFromOneDataNode(datanode, start, end, bb, corruptedBlocks, exceptionMap);
|
||||
return bb;
|
||||
}
|
||||
};
|
||||
@ -1167,7 +1203,8 @@ public ByteBuffer call() throws Exception {
|
||||
* block replica
|
||||
*/
|
||||
void actualGetFromOneDataNode(final DNAddrPair datanode, final long startInBlk,
|
||||
final long endInBlk, ByteBuffer buf, CorruptedBlocks corruptedBlocks)
|
||||
final long endInBlk, ByteBuffer buf, CorruptedBlocks corruptedBlocks,
|
||||
final Map<InetSocketAddress, List<IOException>> exceptionMap)
|
||||
throws IOException {
|
||||
DFSClientFaultInjector.get().startFetchFromDatanode();
|
||||
int refetchToken = 1; // only need to get a new access token once
|
||||
@ -1236,9 +1273,16 @@ void actualGetFromOneDataNode(final DNAddrPair datanode, final long startInBlk,
|
||||
// ignore IOE, since we can retry it later in a loop
|
||||
}
|
||||
} else {
|
||||
String msg = "Failed to connect to " + datanode.addr + " for file "
|
||||
+ src + " for block " + block.getBlock() + ":" + e;
|
||||
DFSClient.LOG.warn("Connection failure: " + msg, e);
|
||||
String msg = String.format("Failed to read block %s for file %s from datanode %s. "
|
||||
+ "Exception is %s. Retry with the next available datanode.",
|
||||
block.getBlock().getBlockName(), src, datanode.addr, e);
|
||||
DFSClient.LOG.warn(msg);
|
||||
|
||||
// Add the exception to the exceptionMap
|
||||
if (!exceptionMap.containsKey(datanode.addr)) {
|
||||
exceptionMap.put(datanode.addr, new LinkedList<IOException>());
|
||||
}
|
||||
exceptionMap.get(datanode.addr).add(e);
|
||||
addToLocalDeadNodes(datanode.info);
|
||||
dfsClient.addNodeToDeadNodeDetector(this, datanode.info);
|
||||
throw new IOException(msg);
|
||||
@ -1270,9 +1314,9 @@ protected LocatedBlock refreshLocatedBlock(LocatedBlock block)
|
||||
* 'hedged' read if the first read is taking longer than configured amount of
|
||||
* time. We then wait on which ever read returns first.
|
||||
*/
|
||||
private void hedgedFetchBlockByteRange(LocatedBlock block, long start,
|
||||
long end, ByteBuffer buf, CorruptedBlocks corruptedBlocks)
|
||||
throws IOException {
|
||||
private void hedgedFetchBlockByteRange(LocatedBlock block, long start, long end, ByteBuffer buf,
|
||||
CorruptedBlocks corruptedBlocks,
|
||||
final Map<InetSocketAddress, List<IOException>> exceptionMap) throws IOException {
|
||||
final DfsClientConf conf = dfsClient.getConf();
|
||||
ArrayList<Future<ByteBuffer>> futures = new ArrayList<>();
|
||||
CompletionService<ByteBuffer> hedgedService =
|
||||
@ -1280,7 +1324,6 @@ private void hedgedFetchBlockByteRange(LocatedBlock block, long start,
|
||||
ArrayList<DatanodeInfo> ignored = new ArrayList<>();
|
||||
ByteBuffer bb;
|
||||
int len = (int) (end - start + 1);
|
||||
int hedgedReadId = 0;
|
||||
while (true) {
|
||||
// see HDFS-6591, this metric is used to verify/catch unnecessary loops
|
||||
hedgedReadOpsLoopNumForTesting++;
|
||||
@ -1293,9 +1336,8 @@ private void hedgedFetchBlockByteRange(LocatedBlock block, long start,
|
||||
// Latest block, if refreshed internally
|
||||
block = chosenNode.block;
|
||||
bb = ByteBuffer.allocate(len);
|
||||
Callable<ByteBuffer> getFromDataNodeCallable = getFromOneDataNode(
|
||||
chosenNode, block, start, end, bb,
|
||||
corruptedBlocks, hedgedReadId++);
|
||||
Callable<ByteBuffer> getFromDataNodeCallable =
|
||||
getFromOneDataNode(chosenNode, start, end, bb, corruptedBlocks, exceptionMap);
|
||||
Future<ByteBuffer> firstRequest = hedgedService
|
||||
.submit(getFromDataNodeCallable);
|
||||
futures.add(firstRequest);
|
||||
@ -1335,8 +1377,7 @@ private void hedgedFetchBlockByteRange(LocatedBlock block, long start,
|
||||
block = chosenNode.block;
|
||||
bb = ByteBuffer.allocate(len);
|
||||
Callable<ByteBuffer> getFromDataNodeCallable =
|
||||
getFromOneDataNode(chosenNode, block, start, end, bb,
|
||||
corruptedBlocks, hedgedReadId++);
|
||||
getFromOneDataNode(chosenNode, start, end, bb, corruptedBlocks, exceptionMap);
|
||||
Future<ByteBuffer> oneMoreRequest =
|
||||
hedgedService.submit(getFromDataNodeCallable);
|
||||
futures.add(oneMoreRequest);
|
||||
@ -1486,6 +1527,11 @@ private int pread(long position, ByteBuffer buffer)
|
||||
List<LocatedBlock> blockRange = getBlockRange(position, realLen);
|
||||
int remaining = realLen;
|
||||
CorruptedBlocks corruptedBlocks = new CorruptedBlocks();
|
||||
// A map to record all IOExceptions happened at each datanode when fetching a block.
|
||||
// In HDFS-17332, we worked on populating this map only for DFSInputStream, but not for
|
||||
// DFSStripedInputStream. If you need the same function for DFSStripedInputStream, please
|
||||
// work on it yourself (fetchBlockByteRange() in DFSStripedInputStream).
|
||||
Map<InetSocketAddress, List<IOException>> exceptionMap = new HashMap<>();
|
||||
for (LocatedBlock blk : blockRange) {
|
||||
long targetStart = position - blk.getStartOffset();
|
||||
int bytesToRead = (int) Math.min(remaining,
|
||||
@ -1494,11 +1540,17 @@ private int pread(long position, ByteBuffer buffer)
|
||||
try {
|
||||
if (dfsClient.isHedgedReadsEnabled() && !blk.isStriped()) {
|
||||
hedgedFetchBlockByteRange(blk, targetStart,
|
||||
targetEnd, buffer, corruptedBlocks);
|
||||
targetEnd, buffer, corruptedBlocks, exceptionMap);
|
||||
} else {
|
||||
fetchBlockByteRange(blk, targetStart, targetEnd,
|
||||
buffer, corruptedBlocks);
|
||||
buffer, corruptedBlocks, exceptionMap);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
// When we reach here, it means we fail to fetch the current block from all available
|
||||
// datanodes. Send IOExceptions in exceptionMap to the log and rethrow the exception to
|
||||
// fail this request.
|
||||
logDataNodeExceptionsOnReadError(position, exceptionMap);
|
||||
throw e;
|
||||
} finally {
|
||||
// Check and report if any block replicas are corrupted.
|
||||
// BlockMissingException may be caught if all block replicas are
|
||||
@ -1507,6 +1559,8 @@ private int pread(long position, ByteBuffer buffer)
|
||||
false);
|
||||
}
|
||||
|
||||
// Reset exceptionMap before fetching the next block.
|
||||
exceptionMap.clear();
|
||||
remaining -= bytesToRead;
|
||||
position += bytesToRead;
|
||||
}
|
||||
|
@ -17,6 +17,8 @@
|
||||
*/
|
||||
package org.apache.hadoop.hdfs;
|
||||
|
||||
import java.net.InetSocketAddress;
|
||||
import java.util.Map;
|
||||
import org.apache.hadoop.classification.VisibleForTesting;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.fs.ReadOption;
|
||||
@ -479,10 +481,14 @@ private LocatedStripedBlock getBlockGroupAt(long offset) throws IOException {
|
||||
|
||||
/**
|
||||
* Real implementation of pread.
|
||||
* <p>
|
||||
* Note: exceptionMap is not populated with ioExceptions as what we added for DFSInputStream. If
|
||||
* you need this function, please implement it.
|
||||
*/
|
||||
@Override
|
||||
protected void fetchBlockByteRange(LocatedBlock block, long start,
|
||||
long end, ByteBuffer buf, CorruptedBlocks corruptedBlocks)
|
||||
long end, ByteBuffer buf, CorruptedBlocks corruptedBlocks,
|
||||
final Map<InetSocketAddress, List<IOException>> exceptionMap)
|
||||
throws IOException {
|
||||
// Refresh the striped block group
|
||||
LocatedStripedBlock blockGroup = getBlockGroupAt(block.getStartOffset());
|
||||
|
@ -17,8 +17,10 @@
|
||||
*/
|
||||
package org.apache.hadoop.hdfs;
|
||||
|
||||
import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.EOFException;
|
||||
import java.io.IOException;
|
||||
@ -35,6 +37,7 @@
|
||||
import java.util.concurrent.TimeoutException;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.ChecksumException;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
@ -52,6 +55,7 @@
|
||||
import org.apache.hadoop.test.GenericTestUtils;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.mockito.Mockito;
|
||||
import org.mockito.invocation.InvocationOnMock;
|
||||
@ -76,7 +80,12 @@ public class TestPread {
|
||||
|
||||
private static final Logger LOG =
|
||||
LoggerFactory.getLogger(TestPread.class.getName());
|
||||
|
||||
private final GenericTestUtils.LogCapturer dfsClientLog =
|
||||
GenericTestUtils.LogCapturer.captureLogs(DFSClient.LOG);
|
||||
@BeforeClass
|
||||
public static void setLogLevel() {
|
||||
GenericTestUtils.setLogLevel(DFSClient.LOG, org.apache.log4j.Level.WARN);
|
||||
}
|
||||
@Before
|
||||
public void setup() {
|
||||
simulatedStorage = false;
|
||||
@ -556,6 +565,164 @@ public Void call() throws IOException {
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test logging in getFromOneDataNode when the number of IOExceptions can be recovered by
|
||||
* retrying on a different datanode or by refreshing data nodes and retrying each data node one
|
||||
* more time.
|
||||
*/
|
||||
@Test(timeout=120000)
|
||||
public void testGetFromOneDataNodeExceptionLogging() throws IOException {
|
||||
// With maxBlockAcquireFailures = 0, we would try on each datanode only once and if
|
||||
// we fail on all three datanodes, we fail the read request.
|
||||
testGetFromOneDataNodeExceptionLogging(0, 0);
|
||||
testGetFromOneDataNodeExceptionLogging(1, 0);
|
||||
testGetFromOneDataNodeExceptionLogging(2, 0);
|
||||
|
||||
// With maxBlockAcquireFailures = 1, we will re-try each datanode a second time.
|
||||
// So, we can tolerate up to 5 datanode fetch failures.
|
||||
testGetFromOneDataNodeExceptionLogging(3, 1);
|
||||
testGetFromOneDataNodeExceptionLogging(4, 1);
|
||||
testGetFromOneDataNodeExceptionLogging(5, 1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Each failed IOException would result in a WARN log of "Failed to connect to XXX. Retry with
|
||||
* the next available datanode.". We verify the number of such log lines match the number of
|
||||
* failed DNs.
|
||||
* <p>
|
||||
* @param ioExceptions number of IOExceptions to throw during a test.
|
||||
* @param maxBlockAcquireFailures number of refreshLocation we would perform once we mark
|
||||
* all current data nodes as dead.
|
||||
*/
|
||||
private void testGetFromOneDataNodeExceptionLogging(final int ioExceptions,
|
||||
int maxBlockAcquireFailures)
|
||||
throws IOException {
|
||||
dfsClientLog.clearOutput();
|
||||
|
||||
if (ioExceptions < 0 || ioExceptions >= 3 * (maxBlockAcquireFailures+1)) {
|
||||
return;
|
||||
}
|
||||
|
||||
Configuration conf = new Configuration();
|
||||
conf.setInt(DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY, maxBlockAcquireFailures);
|
||||
final int[] count = {0};
|
||||
// Set up the InjectionHandler
|
||||
DFSClientFaultInjector.set(Mockito.mock(DFSClientFaultInjector.class));
|
||||
DFSClientFaultInjector injector = DFSClientFaultInjector.get();
|
||||
Mockito.doAnswer(new Answer<Void>() {
|
||||
@Override
|
||||
public Void answer(InvocationOnMock invocation) throws Throwable {
|
||||
if (count[0] < ioExceptions) {
|
||||
LOG.info("-------------- throw IOException " + count[0]);
|
||||
count[0]++;
|
||||
throw new IOException("IOException test");
|
||||
}
|
||||
return null;
|
||||
}
|
||||
}).when(injector).fetchFromDatanodeException();
|
||||
|
||||
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).format(true).build();
|
||||
DistributedFileSystem fileSys = cluster.getFileSystem();
|
||||
DFSClient dfsClient = fileSys.getClient();
|
||||
DFSInputStream input = null;
|
||||
Path file = new Path("/testfile.dat");
|
||||
|
||||
try {
|
||||
DFSTestUtil.createFile(fileSys, file, fileSize, fileSize, blockSize, (short) 3, seed);
|
||||
|
||||
byte[] buffer = new byte[fileSize];
|
||||
input = dfsClient.open(file.toString());
|
||||
input.read(0, buffer, 0, fileSize);
|
||||
assertEquals(ioExceptions, StringUtils.countMatches(dfsClientLog.getOutput(),
|
||||
"Retry with the next available datanode."));
|
||||
} finally {
|
||||
Mockito.reset(injector);
|
||||
IOUtils.cleanupWithLogger(LOG, input);
|
||||
fileSys.close();
|
||||
cluster.shutdown();
|
||||
dfsClientLog.clearOutput();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test the case where we always hit IOExceptions, causing the read request to fail.
|
||||
*/
|
||||
@Test(timeout=60000)
|
||||
public void testFetchFromDataNodeExceptionLoggingFailedRequest()
|
||||
throws IOException {
|
||||
testFetchFromDataNodeExceptionLoggingFailedRequest(0);
|
||||
testFetchFromDataNodeExceptionLoggingFailedRequest(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* We verify that BlockMissingException is threw and there is one ERROR log line of
|
||||
* "Failed to read from all available datanodes for file"
|
||||
* and 3 * (maxBlockAcquireFailures+1) ERROR log lines of
|
||||
* "Exception when fetching file /testfile.dat at position".
|
||||
* <p>
|
||||
* maxBlockAcquireFailures determines how many times we can retry when we fail to read from
|
||||
* all three data nodes.
|
||||
* <ul>
|
||||
* <li>maxBlockAcquireFailures = 0: no retry. We will only read from each of the three
|
||||
* data nodes only once. We expect to see 3 ERROR log lines of "Exception when fetching file
|
||||
* /testfile.dat at position".
|
||||
* </li>
|
||||
* <li>maxBlockAcquireFailures = 1: 1 retry. We will read from each of the three data
|
||||
* nodes twice. We expect to see 6 ERROR log lines of "Exception when fetching file
|
||||
* /testfile.dat at position".
|
||||
* </li>
|
||||
* </ul>
|
||||
*/
|
||||
private void testFetchFromDataNodeExceptionLoggingFailedRequest(int maxBlockAcquireFailures)
|
||||
throws IOException {
|
||||
dfsClientLog.clearOutput();
|
||||
|
||||
Configuration conf = new Configuration();
|
||||
conf.setInt(DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY, maxBlockAcquireFailures);
|
||||
// Set up the InjectionHandler
|
||||
DFSClientFaultInjector.set(Mockito.mock(DFSClientFaultInjector.class));
|
||||
DFSClientFaultInjector injector = DFSClientFaultInjector.get();
|
||||
Mockito.doAnswer(new Answer<Void>() {
|
||||
@Override
|
||||
public Void answer(InvocationOnMock invocation) throws Throwable {
|
||||
LOG.info("-------------- throw IOException ");
|
||||
throw new IOException("IOException test");
|
||||
}
|
||||
}).when(injector).fetchFromDatanodeException();
|
||||
|
||||
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).format(true).build();
|
||||
DistributedFileSystem fileSys = cluster.getFileSystem();
|
||||
DFSClient dfsClient = fileSys.getClient();
|
||||
DFSInputStream input = null;
|
||||
Path file = new Path("/testfile.dat");
|
||||
|
||||
try {
|
||||
DFSTestUtil.createFile(fileSys, file, fileSize, fileSize, blockSize, (short) 3, seed);
|
||||
|
||||
byte[] buffer = new byte[fileSize];
|
||||
input = dfsClient.open(file.toString());
|
||||
input.read(0, buffer, 0, fileSize);
|
||||
fail();
|
||||
} catch (BlockMissingException expected) {
|
||||
// Logging from pread
|
||||
assertEquals(1, StringUtils.countMatches(dfsClientLog.getOutput(),
|
||||
"Failed to read from all available datanodes for file"));
|
||||
assertEquals(3 * (maxBlockAcquireFailures + 1),
|
||||
StringUtils.countMatches(dfsClientLog.getOutput(),
|
||||
"Exception when fetching file /testfile.dat at position"));
|
||||
// Logging from actualGetFromOneDataNode
|
||||
assertEquals(3 * (maxBlockAcquireFailures + 1),
|
||||
StringUtils.countMatches(dfsClientLog.getOutput(),
|
||||
"Retry with the next available datanode."));
|
||||
} finally {
|
||||
Mockito.reset(injector);
|
||||
IOUtils.cleanupWithLogger(LOG, input);
|
||||
fileSys.close();
|
||||
cluster.shutdown();
|
||||
dfsClientLog.clearOutput();
|
||||
}
|
||||
}
|
||||
|
||||
@Test(timeout=30000)
|
||||
public void testHedgedReadFromAllDNFailed() throws IOException {
|
||||
Configuration conf = new Configuration();
|
||||
|
@ -25,6 +25,7 @@
|
||||
import java.nio.channels.ClosedByInterruptException;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
||||
@ -32,6 +33,8 @@
|
||||
import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
|
||||
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.test.GenericTestUtils;
|
||||
import org.apache.log4j.Level;
|
||||
import org.junit.Assert;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
@ -39,10 +42,32 @@
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.DFSTestUtil.ShortCircuitTestContext;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.mockito.Mockito;
|
||||
import org.mockito.invocation.InvocationOnMock;
|
||||
import org.mockito.stubbing.Answer;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
|
||||
public class TestRead {
|
||||
final private int BLOCK_SIZE = 512;
|
||||
static final private int BLOCK_SIZE = 512;
|
||||
static final long SEED = 0xDEADBEEFL;
|
||||
static final int FILE_SIZE = BLOCK_SIZE * 10;
|
||||
private static final Logger LOG =
|
||||
LoggerFactory.getLogger(TestRead.class.getName());
|
||||
private final GenericTestUtils.LogCapturer dfsClientLog =
|
||||
GenericTestUtils.LogCapturer.captureLogs(DFSClient.LOG);
|
||||
|
||||
@BeforeClass
|
||||
public static void setLogLevel() {
|
||||
GenericTestUtils.setLogLevel(DFSClient.LOG, Level.WARN);
|
||||
}
|
||||
|
||||
private void testEOF(MiniDFSCluster cluster, int fileLength) throws IOException {
|
||||
FileSystem fs = cluster.getFileSystem();
|
||||
@ -190,4 +215,126 @@ public boolean isSimulated() {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test logging in readBuffer() when the number of IOExceptions can be recovered by retrying on
|
||||
* a different datanode or by refreshing data nodes and retrying each data node one more time.
|
||||
*/
|
||||
@Test(timeout=120000)
|
||||
public void testReadBufferIOExceptionLogging() throws IOException {
|
||||
testReadBufferIOExceptionLogging(0, 0);
|
||||
testReadBufferIOExceptionLogging(1, 0);
|
||||
testReadBufferIOExceptionLogging(2, 0);
|
||||
testReadBufferIOExceptionLogging(3, 0);
|
||||
testReadBufferIOExceptionLogging(4, 1);
|
||||
testReadBufferIOExceptionLogging(5, 1);
|
||||
testReadBufferIOExceptionLogging(6, 1);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param ioExceptions number of IOExceptions to throw during a test.
|
||||
* @param maxBlockAcquireFailures number of refreshLocation we would perform once we mark
|
||||
* all current data nodes as dead.
|
||||
*/
|
||||
private void testReadBufferIOExceptionLogging(final int ioExceptions,
|
||||
int maxBlockAcquireFailures) throws IOException {
|
||||
dfsClientLog.clearOutput();
|
||||
Configuration conf = new Configuration();
|
||||
conf.setInt(DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY, maxBlockAcquireFailures);
|
||||
final int[] count = {0};
|
||||
// Set up the InjectionHandler
|
||||
DFSClientFaultInjector.set(Mockito.mock(DFSClientFaultInjector.class));
|
||||
DFSClientFaultInjector injector = DFSClientFaultInjector.get();
|
||||
Mockito.doAnswer(new Answer<Void>() {
|
||||
@Override
|
||||
public Void answer(InvocationOnMock invocation) throws Throwable {
|
||||
if (count[0] < ioExceptions) {
|
||||
LOG.info("-------------- throw IOException");
|
||||
count[0]++;
|
||||
throw new IOException("IOException test");
|
||||
}
|
||||
return null;
|
||||
}
|
||||
}).when(injector).fetchFromDatanodeException();
|
||||
|
||||
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).format(true).build();
|
||||
DistributedFileSystem fileSys = cluster.getFileSystem();
|
||||
DFSClient dfsClient = fileSys.getClient();
|
||||
DFSInputStream input = null;
|
||||
Path file = new Path("/testfile.dat");
|
||||
|
||||
try {
|
||||
DFSTestUtil.createFile(fileSys, file, FILE_SIZE, FILE_SIZE, BLOCK_SIZE, (short) 3, SEED);
|
||||
|
||||
byte[] buffer = new byte[FILE_SIZE];
|
||||
input = dfsClient.open(file.toString());
|
||||
input.read(buffer, 0, FILE_SIZE);
|
||||
assertEquals(ioExceptions, StringUtils.countMatches(dfsClientLog.getOutput(),
|
||||
"Retry with the current or next available datanode."));
|
||||
} finally {
|
||||
Mockito.reset(injector);
|
||||
IOUtils.cleanupWithLogger(LOG, input);
|
||||
fileSys.close();
|
||||
cluster.shutdown();
|
||||
dfsClientLog.clearOutput();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test the case where we always hit IOExceptions, causing the read request to fail.
|
||||
*/
|
||||
@Test(timeout=60000)
|
||||
public void testReadBufferIOExceptionLoggingFailedRequest() throws IOException {
|
||||
testReadBufferIOExceptionLoggingFailedRequest(0);
|
||||
testReadBufferIOExceptionLoggingFailedRequest(1);
|
||||
}
|
||||
|
||||
private void testReadBufferIOExceptionLoggingFailedRequest(int maxBlockAcquireFailures)
|
||||
throws IOException {
|
||||
dfsClientLog.clearOutput();
|
||||
Configuration conf = new Configuration();
|
||||
conf.setInt(DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY, maxBlockAcquireFailures);
|
||||
// Set up the InjectionHandler
|
||||
DFSClientFaultInjector.set(Mockito.mock(DFSClientFaultInjector.class));
|
||||
DFSClientFaultInjector injector = DFSClientFaultInjector.get();
|
||||
Mockito.doAnswer(new Answer<Void>() {
|
||||
@Override
|
||||
public Void answer(InvocationOnMock invocation) throws Throwable {
|
||||
LOG.info("-------------- throw IOException");
|
||||
throw new IOException("IOException test");
|
||||
}
|
||||
}).when(injector).fetchFromDatanodeException();
|
||||
|
||||
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).format(true).build();
|
||||
DistributedFileSystem fileSys = cluster.getFileSystem();
|
||||
DFSClient dfsClient = fileSys.getClient();
|
||||
DFSInputStream input = null;
|
||||
Path file = new Path("/testfile.dat");
|
||||
|
||||
try {
|
||||
DFSTestUtil.createFile(fileSys, file, FILE_SIZE, FILE_SIZE, BLOCK_SIZE, (short) 3, SEED);
|
||||
|
||||
byte[] buffer = new byte[FILE_SIZE];
|
||||
input = dfsClient.open(file.toString());
|
||||
input.read(buffer, 0, FILE_SIZE);
|
||||
fail();
|
||||
} catch (BlockMissingException e) {
|
||||
// Logging from readWithStrategy()
|
||||
assertEquals(1, StringUtils.countMatches(dfsClientLog.getOutput(),
|
||||
"Failed to read from all available datanodes for file"));
|
||||
assertEquals(1 + 3L * (maxBlockAcquireFailures + 1),
|
||||
StringUtils.countMatches(dfsClientLog.getOutput(),
|
||||
"Exception when fetching file /testfile.dat at position="));
|
||||
// Logging from actualGetFromOneDataNode
|
||||
assertEquals(1 + 3L * (maxBlockAcquireFailures + 1),
|
||||
StringUtils.countMatches(dfsClientLog.getOutput(),
|
||||
"Retry with the current or next available datanode."));
|
||||
} finally {
|
||||
Mockito.reset(injector);
|
||||
IOUtils.cleanupWithLogger(LOG, input);
|
||||
fileSys.close();
|
||||
cluster.shutdown();
|
||||
dfsClientLog.clearOutput();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user