|
@@ -19,8 +19,10 @@ package org.apache.hadoop.hdfs;
|
|
|
|
|
|
import java.io.EOFException;
|
|
|
import java.io.IOException;
|
|
|
+import java.io.InterruptedIOException;
|
|
|
import java.net.InetSocketAddress;
|
|
|
import java.nio.ByteBuffer;
|
|
|
+import java.nio.channels.ClosedByInterruptException;
|
|
|
import java.util.AbstractMap;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
@@ -307,7 +309,7 @@ public class DFSInputStream extends FSInputStream
|
|
|
try {
|
|
|
Thread.sleep(waitTime);
|
|
|
} catch (InterruptedException e) {
|
|
|
- throw new IOException(
|
|
|
+ throw new InterruptedIOException(
|
|
|
"Interrupted while getting the last block length.");
|
|
|
}
|
|
|
}
|
|
@@ -382,6 +384,7 @@ public class DFSInputStream extends FSInputStream
|
|
|
return n;
|
|
|
}
|
|
|
} catch (IOException ioe) {
|
|
|
+ checkInterrupted(ioe);
|
|
|
if (ioe instanceof RemoteException) {
|
|
|
if (((RemoteException) ioe).unwrapRemoteException() instanceof
|
|
|
ReplicaNotFoundException) {
|
|
@@ -417,7 +420,8 @@ public class DFSInputStream extends FSInputStream
|
|
|
try {
|
|
|
Thread.sleep(500); // delay between retries.
|
|
|
} catch (InterruptedException e) {
|
|
|
- throw new IOException("Interrupted while getting the length.");
|
|
|
+ throw new InterruptedIOException(
|
|
|
+ "Interrupted while getting the length.");
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -663,6 +667,7 @@ public class DFSInputStream extends FSInputStream
|
|
|
}
|
|
|
return chosenNode;
|
|
|
} catch (IOException ex) {
|
|
|
+ checkInterrupted(ex);
|
|
|
if (ex instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
|
|
|
DFSClient.LOG.info("Will fetch a new encryption key and retry, "
|
|
|
+ "encryption key was invalid when connecting to " + targetAddr
|
|
@@ -684,6 +689,15 @@ public class DFSInputStream extends FSInputStream
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ private void checkInterrupted(IOException e) throws IOException {
|
|
|
+ if (Thread.currentThread().isInterrupted() &&
|
|
|
+ (e instanceof ClosedByInterruptException ||
|
|
|
+ e instanceof InterruptedIOException)) {
|
|
|
+ DFSClient.LOG.debug("The reading thread has been interrupted.", e);
|
|
|
+ throw e;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
protected BlockReader getBlockReader(LocatedBlock targetBlock,
|
|
|
long offsetInBlock, long length, InetSocketAddress targetAddr,
|
|
|
StorageType storageType, DatanodeInfo datanode) throws IOException {
|
|
@@ -950,6 +964,7 @@ public class DFSInputStream extends FSInputStream
|
|
|
} catch (ChecksumException ce) {
|
|
|
throw ce;
|
|
|
} catch (IOException e) {
|
|
|
+ checkInterrupted(e);
|
|
|
if (retries == 1) {
|
|
|
DFSClient.LOG.warn("DFS Read", e);
|
|
|
}
|
|
@@ -1064,9 +1079,12 @@ public class DFSInputStream extends FSInputStream
|
|
|
// expanding time window for each failure
|
|
|
timeWindow * (failures + 1) *
|
|
|
ThreadLocalRandom.current().nextDouble();
|
|
|
- DFSClient.LOG.warn("DFS chooseDataNode: got # " + (failures + 1) + " IOException, will wait for " + waitTime + " msec.");
|
|
|
+ DFSClient.LOG.warn("DFS chooseDataNode: got # " + (failures + 1) +
|
|
|
+ " IOException, will wait for " + waitTime + " msec.");
|
|
|
Thread.sleep((long)waitTime);
|
|
|
- } catch (InterruptedException ignored) {
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ throw new InterruptedIOException(
|
|
|
+ "Interrupted while choosing DataNode for read.");
|
|
|
}
|
|
|
deadNodes.clear(); //2nd option is to remove only nodes[blockId]
|
|
|
openInfo(true);
|
|
@@ -1152,7 +1170,8 @@ public class DFSInputStream extends FSInputStream
|
|
|
buf, offset, corruptedBlockMap);
|
|
|
return;
|
|
|
} catch (IOException e) {
|
|
|
- // Ignore. Already processed inside the function.
|
|
|
+ checkInterrupted(e); // check if the read has been interrupted
|
|
|
+ // Ignore other IOException. Already processed inside the function.
|
|
|
// Loop through to try the next node.
|
|
|
}
|
|
|
}
|
|
@@ -1249,6 +1268,7 @@ public class DFSInputStream extends FSInputStream
|
|
|
addToDeadNodes(datanode.info);
|
|
|
throw new IOException(msg);
|
|
|
} catch (IOException e) {
|
|
|
+ checkInterrupted(e);
|
|
|
if (e instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
|
|
|
DFSClient.LOG.info("Will fetch a new encryption key and retry, "
|
|
|
+ "encryption key was invalid when connecting to " + datanode.addr
|
|
@@ -1356,8 +1376,11 @@ public class DFSInputStream extends FSInputStream
|
|
|
ignored.add(chosenNode.info);
|
|
|
dfsClient.getHedgedReadMetrics().incHedgedReadOps();
|
|
|
// continue; no need to refresh block locations
|
|
|
- } catch (InterruptedException | ExecutionException e) {
|
|
|
+ } catch (ExecutionException e) {
|
|
|
// Ignore
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ throw new InterruptedIOException(
|
|
|
+ "Interrupted while waiting for reading task");
|
|
|
}
|
|
|
} else {
|
|
|
// We are starting up a 'hedged' read. We have a read already
|
|
@@ -1630,6 +1653,7 @@ public class DFSInputStream extends FSInputStream
|
|
|
} catch (IOException e) {//make following read to retry
|
|
|
DFSClient.LOG.debug("Exception while seek to {} from {} of {} from "
|
|
|
+ "{}", targetPos, getCurrentBlock(), src, currentNode, e);
|
|
|
+ checkInterrupted(e);
|
|
|
}
|
|
|
}
|
|
|
}
|