|
@@ -28,6 +28,7 @@ import java.util.ArrayList;
|
|
import java.util.Arrays;
|
|
import java.util.Arrays;
|
|
import java.util.Collection;
|
|
import java.util.Collection;
|
|
import java.util.EnumSet;
|
|
import java.util.EnumSet;
|
|
|
|
+import java.util.HashMap;
|
|
import java.util.Iterator;
|
|
import java.util.Iterator;
|
|
import java.util.LinkedList;
|
|
import java.util.LinkedList;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
@@ -65,6 +66,7 @@ import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
|
|
import org.apache.hadoop.hdfs.protocol.BlockType;
|
|
import org.apache.hadoop.hdfs.protocol.BlockType;
|
|
import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
|
|
import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
|
+import org.apache.hadoop.hdfs.protocol.DatanodeInfoWithStorage;
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
@@ -128,18 +130,18 @@ public class DFSInputStream extends FSInputStream
|
|
private long lastBlockBeingWrittenLength = 0;
|
|
private long lastBlockBeingWrittenLength = 0;
|
|
private FileEncryptionInfo fileEncryptionInfo = null;
|
|
private FileEncryptionInfo fileEncryptionInfo = null;
|
|
protected CachingStrategy cachingStrategy;
|
|
protected CachingStrategy cachingStrategy;
|
|
|
|
+ // this is volatile because it will be polled outside the lock,
|
|
|
|
+ // but still only updated within the lock
|
|
|
|
+ private volatile long lastRefreshedBlocksAt = Time.monotonicNow();
|
|
////
|
|
////
|
|
|
|
|
|
|
|
+ private AtomicBoolean refreshingBlockLocations = new AtomicBoolean(false);
|
|
protected final ReadStatistics readStatistics = new ReadStatistics();
|
|
protected final ReadStatistics readStatistics = new ReadStatistics();
|
|
// lock for state shared between read and pread
|
|
// lock for state shared between read and pread
|
|
// Note: Never acquire a lock on <this> with this lock held to avoid deadlocks
|
|
// Note: Never acquire a lock on <this> with this lock held to avoid deadlocks
|
|
// (it's OK to acquire this lock when the lock on <this> is held)
|
|
// (it's OK to acquire this lock when the lock on <this> is held)
|
|
protected final Object infoLock = new Object();
|
|
protected final Object infoLock = new Object();
|
|
|
|
|
|
- // refresh locatedBlocks periodically
|
|
|
|
- private final long refreshReadBlockIntervals;
|
|
|
|
- /** timeStamp of the last time a block location was refreshed. */
|
|
|
|
- private long locatedBlocksTimeStamp;
|
|
|
|
/**
|
|
/**
|
|
* Track the ByteBuffers that we have handed out to readers.
|
|
* Track the ByteBuffers that we have handed out to readers.
|
|
*
|
|
*
|
|
@@ -156,10 +158,6 @@ public class DFSInputStream extends FSInputStream
|
|
return extendedReadBuffers;
|
|
return extendedReadBuffers;
|
|
}
|
|
}
|
|
|
|
|
|
- private boolean isPeriodicRefreshEnabled() {
|
|
|
|
- return (refreshReadBlockIntervals > 0L);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
/**
|
|
/**
|
|
* This variable tracks the number of failures since the start of the
|
|
* This variable tracks the number of failures since the start of the
|
|
* most recent user-facing operation. That is to say, it should be reset
|
|
* most recent user-facing operation. That is to say, it should be reset
|
|
@@ -206,9 +204,6 @@ public class DFSInputStream extends FSInputStream
|
|
DFSInputStream(DFSClient dfsClient, String src, boolean verifyChecksum,
|
|
DFSInputStream(DFSClient dfsClient, String src, boolean verifyChecksum,
|
|
LocatedBlocks locatedBlocks) throws IOException {
|
|
LocatedBlocks locatedBlocks) throws IOException {
|
|
this.dfsClient = dfsClient;
|
|
this.dfsClient = dfsClient;
|
|
- this.refreshReadBlockIntervals =
|
|
|
|
- this.dfsClient.getRefreshReadBlkLocationsInterval();
|
|
|
|
- setLocatedBlocksTimeStamp();
|
|
|
|
this.verifyChecksum = verifyChecksum;
|
|
this.verifyChecksum = verifyChecksum;
|
|
this.src = src;
|
|
this.src = src;
|
|
synchronized (infoLock) {
|
|
synchronized (infoLock) {
|
|
@@ -228,19 +223,6 @@ public class DFSInputStream extends FSInputStream
|
|
return deadNodes.containsKey(nodeInfo);
|
|
return deadNodes.containsKey(nodeInfo);
|
|
}
|
|
}
|
|
|
|
|
|
- @VisibleForTesting
|
|
|
|
- void setReadTimeStampsForTesting(long timeStamp) {
|
|
|
|
- setLocatedBlocksTimeStamp(timeStamp);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- private void setLocatedBlocksTimeStamp() {
|
|
|
|
- setLocatedBlocksTimeStamp(Time.monotonicNow());
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- private void setLocatedBlocksTimeStamp(long timeStamp) {
|
|
|
|
- this.locatedBlocksTimeStamp = timeStamp;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
/**
|
|
/**
|
|
* Grab the open-file info from namenode
|
|
* Grab the open-file info from namenode
|
|
* @param refreshLocatedBlocks whether to re-fetch locatedblocks
|
|
* @param refreshLocatedBlocks whether to re-fetch locatedblocks
|
|
@@ -248,33 +230,50 @@ public class DFSInputStream extends FSInputStream
|
|
void openInfo(boolean refreshLocatedBlocks) throws IOException {
|
|
void openInfo(boolean refreshLocatedBlocks) throws IOException {
|
|
final DfsClientConf conf = dfsClient.getConf();
|
|
final DfsClientConf conf = dfsClient.getConf();
|
|
synchronized(infoLock) {
|
|
synchronized(infoLock) {
|
|
- lastBlockBeingWrittenLength =
|
|
|
|
- fetchLocatedBlocksAndGetLastBlockLength(refreshLocatedBlocks);
|
|
|
|
int retriesForLastBlockLength = conf.getRetryTimesForGetLastBlockLength();
|
|
int retriesForLastBlockLength = conf.getRetryTimesForGetLastBlockLength();
|
|
- while (retriesForLastBlockLength > 0) {
|
|
|
|
|
|
+
|
|
|
|
+ while (true) {
|
|
|
|
+ LocatedBlocks newLocatedBlocks;
|
|
|
|
+ if (locatedBlocks == null || refreshLocatedBlocks) {
|
|
|
|
+ newLocatedBlocks = fetchAndCheckLocatedBlocks(locatedBlocks);
|
|
|
|
+ } else {
|
|
|
|
+ newLocatedBlocks = locatedBlocks;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ long lastBlockLength = getLastBlockLength(newLocatedBlocks);
|
|
|
|
+ if (lastBlockLength != -1) {
|
|
|
|
+ setLocatedBlocksFields(newLocatedBlocks, lastBlockLength);
|
|
|
|
+ return;
|
|
|
|
+ }
|
|
|
|
+
|
|
// Getting last block length as -1 is a special case. When cluster
|
|
// Getting last block length as -1 is a special case. When cluster
|
|
// restarts, DNs may not report immediately. At this time partial block
|
|
// restarts, DNs may not report immediately. At this time partial block
|
|
// locations will not be available with NN for getting the length. Lets
|
|
// locations will not be available with NN for getting the length. Lets
|
|
// retry for 3 times to get the length.
|
|
// retry for 3 times to get the length.
|
|
- if (lastBlockBeingWrittenLength == -1) {
|
|
|
|
- DFSClient.LOG.warn("Last block locations not available. "
|
|
|
|
- + "Datanodes might not have reported blocks completely."
|
|
|
|
- + " Will retry for " + retriesForLastBlockLength + " times");
|
|
|
|
- waitFor(conf.getRetryIntervalForGetLastBlockLength());
|
|
|
|
- lastBlockBeingWrittenLength =
|
|
|
|
- fetchLocatedBlocksAndGetLastBlockLength(true);
|
|
|
|
- } else {
|
|
|
|
- break;
|
|
|
|
|
|
+
|
|
|
|
+ if (retriesForLastBlockLength-- <= 0) {
|
|
|
|
+ throw new IOException("Could not obtain the last block locations.");
|
|
}
|
|
}
|
|
- retriesForLastBlockLength--;
|
|
|
|
- }
|
|
|
|
- if (lastBlockBeingWrittenLength == -1
|
|
|
|
- && retriesForLastBlockLength == 0) {
|
|
|
|
- throw new IOException("Could not obtain the last block locations.");
|
|
|
|
|
|
+
|
|
|
|
+ DFSClient.LOG.warn("Last block locations not available. "
|
|
|
|
+ + "Datanodes might not have reported blocks completely."
|
|
|
|
+ + " Will retry for " + retriesForLastBlockLength + " times");
|
|
|
|
+ waitFor(conf.getRetryIntervalForGetLastBlockLength());
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Set locatedBlocks and related fields, using the passed lastBlockLength.
|
|
|
|
+ * Should be called within infoLock.
|
|
|
|
+ */
|
|
|
|
+ private void setLocatedBlocksFields(LocatedBlocks locatedBlocksToSet, long lastBlockLength) {
|
|
|
|
+ locatedBlocks = locatedBlocksToSet;
|
|
|
|
+ lastBlockBeingWrittenLength = lastBlockLength;
|
|
|
|
+ fileEncryptionInfo = locatedBlocks.getFileEncryptionInfo();
|
|
|
|
+ setLastRefreshedBlocksAt();
|
|
|
|
+ }
|
|
|
|
+
|
|
private void waitFor(int waitTime) throws IOException {
|
|
private void waitFor(int waitTime) throws IOException {
|
|
try {
|
|
try {
|
|
Thread.sleep(waitTime);
|
|
Thread.sleep(waitTime);
|
|
@@ -285,62 +284,18 @@ public class DFSInputStream extends FSInputStream
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- /**
|
|
|
|
- * Checks whether the block locations timestamps have expired.
|
|
|
|
- * In the case of expired timestamp:
|
|
|
|
- * - clear list of deadNodes
|
|
|
|
- * - call openInfo(true) which will re-fetch locatedblocks
|
|
|
|
- * - update locatedBlocksTimeStamp
|
|
|
|
- * @return true when the expiration feature is enabled and locatedblocks
|
|
|
|
- * timestamp has expired.
|
|
|
|
- * @throws IOException
|
|
|
|
- */
|
|
|
|
- private boolean isLocatedBlocksExpired() {
|
|
|
|
- if (!isPeriodicRefreshEnabled()) {
|
|
|
|
- return false;
|
|
|
|
- }
|
|
|
|
- long now = Time.monotonicNow();
|
|
|
|
- long elapsed = now - locatedBlocksTimeStamp;
|
|
|
|
- if (elapsed < refreshReadBlockIntervals) {
|
|
|
|
- return false;
|
|
|
|
- }
|
|
|
|
- return true;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Update the block locations timestamps if they have expired.
|
|
|
|
- * In the case of expired timestamp:
|
|
|
|
- * - clear list of deadNodes
|
|
|
|
- * - call openInfo(true) which will re-fetch locatedblocks
|
|
|
|
- * - update locatedBlocksTimeStamp
|
|
|
|
- * @return true when the locatedblocks list is re-fetched from the namenode.
|
|
|
|
- * @throws IOException
|
|
|
|
- */
|
|
|
|
- private boolean updateBlockLocationsStamp() throws IOException {
|
|
|
|
- if (!isLocatedBlocksExpired()) {
|
|
|
|
- return false;
|
|
|
|
- }
|
|
|
|
- // clear dead nodes
|
|
|
|
- deadNodes.clear();
|
|
|
|
- openInfo(true);
|
|
|
|
- setLocatedBlocksTimeStamp();
|
|
|
|
- return true;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- private long fetchLocatedBlocksAndGetLastBlockLength(boolean refresh)
|
|
|
|
|
|
+ private LocatedBlocks fetchAndCheckLocatedBlocks(LocatedBlocks existing)
|
|
throws IOException {
|
|
throws IOException {
|
|
- LocatedBlocks newInfo = locatedBlocks;
|
|
|
|
- if (locatedBlocks == null || refresh) {
|
|
|
|
- newInfo = dfsClient.getLocatedBlocks(src, 0);
|
|
|
|
- }
|
|
|
|
|
|
+ LocatedBlocks newInfo = dfsClient.getLocatedBlocks(src, 0);
|
|
|
|
+
|
|
DFSClient.LOG.debug("newInfo = {}", newInfo);
|
|
DFSClient.LOG.debug("newInfo = {}", newInfo);
|
|
if (newInfo == null) {
|
|
if (newInfo == null) {
|
|
throw new IOException("Cannot open filename " + src);
|
|
throw new IOException("Cannot open filename " + src);
|
|
}
|
|
}
|
|
|
|
|
|
- if (locatedBlocks != null) {
|
|
|
|
|
|
+ if (existing != null) {
|
|
Iterator<LocatedBlock> oldIter =
|
|
Iterator<LocatedBlock> oldIter =
|
|
- locatedBlocks.getLocatedBlocks().iterator();
|
|
|
|
|
|
+ existing.getLocatedBlocks().iterator();
|
|
Iterator<LocatedBlock> newIter = newInfo.getLocatedBlocks().iterator();
|
|
Iterator<LocatedBlock> newIter = newInfo.getLocatedBlocks().iterator();
|
|
while (oldIter.hasNext() && newIter.hasNext()) {
|
|
while (oldIter.hasNext() && newIter.hasNext()) {
|
|
if (!oldIter.next().getBlock().equals(newIter.next().getBlock())) {
|
|
if (!oldIter.next().getBlock().equals(newIter.next().getBlock())) {
|
|
@@ -348,17 +303,14 @@ public class DFSInputStream extends FSInputStream
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
- locatedBlocks = newInfo;
|
|
|
|
- long lastBlkBeingWrittenLength = getLastBlockLength();
|
|
|
|
- fileEncryptionInfo = locatedBlocks.getFileEncryptionInfo();
|
|
|
|
|
|
|
|
- return lastBlkBeingWrittenLength;
|
|
|
|
|
|
+ return newInfo;
|
|
}
|
|
}
|
|
|
|
|
|
- private long getLastBlockLength() throws IOException{
|
|
|
|
|
|
+ private long getLastBlockLength(LocatedBlocks blocks) throws IOException{
|
|
long lastBlockBeingWrittenLength = 0;
|
|
long lastBlockBeingWrittenLength = 0;
|
|
- if (!locatedBlocks.isLastBlockComplete()) {
|
|
|
|
- final LocatedBlock last = locatedBlocks.getLastLocatedBlock();
|
|
|
|
|
|
+ if (!blocks.isLastBlockComplete()) {
|
|
|
|
+ final LocatedBlock last = blocks.getLastLocatedBlock();
|
|
if (last != null) {
|
|
if (last != null) {
|
|
if (last.getLocations().length == 0) {
|
|
if (last.getLocations().length == 0) {
|
|
if (last.getBlockSize() == 0) {
|
|
if (last.getBlockSize() == 0) {
|
|
@@ -501,6 +453,14 @@ public class DFSInputStream extends FSInputStream
|
|
return getBlockRange(0, getFileLength());
|
|
return getBlockRange(0, getFileLength());
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ protected String getSrc() {
|
|
|
|
+ return src;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ protected LocatedBlocks getLocatedBlocks() {
|
|
|
|
+ return locatedBlocks;
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Get block at the specified position.
|
|
* Get block at the specified position.
|
|
* Fetch it from the namenode if not cached.
|
|
* Fetch it from the namenode if not cached.
|
|
@@ -543,8 +503,8 @@ public class DFSInputStream extends FSInputStream
|
|
/** Fetch a block from namenode and cache it */
|
|
/** Fetch a block from namenode and cache it */
|
|
private LocatedBlock fetchBlockAt(long offset, long length, boolean useCache)
|
|
private LocatedBlock fetchBlockAt(long offset, long length, boolean useCache)
|
|
throws IOException {
|
|
throws IOException {
|
|
|
|
+ maybeRegisterBlockRefresh();
|
|
synchronized(infoLock) {
|
|
synchronized(infoLock) {
|
|
- updateBlockLocationsStamp();
|
|
|
|
int targetBlockIdx = locatedBlocks.findBlock(offset);
|
|
int targetBlockIdx = locatedBlocks.findBlock(offset);
|
|
if (targetBlockIdx < 0) { // block is not cached
|
|
if (targetBlockIdx < 0) { // block is not cached
|
|
targetBlockIdx = LocatedBlocks.getInsertIndex(targetBlockIdx);
|
|
targetBlockIdx = LocatedBlocks.getInsertIndex(targetBlockIdx);
|
|
@@ -559,8 +519,7 @@ public class DFSInputStream extends FSInputStream
|
|
}
|
|
}
|
|
// Update the LastLocatedBlock, if offset is for last block.
|
|
// Update the LastLocatedBlock, if offset is for last block.
|
|
if (offset >= locatedBlocks.getFileLength()) {
|
|
if (offset >= locatedBlocks.getFileLength()) {
|
|
- locatedBlocks = newBlocks;
|
|
|
|
- lastBlockBeingWrittenLength = getLastBlockLength();
|
|
|
|
|
|
+ setLocatedBlocksFields(newBlocks, getLastBlockLength(newBlocks));
|
|
} else {
|
|
} else {
|
|
locatedBlocks.insertRange(targetBlockIdx,
|
|
locatedBlocks.insertRange(targetBlockIdx,
|
|
newBlocks.getLocatedBlocks());
|
|
newBlocks.getLocatedBlocks());
|
|
@@ -587,6 +546,7 @@ public class DFSInputStream extends FSInputStream
|
|
throw new IOException("Offset: " + offset +
|
|
throw new IOException("Offset: " + offset +
|
|
" exceeds file length: " + getFileLength());
|
|
" exceeds file length: " + getFileLength());
|
|
}
|
|
}
|
|
|
|
+
|
|
synchronized(infoLock) {
|
|
synchronized(infoLock) {
|
|
final List<LocatedBlock> blocks;
|
|
final List<LocatedBlock> blocks;
|
|
final long lengthOfCompleteBlk = locatedBlocks.getFileLength();
|
|
final long lengthOfCompleteBlk = locatedBlocks.getFileLength();
|
|
@@ -644,6 +604,9 @@ public class DFSInputStream extends FSInputStream
|
|
if (target >= getFileLength()) {
|
|
if (target >= getFileLength()) {
|
|
throw new IOException("Attempted to read past end of file");
|
|
throw new IOException("Attempted to read past end of file");
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ maybeRegisterBlockRefresh();
|
|
|
|
+
|
|
// Will be getting a new BlockReader.
|
|
// Will be getting a new BlockReader.
|
|
closeCurrentBlockReaders();
|
|
closeCurrentBlockReaders();
|
|
|
|
|
|
@@ -657,9 +620,6 @@ public class DFSInputStream extends FSInputStream
|
|
boolean connectFailedOnce = false;
|
|
boolean connectFailedOnce = false;
|
|
|
|
|
|
while (true) {
|
|
while (true) {
|
|
- // Re-fetch the locatedBlocks from NN if the timestamp has expired.
|
|
|
|
- updateBlockLocationsStamp();
|
|
|
|
-
|
|
|
|
//
|
|
//
|
|
// Compute desired block
|
|
// Compute desired block
|
|
//
|
|
//
|
|
@@ -793,6 +753,7 @@ public class DFSInputStream extends FSInputStream
|
|
* this dfsInputStream anymore.
|
|
* this dfsInputStream anymore.
|
|
*/
|
|
*/
|
|
dfsClient.removeNodeFromDeadNodeDetector(this, locatedBlocks);
|
|
dfsClient.removeNodeFromDeadNodeDetector(this, locatedBlocks);
|
|
|
|
+ maybeDeRegisterBlockRefresh();
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -871,16 +832,16 @@ public class DFSInputStream extends FSInputStream
|
|
int len = strategy.getTargetLength();
|
|
int len = strategy.getTargetLength();
|
|
CorruptedBlocks corruptedBlocks = new CorruptedBlocks();
|
|
CorruptedBlocks corruptedBlocks = new CorruptedBlocks();
|
|
failures = 0;
|
|
failures = 0;
|
|
|
|
+
|
|
|
|
+ maybeRegisterBlockRefresh();
|
|
|
|
+
|
|
if (pos < getFileLength()) {
|
|
if (pos < getFileLength()) {
|
|
int retries = 2;
|
|
int retries = 2;
|
|
while (retries > 0) {
|
|
while (retries > 0) {
|
|
try {
|
|
try {
|
|
// currentNode can be left as null if previous read had a checksum
|
|
// currentNode can be left as null if previous read had a checksum
|
|
// error on the same block. See HDFS-3067
|
|
// error on the same block. See HDFS-3067
|
|
- // currentNode needs to be updated if the blockLocations timestamp has
|
|
|
|
- // expired.
|
|
|
|
- if (pos > blockEnd || currentNode == null
|
|
|
|
- || updateBlockLocationsStamp()) {
|
|
|
|
|
|
+ if (pos > blockEnd || currentNode == null) {
|
|
currentNode = blockSeekTo(pos);
|
|
currentNode = blockSeekTo(pos);
|
|
}
|
|
}
|
|
int realLen = (int) Math.min(len, (blockEnd - pos + 1L));
|
|
int realLen = (int) Math.min(len, (blockEnd - pos + 1L));
|
|
@@ -1958,4 +1919,153 @@ public class DFSInputStream extends FSInputStream
|
|
return false;
|
|
return false;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Many DFSInputStreams can be opened and closed in quick succession, in which case
|
|
|
|
+ * they would be registered/deregistered but never need to be refreshed.
|
|
|
|
+ * Defers registering with the located block refresher, in order to avoid an additional
|
|
|
|
+ * source of unnecessary synchronization for short-lived DFSInputStreams.
|
|
|
|
+ */
|
|
|
|
+ protected void maybeRegisterBlockRefresh() {
|
|
|
|
+ if (!dfsClient.getConf().isRefreshReadBlockLocationsAutomatically()
|
|
|
|
+ || !dfsClient.getConf().isLocatedBlocksRefresherEnabled()) {
|
|
|
|
+ return;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ if (refreshingBlockLocations.get()) {
|
|
|
|
+ return;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // not enough time elapsed to refresh
|
|
|
|
+ long timeSinceLastRefresh = Time.monotonicNow() - lastRefreshedBlocksAt;
|
|
|
|
+ if (timeSinceLastRefresh < dfsClient.getConf().getLocatedBlocksRefresherInterval()) {
|
|
|
|
+ return;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ if (!refreshingBlockLocations.getAndSet(true)) {
|
|
|
|
+ dfsClient.addLocatedBlocksRefresh(this);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * De-register periodic refresh of this inputstream, if it was added to begin with.
|
|
|
|
+ */
|
|
|
|
+ private void maybeDeRegisterBlockRefresh() {
|
|
|
|
+ if (refreshingBlockLocations.get()) {
|
|
|
|
+ dfsClient.removeLocatedBlocksRefresh(this);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Refresh blocks for the input stream, if necessary.
|
|
|
|
+ *
|
|
|
|
+ * @param addressCache optional map to use as a cache for resolving datanode InetSocketAddress
|
|
|
|
+ * @return whether a refresh was performed or not
|
|
|
|
+ */
|
|
|
|
+ boolean refreshBlockLocations(Map<String, InetSocketAddress> addressCache) {
|
|
|
|
+ LocatedBlocks blocks;
|
|
|
|
+ synchronized (infoLock) {
|
|
|
|
+ blocks = getLocatedBlocks();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ if (getLocalDeadNodes().isEmpty() && allBlocksLocal(blocks, addressCache)) {
|
|
|
|
+ return false;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ try {
|
|
|
|
+ DFSClient.LOG.debug("Refreshing {} for path {}", this, getSrc());
|
|
|
|
+ LocatedBlocks newLocatedBlocks = fetchAndCheckLocatedBlocks(blocks);
|
|
|
|
+ long lastBlockLength = getLastBlockLength(newLocatedBlocks);
|
|
|
|
+ if (lastBlockLength == -1) {
|
|
|
|
+ DFSClient.LOG.debug(
|
|
|
|
+ "Discarding refreshed blocks for path {} because lastBlockLength was -1",
|
|
|
|
+ getSrc());
|
|
|
|
+ return true;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ setRefreshedValues(newLocatedBlocks, lastBlockLength);
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ DFSClient.LOG.debug("Failed to refresh DFSInputStream for path {}", getSrc(), e);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ return true;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Once new LocatedBlocks have been fetched, sets them on the DFSInputStream and
|
|
|
|
+ * updates stateful read location within the necessary locks.
|
|
|
|
+ */
|
|
|
|
+ private synchronized void setRefreshedValues(LocatedBlocks blocks, long lastBlockLength)
|
|
|
|
+ throws IOException {
|
|
|
|
+ synchronized (infoLock) {
|
|
|
|
+ setLocatedBlocksFields(blocks, lastBlockLength);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ getLocalDeadNodes().clear();
|
|
|
|
+
|
|
|
|
+ // if a stateful read has been initialized, refresh it
|
|
|
|
+ if (currentNode != null) {
|
|
|
|
+ currentNode = blockSeekTo(pos);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private boolean allBlocksLocal(LocatedBlocks blocks,
|
|
|
|
+ Map<String, InetSocketAddress> addressCache) {
|
|
|
|
+ if (addressCache == null) {
|
|
|
|
+ addressCache = new HashMap<>();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // we only need to check the first location of each block, because the blocks are already
|
|
|
|
+ // sorted by distance from the current host
|
|
|
|
+ for (LocatedBlock lb : blocks.getLocatedBlocks()) {
|
|
|
|
+ if (lb.getLocations().length == 0) {
|
|
|
|
+ return false;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ DatanodeInfoWithStorage location = lb.getLocations()[0];
|
|
|
|
+ if (location == null) {
|
|
|
|
+ return false;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ InetSocketAddress targetAddr = addressCache.computeIfAbsent(
|
|
|
|
+ location.getDatanodeUuid(),
|
|
|
|
+ unused -> {
|
|
|
|
+ String dnAddr = location.getXferAddr(dfsClient.getConf().isConnectToDnViaHostname());
|
|
|
|
+ return NetUtils.createSocketAddr(
|
|
|
|
+ dnAddr,
|
|
|
|
+ -1,
|
|
|
|
+ null,
|
|
|
|
+ dfsClient.getConf().isUriCacheEnabled());
|
|
|
|
+ });
|
|
|
|
+
|
|
|
|
+ if (!isResolveableAndLocal(targetAddr)) {
|
|
|
|
+ return false;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ return true;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private boolean isResolveableAndLocal(InetSocketAddress targetAddr) {
|
|
|
|
+ try {
|
|
|
|
+ return DFSUtilClient.isLocalAddress(targetAddr);
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ DFSClient.LOG.debug("Got an error checking if {} is local", targetAddr, e);
|
|
|
|
+ return false;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @VisibleForTesting
|
|
|
|
+ void setLastRefreshedBlocksAtForTesting(long timestamp) {
|
|
|
|
+ lastRefreshedBlocksAt = timestamp;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @VisibleForTesting
|
|
|
|
+ long getLastRefreshedBlocksAtForTesting() {
|
|
|
|
+ return lastRefreshedBlocksAt;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void setLastRefreshedBlocksAt() {
|
|
|
|
+ lastRefreshedBlocksAt = Time.monotonicNow();
|
|
|
|
+ }
|
|
}
|
|
}
|