001 /**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements. See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership. The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License. You may obtain a copy of the License at
009 *
010 * http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018 package org.apache.hadoop.hdfs;
019
020 import java.io.FileInputStream;
021 import java.io.IOException;
022 import java.net.InetSocketAddress;
023 import java.net.Socket;
024 import java.nio.ByteBuffer;
025 import java.util.AbstractMap;
026 import java.util.ArrayList;
027 import java.util.HashMap;
028 import java.util.HashSet;
029 import java.util.Iterator;
030 import java.util.List;
031 import java.util.Map;
032 import java.util.Map.Entry;
033 import java.util.Set;
034 import java.util.concurrent.ConcurrentHashMap;
035
036 import org.apache.commons.io.IOUtils;
037 import org.apache.hadoop.classification.InterfaceAudience;
038 import org.apache.hadoop.fs.ByteBufferReadable;
039 import org.apache.hadoop.fs.CanSetDropBehind;
040 import org.apache.hadoop.fs.CanSetReadahead;
041 import org.apache.hadoop.fs.ChecksumException;
042 import org.apache.hadoop.fs.FSInputStream;
043 import org.apache.hadoop.fs.UnresolvedLinkException;
044 import org.apache.hadoop.hdfs.net.DomainPeer;
045 import org.apache.hadoop.hdfs.net.Peer;
046 import org.apache.hadoop.hdfs.net.TcpPeerServer;
047 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
048 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
049 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
050 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
051 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
052 import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
053 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
054 import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
055 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
056 import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
057 import org.apache.hadoop.ipc.RPC;
058 import org.apache.hadoop.ipc.RemoteException;
059 import org.apache.hadoop.net.NetUtils;
060 import org.apache.hadoop.net.unix.DomainSocket;
061 import org.apache.hadoop.security.AccessControlException;
062 import org.apache.hadoop.security.token.Token;
063
064 import com.google.common.annotations.VisibleForTesting;
065
066 /****************************************************************
067 * DFSInputStream provides bytes from a named file. It handles
068 * negotiation of the namenode and various datanodes as necessary.
069 ****************************************************************/
070 @InterfaceAudience.Private
071 public class DFSInputStream extends FSInputStream
072 implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead {
073 @VisibleForTesting
074 static boolean tcpReadsDisabledForTesting = false;
075 private final PeerCache peerCache;
076 private final DFSClient dfsClient;
077 private boolean closed = false;
078 private final String src;
079 private BlockReader blockReader = null;
080 private final boolean verifyChecksum;
081 private LocatedBlocks locatedBlocks = null;
082 private long lastBlockBeingWrittenLength = 0;
083 private DatanodeInfo currentNode = null;
084 private LocatedBlock currentLocatedBlock = null;
085 private long pos = 0;
086 private long blockEnd = -1;
087 private CachingStrategy cachingStrategy;
088 private final ReadStatistics readStatistics = new ReadStatistics();
089
090 public static class ReadStatistics {
091 public ReadStatistics() {
092 this.totalBytesRead = 0;
093 this.totalLocalBytesRead = 0;
094 this.totalShortCircuitBytesRead = 0;
095 }
096
097 public ReadStatistics(ReadStatistics rhs) {
098 this.totalBytesRead = rhs.getTotalBytesRead();
099 this.totalLocalBytesRead = rhs.getTotalLocalBytesRead();
100 this.totalShortCircuitBytesRead = rhs.getTotalShortCircuitBytesRead();
101 }
102
103 /**
104 * @return The total bytes read. This will always be at least as
105 * high as the other numbers, since it includes all of them.
106 */
107 public long getTotalBytesRead() {
108 return totalBytesRead;
109 }
110
111 /**
112 * @return The total local bytes read. This will always be at least
113 * as high as totalShortCircuitBytesRead, since all short-circuit
114 * reads are also local.
115 */
116 public long getTotalLocalBytesRead() {
117 return totalLocalBytesRead;
118 }
119
120 /**
121 * @return The total short-circuit local bytes read.
122 */
123 public long getTotalShortCircuitBytesRead() {
124 return totalShortCircuitBytesRead;
125 }
126
127 /**
128 * @return The total number of bytes read which were not local.
129 */
130 public long getRemoteBytesRead() {
131 return totalBytesRead - totalLocalBytesRead;
132 }
133
134 void addRemoteBytes(long amt) {
135 this.totalBytesRead += amt;
136 }
137
138 void addLocalBytes(long amt) {
139 this.totalBytesRead += amt;
140 this.totalLocalBytesRead += amt;
141 }
142
143 void addShortCircuitBytes(long amt) {
144 this.totalBytesRead += amt;
145 this.totalLocalBytesRead += amt;
146 this.totalShortCircuitBytesRead += amt;
147 }
148
149 private long totalBytesRead;
150
151 private long totalLocalBytesRead;
152
153 private long totalShortCircuitBytesRead;
154 }
155
156 private final FileInputStreamCache fileInputStreamCache;
157
158 /**
159 * This variable tracks the number of failures since the start of the
160 * most recent user-facing operation. That is to say, it should be reset
161 * whenever the user makes a call on this stream, and if at any point
162 * during the retry logic, the failure count exceeds a threshold,
163 * the errors will be thrown back to the operation.
164 *
165 * Specifically this counts the number of times the client has gone
166 * back to the namenode to get a new list of block locations, and is
167 * capped at maxBlockAcquireFailures
168 */
169 private int failures = 0;
170
171 /* XXX Use of CocurrentHashMap is temp fix. Need to fix
172 * parallel accesses to DFSInputStream (through ptreads) properly */
173 private final ConcurrentHashMap<DatanodeInfo, DatanodeInfo> deadNodes =
174 new ConcurrentHashMap<DatanodeInfo, DatanodeInfo>();
175 private int buffersize = 1;
176
177 private final byte[] oneByteBuf = new byte[1]; // used for 'int read()'
178
179 void addToDeadNodes(DatanodeInfo dnInfo) {
180 deadNodes.put(dnInfo, dnInfo);
181 }
182
183 DFSInputStream(DFSClient dfsClient, String src, int buffersize, boolean verifyChecksum
184 ) throws IOException, UnresolvedLinkException {
185 this.dfsClient = dfsClient;
186 this.verifyChecksum = verifyChecksum;
187 this.buffersize = buffersize;
188 this.src = src;
189 this.peerCache = dfsClient.peerCache;
190 this.fileInputStreamCache = new FileInputStreamCache(
191 dfsClient.getConf().shortCircuitStreamsCacheSize,
192 dfsClient.getConf().shortCircuitStreamsCacheExpiryMs);
193 this.cachingStrategy =
194 dfsClient.getDefaultReadCachingStrategy().duplicate();
195 openInfo();
196 }
197
198 /**
199 * Grab the open-file info from namenode
200 */
201 synchronized void openInfo() throws IOException, UnresolvedLinkException {
202 lastBlockBeingWrittenLength = fetchLocatedBlocksAndGetLastBlockLength();
203 int retriesForLastBlockLength = 3;
204 while (retriesForLastBlockLength > 0) {
205 // Getting last block length as -1 is a special case. When cluster
206 // restarts, DNs may not report immediately. At this time partial block
207 // locations will not be available with NN for getting the length. Lets
208 // retry for 3 times to get the length.
209 if (lastBlockBeingWrittenLength == -1) {
210 DFSClient.LOG.warn("Last block locations not available. "
211 + "Datanodes might not have reported blocks completely."
212 + " Will retry for " + retriesForLastBlockLength + " times");
213 waitFor(4000);
214 lastBlockBeingWrittenLength = fetchLocatedBlocksAndGetLastBlockLength();
215 } else {
216 break;
217 }
218 retriesForLastBlockLength--;
219 }
220 if (retriesForLastBlockLength == 0) {
221 throw new IOException("Could not obtain the last block locations.");
222 }
223 }
224
225 private void waitFor(int waitTime) throws IOException {
226 try {
227 Thread.sleep(waitTime);
228 } catch (InterruptedException e) {
229 throw new IOException(
230 "Interrupted while getting the last block length.");
231 }
232 }
233
234 private long fetchLocatedBlocksAndGetLastBlockLength() throws IOException {
235 final LocatedBlocks newInfo = dfsClient.getLocatedBlocks(src, 0);
236 if (DFSClient.LOG.isDebugEnabled()) {
237 DFSClient.LOG.debug("newInfo = " + newInfo);
238 }
239 if (newInfo == null) {
240 throw new IOException("Cannot open filename " + src);
241 }
242
243 if (locatedBlocks != null) {
244 Iterator<LocatedBlock> oldIter = locatedBlocks.getLocatedBlocks().iterator();
245 Iterator<LocatedBlock> newIter = newInfo.getLocatedBlocks().iterator();
246 while (oldIter.hasNext() && newIter.hasNext()) {
247 if (! oldIter.next().getBlock().equals(newIter.next().getBlock())) {
248 throw new IOException("Blocklist for " + src + " has changed!");
249 }
250 }
251 }
252 locatedBlocks = newInfo;
253 long lastBlockBeingWrittenLength = 0;
254 if (!locatedBlocks.isLastBlockComplete()) {
255 final LocatedBlock last = locatedBlocks.getLastLocatedBlock();
256 if (last != null) {
257 if (last.getLocations().length == 0) {
258 if (last.getBlockSize() == 0) {
259 // if the length is zero, then no data has been written to
260 // datanode. So no need to wait for the locations.
261 return 0;
262 }
263 return -1;
264 }
265 final long len = readBlockLength(last);
266 last.getBlock().setNumBytes(len);
267 lastBlockBeingWrittenLength = len;
268 }
269 }
270
271 currentNode = null;
272 return lastBlockBeingWrittenLength;
273 }
274
275 /** Read the block length from one of the datanodes. */
276 private long readBlockLength(LocatedBlock locatedblock) throws IOException {
277 assert locatedblock != null : "LocatedBlock cannot be null";
278 int replicaNotFoundCount = locatedblock.getLocations().length;
279
280 for(DatanodeInfo datanode : locatedblock.getLocations()) {
281 ClientDatanodeProtocol cdp = null;
282
283 try {
284 cdp = DFSUtil.createClientDatanodeProtocolProxy(datanode,
285 dfsClient.getConfiguration(), dfsClient.getConf().socketTimeout,
286 dfsClient.getConf().connectToDnViaHostname, locatedblock);
287
288 final long n = cdp.getReplicaVisibleLength(locatedblock.getBlock());
289
290 if (n >= 0) {
291 return n;
292 }
293 }
294 catch(IOException ioe) {
295 if (ioe instanceof RemoteException &&
296 (((RemoteException) ioe).unwrapRemoteException() instanceof
297 ReplicaNotFoundException)) {
298 // special case : replica might not be on the DN, treat as 0 length
299 replicaNotFoundCount--;
300 }
301
302 if (DFSClient.LOG.isDebugEnabled()) {
303 DFSClient.LOG.debug("Failed to getReplicaVisibleLength from datanode "
304 + datanode + " for block " + locatedblock.getBlock(), ioe);
305 }
306 } finally {
307 if (cdp != null) {
308 RPC.stopProxy(cdp);
309 }
310 }
311 }
312
313 // Namenode told us about these locations, but none know about the replica
314 // means that we hit the race between pipeline creation start and end.
315 // we require all 3 because some other exception could have happened
316 // on a DN that has it. we want to report that error
317 if (replicaNotFoundCount == 0) {
318 return 0;
319 }
320
321 throw new IOException("Cannot obtain block length for " + locatedblock);
322 }
323
324 public synchronized long getFileLength() {
325 return locatedBlocks == null? 0:
326 locatedBlocks.getFileLength() + lastBlockBeingWrittenLength;
327 }
328
329 // Short circuit local reads are forbidden for files that are
330 // under construction. See HDFS-2757.
331 synchronized boolean shortCircuitForbidden() {
332 return locatedBlocks.isUnderConstruction();
333 }
334
335 /**
336 * Returns the datanode from which the stream is currently reading.
337 */
338 public DatanodeInfo getCurrentDatanode() {
339 return currentNode;
340 }
341
342 /**
343 * Returns the block containing the target position.
344 */
345 synchronized public ExtendedBlock getCurrentBlock() {
346 if (currentLocatedBlock == null){
347 return null;
348 }
349 return currentLocatedBlock.getBlock();
350 }
351
352 /**
353 * Return collection of blocks that has already been located.
354 */
355 public synchronized List<LocatedBlock> getAllBlocks() throws IOException {
356 return getBlockRange(0, getFileLength());
357 }
358
359 /**
360 * Get block at the specified position.
361 * Fetch it from the namenode if not cached.
362 *
363 * @param offset
364 * @param updatePosition whether to update current position
365 * @return located block
366 * @throws IOException
367 */
368 private synchronized LocatedBlock getBlockAt(long offset,
369 boolean updatePosition) throws IOException {
370 assert (locatedBlocks != null) : "locatedBlocks is null";
371
372 final LocatedBlock blk;
373
374 //check offset
375 if (offset < 0 || offset >= getFileLength()) {
376 throw new IOException("offset < 0 || offset >= getFileLength(), offset="
377 + offset
378 + ", updatePosition=" + updatePosition
379 + ", locatedBlocks=" + locatedBlocks);
380 }
381 else if (offset >= locatedBlocks.getFileLength()) {
382 // offset to the portion of the last block,
383 // which is not known to the name-node yet;
384 // getting the last block
385 blk = locatedBlocks.getLastLocatedBlock();
386 }
387 else {
388 // search cached blocks first
389 int targetBlockIdx = locatedBlocks.findBlock(offset);
390 if (targetBlockIdx < 0) { // block is not cached
391 targetBlockIdx = LocatedBlocks.getInsertIndex(targetBlockIdx);
392 // fetch more blocks
393 final LocatedBlocks newBlocks = dfsClient.getLocatedBlocks(src, offset);
394 assert (newBlocks != null) : "Could not find target position " + offset;
395 locatedBlocks.insertRange(targetBlockIdx, newBlocks.getLocatedBlocks());
396 }
397 blk = locatedBlocks.get(targetBlockIdx);
398 }
399
400 // update current position
401 if (updatePosition) {
402 pos = offset;
403 blockEnd = blk.getStartOffset() + blk.getBlockSize() - 1;
404 currentLocatedBlock = blk;
405 }
406 return blk;
407 }
408
409 /** Fetch a block from namenode and cache it */
410 private synchronized void fetchBlockAt(long offset) throws IOException {
411 int targetBlockIdx = locatedBlocks.findBlock(offset);
412 if (targetBlockIdx < 0) { // block is not cached
413 targetBlockIdx = LocatedBlocks.getInsertIndex(targetBlockIdx);
414 }
415 // fetch blocks
416 final LocatedBlocks newBlocks = dfsClient.getLocatedBlocks(src, offset);
417 if (newBlocks == null) {
418 throw new IOException("Could not find target position " + offset);
419 }
420 locatedBlocks.insertRange(targetBlockIdx, newBlocks.getLocatedBlocks());
421 }
422
423 /**
424 * Get blocks in the specified range.
425 * Fetch them from the namenode if not cached. This function
426 * will not get a read request beyond the EOF.
427 * @param offset
428 * @param length
429 * @return consequent segment of located blocks
430 * @throws IOException
431 */
432 private synchronized List<LocatedBlock> getBlockRange(long offset,
433 long length)
434 throws IOException {
435 // getFileLength(): returns total file length
436 // locatedBlocks.getFileLength(): returns length of completed blocks
437 if (offset >= getFileLength()) {
438 throw new IOException("Offset: " + offset +
439 " exceeds file length: " + getFileLength());
440 }
441
442 final List<LocatedBlock> blocks;
443 final long lengthOfCompleteBlk = locatedBlocks.getFileLength();
444 final boolean readOffsetWithinCompleteBlk = offset < lengthOfCompleteBlk;
445 final boolean readLengthPastCompleteBlk = offset + length > lengthOfCompleteBlk;
446
447 if (readOffsetWithinCompleteBlk) {
448 //get the blocks of finalized (completed) block range
449 blocks = getFinalizedBlockRange(offset,
450 Math.min(length, lengthOfCompleteBlk - offset));
451 } else {
452 blocks = new ArrayList<LocatedBlock>(1);
453 }
454
455 // get the blocks from incomplete block range
456 if (readLengthPastCompleteBlk) {
457 blocks.add(locatedBlocks.getLastLocatedBlock());
458 }
459
460 return blocks;
461 }
462
463 /**
464 * Get blocks in the specified range.
465 * Includes only the complete blocks.
466 * Fetch them from the namenode if not cached.
467 */
468 private synchronized List<LocatedBlock> getFinalizedBlockRange(
469 long offset, long length) throws IOException {
470 assert (locatedBlocks != null) : "locatedBlocks is null";
471 List<LocatedBlock> blockRange = new ArrayList<LocatedBlock>();
472 // search cached blocks first
473 int blockIdx = locatedBlocks.findBlock(offset);
474 if (blockIdx < 0) { // block is not cached
475 blockIdx = LocatedBlocks.getInsertIndex(blockIdx);
476 }
477 long remaining = length;
478 long curOff = offset;
479 while(remaining > 0) {
480 LocatedBlock blk = null;
481 if(blockIdx < locatedBlocks.locatedBlockCount())
482 blk = locatedBlocks.get(blockIdx);
483 if (blk == null || curOff < blk.getStartOffset()) {
484 LocatedBlocks newBlocks;
485 newBlocks = dfsClient.getLocatedBlocks(src, curOff, remaining);
486 locatedBlocks.insertRange(blockIdx, newBlocks.getLocatedBlocks());
487 continue;
488 }
489 assert curOff >= blk.getStartOffset() : "Block not found";
490 blockRange.add(blk);
491 long bytesRead = blk.getStartOffset() + blk.getBlockSize() - curOff;
492 remaining -= bytesRead;
493 curOff += bytesRead;
494 blockIdx++;
495 }
496 return blockRange;
497 }
498
499 /**
500 * Open a DataInputStream to a DataNode so that it can be read from.
501 * We get block ID and the IDs of the destinations at startup, from the namenode.
502 */
503 private synchronized DatanodeInfo blockSeekTo(long target) throws IOException {
504 if (target >= getFileLength()) {
505 throw new IOException("Attempted to read past end of file");
506 }
507
508 // Will be getting a new BlockReader.
509 if (blockReader != null) {
510 blockReader.close();
511 blockReader = null;
512 }
513
514 //
515 // Connect to best DataNode for desired Block, with potential offset
516 //
517 DatanodeInfo chosenNode = null;
518 int refetchToken = 1; // only need to get a new access token once
519 int refetchEncryptionKey = 1; // only need to get a new encryption key once
520
521 boolean connectFailedOnce = false;
522
523 while (true) {
524 //
525 // Compute desired block
526 //
527 LocatedBlock targetBlock = getBlockAt(target, true);
528 assert (target==pos) : "Wrong postion " + pos + " expect " + target;
529 long offsetIntoBlock = target - targetBlock.getStartOffset();
530
531 DNAddrPair retval = chooseDataNode(targetBlock);
532 chosenNode = retval.info;
533 InetSocketAddress targetAddr = retval.addr;
534
535 try {
536 ExtendedBlock blk = targetBlock.getBlock();
537 Token<BlockTokenIdentifier> accessToken = targetBlock.getBlockToken();
538 blockReader = getBlockReader(targetAddr, chosenNode, src, blk,
539 accessToken, offsetIntoBlock, blk.getNumBytes() - offsetIntoBlock,
540 buffersize, verifyChecksum, dfsClient.clientName);
541 if(connectFailedOnce) {
542 DFSClient.LOG.info("Successfully connected to " + targetAddr +
543 " for " + blk);
544 }
545 return chosenNode;
546 } catch (AccessControlException ex) {
547 DFSClient.LOG.warn("Short circuit access failed " + ex);
548 dfsClient.disableLegacyBlockReaderLocal();
549 continue;
550 } catch (IOException ex) {
551 if (ex instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
552 DFSClient.LOG.info("Will fetch a new encryption key and retry, "
553 + "encryption key was invalid when connecting to " + targetAddr
554 + " : " + ex);
555 // The encryption key used is invalid.
556 refetchEncryptionKey--;
557 dfsClient.clearDataEncryptionKey();
558 } else if (ex instanceof InvalidBlockTokenException && refetchToken > 0) {
559 DFSClient.LOG.info("Will fetch a new access token and retry, "
560 + "access token was invalid when connecting to " + targetAddr
561 + " : " + ex);
562 /*
563 * Get a new access token and retry. Retry is needed in 2 cases. 1)
564 * When both NN and DN re-started while DFSClient holding a cached
565 * access token. 2) In the case that NN fails to update its
566 * access key at pre-set interval (by a wide margin) and
567 * subsequently restarts. In this case, DN re-registers itself with
568 * NN and receives a new access key, but DN will delete the old
569 * access key from its memory since it's considered expired based on
570 * the estimated expiration date.
571 */
572 refetchToken--;
573 fetchBlockAt(target);
574 } else {
575 connectFailedOnce = true;
576 DFSClient.LOG.warn("Failed to connect to " + targetAddr + " for block"
577 + ", add to deadNodes and continue. " + ex, ex);
578 // Put chosen node into dead list, continue
579 addToDeadNodes(chosenNode);
580 }
581 }
582 }
583 }
584
585 /**
586 * Close it down!
587 */
588 @Override
589 public synchronized void close() throws IOException {
590 if (closed) {
591 return;
592 }
593 dfsClient.checkOpen();
594
595 if (blockReader != null) {
596 blockReader.close();
597 blockReader = null;
598 }
599 super.close();
600 fileInputStreamCache.close();
601 closed = true;
602 }
603
604 @Override
605 public synchronized int read() throws IOException {
606 int ret = read( oneByteBuf, 0, 1 );
607 return ( ret <= 0 ) ? -1 : (oneByteBuf[0] & 0xff);
608 }
609
610 /**
611 * Wraps different possible read implementations so that readBuffer can be
612 * strategy-agnostic.
613 */
614 private interface ReaderStrategy {
615 public int doRead(BlockReader blockReader, int off, int len,
616 ReadStatistics readStatistics) throws ChecksumException, IOException;
617 }
618
619 private static void updateReadStatistics(ReadStatistics readStatistics,
620 int nRead, BlockReader blockReader) {
621 if (nRead <= 0) return;
622 if (blockReader.isShortCircuit()) {
623 readStatistics.totalBytesRead += nRead;
624 readStatistics.totalLocalBytesRead += nRead;
625 readStatistics.totalShortCircuitBytesRead += nRead;
626 } else if (blockReader.isLocal()) {
627 readStatistics.totalBytesRead += nRead;
628 readStatistics.totalLocalBytesRead += nRead;
629 } else {
630 readStatistics.totalBytesRead += nRead;
631 }
632 }
633
634 /**
635 * Used to read bytes into a byte[]
636 */
637 private static class ByteArrayStrategy implements ReaderStrategy {
638 final byte[] buf;
639
640 public ByteArrayStrategy(byte[] buf) {
641 this.buf = buf;
642 }
643
644 @Override
645 public int doRead(BlockReader blockReader, int off, int len,
646 ReadStatistics readStatistics) throws ChecksumException, IOException {
647 int nRead = blockReader.read(buf, off, len);
648 updateReadStatistics(readStatistics, nRead, blockReader);
649 return nRead;
650 }
651 }
652
653 /**
654 * Used to read bytes into a user-supplied ByteBuffer
655 */
656 private static class ByteBufferStrategy implements ReaderStrategy {
657 final ByteBuffer buf;
658 ByteBufferStrategy(ByteBuffer buf) {
659 this.buf = buf;
660 }
661
662 @Override
663 public int doRead(BlockReader blockReader, int off, int len,
664 ReadStatistics readStatistics) throws ChecksumException, IOException {
665 int oldpos = buf.position();
666 int oldlimit = buf.limit();
667 boolean success = false;
668 try {
669 int ret = blockReader.read(buf);
670 success = true;
671 updateReadStatistics(readStatistics, ret, blockReader);
672 return ret;
673 } finally {
674 if (!success) {
675 // Reset to original state so that retries work correctly.
676 buf.position(oldpos);
677 buf.limit(oldlimit);
678 }
679 }
680 }
681 }
682
683 /* This is a used by regular read() and handles ChecksumExceptions.
684 * name readBuffer() is chosen to imply similarity to readBuffer() in
685 * ChecksumFileSystem
686 */
687 private synchronized int readBuffer(ReaderStrategy reader, int off, int len,
688 Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
689 throws IOException {
690 IOException ioe;
691
692 /* we retry current node only once. So this is set to true only here.
693 * Intention is to handle one common case of an error that is not a
694 * failure on datanode or client : when DataNode closes the connection
695 * since client is idle. If there are other cases of "non-errors" then
696 * then a datanode might be retried by setting this to true again.
697 */
698 boolean retryCurrentNode = true;
699
700 while (true) {
701 // retry as many times as seekToNewSource allows.
702 try {
703 return reader.doRead(blockReader, off, len, readStatistics);
704 } catch ( ChecksumException ce ) {
705 DFSClient.LOG.warn("Found Checksum error for "
706 + getCurrentBlock() + " from " + currentNode
707 + " at " + ce.getPos());
708 ioe = ce;
709 retryCurrentNode = false;
710 // we want to remember which block replicas we have tried
711 addIntoCorruptedBlockMap(getCurrentBlock(), currentNode,
712 corruptedBlockMap);
713 } catch ( IOException e ) {
714 if (!retryCurrentNode) {
715 DFSClient.LOG.warn("Exception while reading from "
716 + getCurrentBlock() + " of " + src + " from "
717 + currentNode, e);
718 }
719 ioe = e;
720 }
721 boolean sourceFound = false;
722 if (retryCurrentNode) {
723 /* possibly retry the same node so that transient errors don't
724 * result in application level failures (e.g. Datanode could have
725 * closed the connection because the client is idle for too long).
726 */
727 sourceFound = seekToBlockSource(pos);
728 } else {
729 addToDeadNodes(currentNode);
730 sourceFound = seekToNewSource(pos);
731 }
732 if (!sourceFound) {
733 throw ioe;
734 }
735 retryCurrentNode = false;
736 }
737 }
738
739 private int readWithStrategy(ReaderStrategy strategy, int off, int len) throws IOException {
740 dfsClient.checkOpen();
741 if (closed) {
742 throw new IOException("Stream closed");
743 }
744 Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap
745 = new HashMap<ExtendedBlock, Set<DatanodeInfo>>();
746 failures = 0;
747 if (pos < getFileLength()) {
748 int retries = 2;
749 while (retries > 0) {
750 try {
751 // currentNode can be left as null if previous read had a checksum
752 // error on the same block. See HDFS-3067
753 if (pos > blockEnd || currentNode == null) {
754 currentNode = blockSeekTo(pos);
755 }
756 int realLen = (int) Math.min(len, (blockEnd - pos + 1L));
757 int result = readBuffer(strategy, off, realLen, corruptedBlockMap);
758
759 if (result >= 0) {
760 pos += result;
761 } else {
762 // got a EOS from reader though we expect more data on it.
763 throw new IOException("Unexpected EOS from the reader");
764 }
765 if (dfsClient.stats != null && result != -1) {
766 dfsClient.stats.incrementBytesRead(result);
767 }
768 return result;
769 } catch (ChecksumException ce) {
770 throw ce;
771 } catch (IOException e) {
772 if (retries == 1) {
773 DFSClient.LOG.warn("DFS Read", e);
774 }
775 blockEnd = -1;
776 if (currentNode != null) { addToDeadNodes(currentNode); }
777 if (--retries == 0) {
778 throw e;
779 }
780 } finally {
781 // Check if need to report block replicas corruption either read
782 // was successful or ChecksumException occured.
783 reportCheckSumFailure(corruptedBlockMap,
784 currentLocatedBlock.getLocations().length);
785 }
786 }
787 }
788 return -1;
789 }
790
791 /**
792 * Read the entire buffer.
793 */
794 @Override
795 public synchronized int read(final byte buf[], int off, int len) throws IOException {
796 ReaderStrategy byteArrayReader = new ByteArrayStrategy(buf);
797
798 return readWithStrategy(byteArrayReader, off, len);
799 }
800
801 @Override
802 public synchronized int read(final ByteBuffer buf) throws IOException {
803 ReaderStrategy byteBufferReader = new ByteBufferStrategy(buf);
804
805 return readWithStrategy(byteBufferReader, 0, buf.remaining());
806 }
807
808
809 /**
810 * Add corrupted block replica into map.
811 * @param corruptedBlockMap
812 */
813 private void addIntoCorruptedBlockMap(ExtendedBlock blk, DatanodeInfo node,
814 Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
815 Set<DatanodeInfo> dnSet = null;
816 if((corruptedBlockMap.containsKey(blk))) {
817 dnSet = corruptedBlockMap.get(blk);
818 }else {
819 dnSet = new HashSet<DatanodeInfo>();
820 }
821 if (!dnSet.contains(node)) {
822 dnSet.add(node);
823 corruptedBlockMap.put(blk, dnSet);
824 }
825 }
826
827 private DNAddrPair chooseDataNode(LocatedBlock block)
828 throws IOException {
829 while (true) {
830 DatanodeInfo[] nodes = block.getLocations();
831 try {
832 DatanodeInfo chosenNode = bestNode(nodes, deadNodes);
833 final String dnAddr =
834 chosenNode.getXferAddr(dfsClient.getConf().connectToDnViaHostname);
835 if (DFSClient.LOG.isDebugEnabled()) {
836 DFSClient.LOG.debug("Connecting to datanode " + dnAddr);
837 }
838 InetSocketAddress targetAddr = NetUtils.createSocketAddr(dnAddr);
839 return new DNAddrPair(chosenNode, targetAddr);
840 } catch (IOException ie) {
841 String blockInfo = block.getBlock() + " file=" + src;
842 if (failures >= dfsClient.getMaxBlockAcquireFailures()) {
843 throw new BlockMissingException(src, "Could not obtain block: " + blockInfo,
844 block.getStartOffset());
845 }
846
847 if (nodes == null || nodes.length == 0) {
848 DFSClient.LOG.info("No node available for " + blockInfo);
849 }
850 DFSClient.LOG.info("Could not obtain " + block.getBlock()
851 + " from any node: " + ie
852 + ". Will get new block locations from namenode and retry...");
853 try {
854 // Introducing a random factor to the wait time before another retry.
855 // The wait time is dependent on # of failures and a random factor.
856 // At the first time of getting a BlockMissingException, the wait time
857 // is a random number between 0..3000 ms. If the first retry
858 // still fails, we will wait 3000 ms grace period before the 2nd retry.
859 // Also at the second retry, the waiting window is expanded to 6000 ms
860 // alleviating the request rate from the server. Similarly the 3rd retry
861 // will wait 6000ms grace period before retry and the waiting window is
862 // expanded to 9000ms.
863 final int timeWindow = dfsClient.getConf().timeWindow;
864 double waitTime = timeWindow * failures + // grace period for the last round of attempt
865 timeWindow * (failures + 1) * DFSUtil.getRandom().nextDouble(); // expanding time window for each failure
866 DFSClient.LOG.warn("DFS chooseDataNode: got # " + (failures + 1) + " IOException, will wait for " + waitTime + " msec.");
867 Thread.sleep((long)waitTime);
868 } catch (InterruptedException iex) {
869 }
870 deadNodes.clear(); //2nd option is to remove only nodes[blockId]
871 openInfo();
872 block = getBlockAt(block.getStartOffset(), false);
873 failures++;
874 continue;
875 }
876 }
877 }
878
879 private void fetchBlockByteRange(LocatedBlock block, long start, long end,
880 byte[] buf, int offset,
881 Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
882 throws IOException {
883 //
884 // Connect to best DataNode for desired Block, with potential offset
885 //
886 int refetchToken = 1; // only need to get a new access token once
887 int refetchEncryptionKey = 1; // only need to get a new encryption key once
888
889 while (true) {
890 // cached block locations may have been updated by chooseDataNode()
891 // or fetchBlockAt(). Always get the latest list of locations at the
892 // start of the loop.
893 block = getBlockAt(block.getStartOffset(), false);
894 DNAddrPair retval = chooseDataNode(block);
895 DatanodeInfo chosenNode = retval.info;
896 InetSocketAddress targetAddr = retval.addr;
897 BlockReader reader = null;
898
899 try {
900 Token<BlockTokenIdentifier> blockToken = block.getBlockToken();
901
902 int len = (int) (end - start + 1);
903 reader = getBlockReader(targetAddr, chosenNode, src, block.getBlock(),
904 blockToken, start, len, buffersize, verifyChecksum,
905 dfsClient.clientName);
906 int nread = reader.readAll(buf, offset, len);
907 if (nread != len) {
908 throw new IOException("truncated return from reader.read(): " +
909 "excpected " + len + ", got " + nread);
910 }
911 return;
912 } catch (ChecksumException e) {
913 DFSClient.LOG.warn("fetchBlockByteRange(). Got a checksum exception for " +
914 src + " at " + block.getBlock() + ":" +
915 e.getPos() + " from " + chosenNode);
916 // we want to remember what we have tried
917 addIntoCorruptedBlockMap(block.getBlock(), chosenNode, corruptedBlockMap);
918 } catch (AccessControlException ex) {
919 DFSClient.LOG.warn("Short circuit access failed " + ex);
920 dfsClient.disableLegacyBlockReaderLocal();
921 continue;
922 } catch (IOException e) {
923 if (e instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
924 DFSClient.LOG.info("Will fetch a new encryption key and retry, "
925 + "encryption key was invalid when connecting to " + targetAddr
926 + " : " + e);
927 // The encryption key used is invalid.
928 refetchEncryptionKey--;
929 dfsClient.clearDataEncryptionKey();
930 } else if (e instanceof InvalidBlockTokenException && refetchToken > 0) {
931 DFSClient.LOG.info("Will get a new access token and retry, "
932 + "access token was invalid when connecting to " + targetAddr
933 + " : " + e);
934 refetchToken--;
935 fetchBlockAt(block.getStartOffset());
936 continue;
937 } else {
938 DFSClient.LOG.warn("Failed to connect to " + targetAddr +
939 " for file " + src + " for block " + block.getBlock() + ":" + e);
940 if (DFSClient.LOG.isDebugEnabled()) {
941 DFSClient.LOG.debug("Connection failure ", e);
942 }
943 }
944 } finally {
945 if (reader != null) {
946 reader.close();
947 }
948 }
949 // Put chosen node into dead list, continue
950 addToDeadNodes(chosenNode);
951 }
952 }
953
954 private Peer newTcpPeer(InetSocketAddress addr) throws IOException {
955 Peer peer = null;
956 boolean success = false;
957 Socket sock = null;
958 try {
959 sock = dfsClient.socketFactory.createSocket();
960 NetUtils.connect(sock, addr,
961 dfsClient.getRandomLocalInterfaceAddr(),
962 dfsClient.getConf().socketTimeout);
963 peer = TcpPeerServer.peerFromSocketAndKey(sock,
964 dfsClient.getDataEncryptionKey());
965 success = true;
966 return peer;
967 } finally {
968 if (!success) {
969 IOUtils.closeQuietly(peer);
970 IOUtils.closeQuietly(sock);
971 }
972 }
973 }
974
975 /**
976 * Retrieve a BlockReader suitable for reading.
977 * This method will reuse the cached connection to the DN if appropriate.
978 * Otherwise, it will create a new connection.
979 * Throwing an IOException from this method is basically equivalent to
980 * declaring the DataNode bad, so we try to connect a lot of different ways
981 * before doing that.
982 *
983 * @param dnAddr Address of the datanode
984 * @param chosenNode Chosen datanode information
985 * @param file File location
986 * @param block The Block object
987 * @param blockToken The access token for security
988 * @param startOffset The read offset, relative to block head
989 * @param len The number of bytes to read
990 * @param bufferSize The IO buffer size (not the client buffer size)
991 * @param verifyChecksum Whether to verify checksum
992 * @param clientName Client name
993 * @return New BlockReader instance
994 */
995 protected BlockReader getBlockReader(InetSocketAddress dnAddr,
996 DatanodeInfo chosenNode,
997 String file,
998 ExtendedBlock block,
999 Token<BlockTokenIdentifier> blockToken,
1000 long startOffset,
1001 long len,
1002 int bufferSize,
1003 boolean verifyChecksum,
1004 String clientName)
1005 throws IOException {
1006 // Firstly, we check to see if we have cached any file descriptors for
1007 // local blocks. If so, we can just re-use those file descriptors.
1008 FileInputStream fis[] = fileInputStreamCache.get(chosenNode, block);
1009 if (fis != null) {
1010 if (DFSClient.LOG.isDebugEnabled()) {
1011 DFSClient.LOG.debug("got FileInputStreams for " + block + " from " +
1012 "the FileInputStreamCache.");
1013 }
1014 return new BlockReaderLocal(dfsClient.getConf(), file,
1015 block, startOffset, len, fis[0], fis[1], chosenNode, verifyChecksum,
1016 fileInputStreamCache);
1017 }
1018
1019 // If the legacy local block reader is enabled and we are reading a local
1020 // block, try to create a BlockReaderLocalLegacy. The legacy local block
1021 // reader implements local reads in the style first introduced by HDFS-2246.
1022 if ((dfsClient.useLegacyBlockReaderLocal()) &&
1023 DFSClient.isLocalAddress(dnAddr) &&
1024 (!shortCircuitForbidden())) {
1025 try {
1026 return BlockReaderFactory.getLegacyBlockReaderLocal(dfsClient,
1027 clientName, block, blockToken, chosenNode, startOffset);
1028 } catch (IOException e) {
1029 DFSClient.LOG.warn("error creating legacy BlockReaderLocal. " +
1030 "Disabling legacy local reads.", e);
1031 dfsClient.disableLegacyBlockReaderLocal();
1032 }
1033 }
1034
1035 // Look for cached domain peers.
1036 int cacheTries = 0;
1037 DomainSocketFactory dsFactory = dfsClient.getDomainSocketFactory();
1038 BlockReader reader = null;
1039 final int nCachedConnRetry = dfsClient.getConf().nCachedConnRetry;
1040 for (; cacheTries < nCachedConnRetry; ++cacheTries) {
1041 Peer peer = peerCache.get(chosenNode, true);
1042 if (peer == null) break;
1043 try {
1044 boolean allowShortCircuitLocalReads = dfsClient.getConf().
1045 shortCircuitLocalReads && (!shortCircuitForbidden());
1046 reader = BlockReaderFactory.newBlockReader(
1047 dfsClient.getConf(), file, block, blockToken, startOffset,
1048 len, verifyChecksum, clientName, peer, chosenNode,
1049 dsFactory, peerCache, fileInputStreamCache,
1050 allowShortCircuitLocalReads, cachingStrategy);
1051 return reader;
1052 } catch (IOException ex) {
1053 DFSClient.LOG.debug("Error making BlockReader with DomainSocket. " +
1054 "Closing stale " + peer, ex);
1055 } finally {
1056 if (reader == null) {
1057 IOUtils.closeQuietly(peer);
1058 }
1059 }
1060 }
1061
1062 // Try to create a DomainPeer.
1063 DomainSocket domSock = dsFactory.create(dnAddr, this);
1064 if (domSock != null) {
1065 Peer peer = new DomainPeer(domSock);
1066 try {
1067 boolean allowShortCircuitLocalReads = dfsClient.getConf().
1068 shortCircuitLocalReads && (!shortCircuitForbidden());
1069 reader = BlockReaderFactory.newBlockReader(
1070 dfsClient.getConf(), file, block, blockToken, startOffset,
1071 len, verifyChecksum, clientName, peer, chosenNode,
1072 dsFactory, peerCache, fileInputStreamCache,
1073 allowShortCircuitLocalReads, cachingStrategy);
1074 return reader;
1075 } catch (IOException e) {
1076 DFSClient.LOG.warn("failed to connect to " + domSock, e);
1077 } finally {
1078 if (reader == null) {
1079 // If the Peer that we got the error from was a DomainPeer,
1080 // mark the socket path as bad, so that newDataSocket will not try
1081 // to re-open this socket for a while.
1082 dsFactory.disableDomainSocketPath(domSock.getPath());
1083 IOUtils.closeQuietly(peer);
1084 }
1085 }
1086 }
1087
1088 // Look for cached peers.
1089 for (; cacheTries < nCachedConnRetry; ++cacheTries) {
1090 Peer peer = peerCache.get(chosenNode, false);
1091 if (peer == null) break;
1092 try {
1093 reader = BlockReaderFactory.newBlockReader(
1094 dfsClient.getConf(), file, block, blockToken, startOffset,
1095 len, verifyChecksum, clientName, peer, chosenNode,
1096 dsFactory, peerCache, fileInputStreamCache, false,
1097 cachingStrategy);
1098 return reader;
1099 } catch (IOException ex) {
1100 DFSClient.LOG.debug("Error making BlockReader. Closing stale " +
1101 peer, ex);
1102 } finally {
1103 if (reader == null) {
1104 IOUtils.closeQuietly(peer);
1105 }
1106 }
1107 }
1108 if (tcpReadsDisabledForTesting) {
1109 throw new IOException("TCP reads are disabled.");
1110 }
1111 // Try to create a new remote peer.
1112 Peer peer = newTcpPeer(dnAddr);
1113 return BlockReaderFactory.newBlockReader(
1114 dfsClient.getConf(), file, block, blockToken, startOffset,
1115 len, verifyChecksum, clientName, peer, chosenNode,
1116 dsFactory, peerCache, fileInputStreamCache, false,
1117 cachingStrategy);
1118 }
1119
1120
1121 /**
1122 * Read bytes starting from the specified position.
1123 *
1124 * @param position start read from this position
1125 * @param buffer read buffer
1126 * @param offset offset into buffer
1127 * @param length number of bytes to read
1128 *
1129 * @return actual number of bytes read
1130 */
1131 @Override
1132 public int read(long position, byte[] buffer, int offset, int length)
1133 throws IOException {
1134 // sanity checks
1135 dfsClient.checkOpen();
1136 if (closed) {
1137 throw new IOException("Stream closed");
1138 }
1139 failures = 0;
1140 long filelen = getFileLength();
1141 if ((position < 0) || (position >= filelen)) {
1142 return -1;
1143 }
1144 int realLen = length;
1145 if ((position + length) > filelen) {
1146 realLen = (int)(filelen - position);
1147 }
1148
1149 // determine the block and byte range within the block
1150 // corresponding to position and realLen
1151 List<LocatedBlock> blockRange = getBlockRange(position, realLen);
1152 int remaining = realLen;
1153 Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap
1154 = new HashMap<ExtendedBlock, Set<DatanodeInfo>>();
1155 for (LocatedBlock blk : blockRange) {
1156 long targetStart = position - blk.getStartOffset();
1157 long bytesToRead = Math.min(remaining, blk.getBlockSize() - targetStart);
1158 try {
1159 fetchBlockByteRange(blk, targetStart,
1160 targetStart + bytesToRead - 1, buffer, offset, corruptedBlockMap);
1161 } finally {
1162 // Check and report if any block replicas are corrupted.
1163 // BlockMissingException may be caught if all block replicas are
1164 // corrupted.
1165 reportCheckSumFailure(corruptedBlockMap, blk.getLocations().length);
1166 }
1167
1168 remaining -= bytesToRead;
1169 position += bytesToRead;
1170 offset += bytesToRead;
1171 }
1172 assert remaining == 0 : "Wrong number of bytes read.";
1173 if (dfsClient.stats != null) {
1174 dfsClient.stats.incrementBytesRead(realLen);
1175 }
1176 return realLen;
1177 }
1178
1179 /**
1180 * DFSInputStream reports checksum failure.
1181 * Case I : client has tried multiple data nodes and at least one of the
1182 * attempts has succeeded. We report the other failures as corrupted block to
1183 * namenode.
1184 * Case II: client has tried out all data nodes, but all failed. We
1185 * only report if the total number of replica is 1. We do not
1186 * report otherwise since this maybe due to the client is a handicapped client
1187 * (who can not read).
1188 * @param corruptedBlockMap map of corrupted blocks
1189 * @param dataNodeCount number of data nodes who contains the block replicas
1190 */
1191 private void reportCheckSumFailure(
1192 Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap,
1193 int dataNodeCount) {
1194 if (corruptedBlockMap.isEmpty()) {
1195 return;
1196 }
1197 Iterator<Entry<ExtendedBlock, Set<DatanodeInfo>>> it = corruptedBlockMap
1198 .entrySet().iterator();
1199 Entry<ExtendedBlock, Set<DatanodeInfo>> entry = it.next();
1200 ExtendedBlock blk = entry.getKey();
1201 Set<DatanodeInfo> dnSet = entry.getValue();
1202 if (((dnSet.size() < dataNodeCount) && (dnSet.size() > 0))
1203 || ((dataNodeCount == 1) && (dnSet.size() == dataNodeCount))) {
1204 DatanodeInfo[] locs = new DatanodeInfo[dnSet.size()];
1205 int i = 0;
1206 for (DatanodeInfo dn:dnSet) {
1207 locs[i++] = dn;
1208 }
1209 LocatedBlock [] lblocks = { new LocatedBlock(blk, locs) };
1210 dfsClient.reportChecksumFailure(src, lblocks);
1211 }
1212 corruptedBlockMap.clear();
1213 }
1214
1215 @Override
1216 public long skip(long n) throws IOException {
1217 if ( n > 0 ) {
1218 long curPos = getPos();
1219 long fileLen = getFileLength();
1220 if( n+curPos > fileLen ) {
1221 n = fileLen - curPos;
1222 }
1223 seek(curPos+n);
1224 return n;
1225 }
1226 return n < 0 ? -1 : 0;
1227 }
1228
1229 /**
1230 * Seek to a new arbitrary location
1231 */
1232 @Override
1233 public synchronized void seek(long targetPos) throws IOException {
1234 if (targetPos > getFileLength()) {
1235 throw new IOException("Cannot seek after EOF");
1236 }
1237 if (targetPos < 0) {
1238 throw new IOException("Cannot seek to negative offset");
1239 }
1240 if (closed) {
1241 throw new IOException("Stream is closed!");
1242 }
1243 boolean done = false;
1244 if (pos <= targetPos && targetPos <= blockEnd) {
1245 //
1246 // If this seek is to a positive position in the current
1247 // block, and this piece of data might already be lying in
1248 // the TCP buffer, then just eat up the intervening data.
1249 //
1250 int diff = (int)(targetPos - pos);
1251 if (diff <= blockReader.available()) {
1252 try {
1253 pos += blockReader.skip(diff);
1254 if (pos == targetPos) {
1255 done = true;
1256 }
1257 } catch (IOException e) {//make following read to retry
1258 if(DFSClient.LOG.isDebugEnabled()) {
1259 DFSClient.LOG.debug("Exception while seek to " + targetPos
1260 + " from " + getCurrentBlock() + " of " + src + " from "
1261 + currentNode, e);
1262 }
1263 }
1264 }
1265 }
1266 if (!done) {
1267 pos = targetPos;
1268 blockEnd = -1;
1269 }
1270 }
1271
1272 /**
1273 * Same as {@link #seekToNewSource(long)} except that it does not exclude
1274 * the current datanode and might connect to the same node.
1275 */
1276 private synchronized boolean seekToBlockSource(long targetPos)
1277 throws IOException {
1278 currentNode = blockSeekTo(targetPos);
1279 return true;
1280 }
1281
1282 /**
1283 * Seek to given position on a node other than the current node. If
1284 * a node other than the current node is found, then returns true.
1285 * If another node could not be found, then returns false.
1286 */
1287 @Override
1288 public synchronized boolean seekToNewSource(long targetPos) throws IOException {
1289 boolean markedDead = deadNodes.containsKey(currentNode);
1290 addToDeadNodes(currentNode);
1291 DatanodeInfo oldNode = currentNode;
1292 DatanodeInfo newNode = blockSeekTo(targetPos);
1293 if (!markedDead) {
1294 /* remove it from deadNodes. blockSeekTo could have cleared
1295 * deadNodes and added currentNode again. Thats ok. */
1296 deadNodes.remove(oldNode);
1297 }
1298 if (!oldNode.getStorageID().equals(newNode.getStorageID())) {
1299 currentNode = newNode;
1300 return true;
1301 } else {
1302 return false;
1303 }
1304 }
1305
1306 /**
1307 */
1308 @Override
1309 public synchronized long getPos() throws IOException {
1310 return pos;
1311 }
1312
1313 /** Return the size of the remaining available bytes
1314 * if the size is less than or equal to {@link Integer#MAX_VALUE},
1315 * otherwise, return {@link Integer#MAX_VALUE}.
1316 */
1317 @Override
1318 public synchronized int available() throws IOException {
1319 if (closed) {
1320 throw new IOException("Stream closed");
1321 }
1322
1323 final long remaining = getFileLength() - pos;
1324 return remaining <= Integer.MAX_VALUE? (int)remaining: Integer.MAX_VALUE;
1325 }
1326
1327 /**
1328 * We definitely don't support marks
1329 */
1330 @Override
1331 public boolean markSupported() {
1332 return false;
1333 }
1334 @Override
1335 public void mark(int readLimit) {
1336 }
1337 @Override
1338 public void reset() throws IOException {
1339 throw new IOException("Mark/reset not supported");
1340 }
1341
1342 /**
1343 * Pick the best node from which to stream the data.
1344 * Entries in <i>nodes</i> are already in the priority order
1345 */
1346 static DatanodeInfo bestNode(DatanodeInfo nodes[],
1347 AbstractMap<DatanodeInfo, DatanodeInfo> deadNodes)
1348 throws IOException {
1349 if (nodes != null) {
1350 for (int i = 0; i < nodes.length; i++) {
1351 if (!deadNodes.containsKey(nodes[i])) {
1352 return nodes[i];
1353 }
1354 }
1355 }
1356 throw new IOException("No live nodes contain current block");
1357 }
1358
1359 /** Utility class to encapsulate data node info and its address. */
1360 static class DNAddrPair {
1361 DatanodeInfo info;
1362 InetSocketAddress addr;
1363 DNAddrPair(DatanodeInfo info, InetSocketAddress addr) {
1364 this.info = info;
1365 this.addr = addr;
1366 }
1367 }
1368
1369 /**
1370 * Get statistics about the reads which this DFSInputStream has done.
1371 */
1372 public synchronized ReadStatistics getReadStatistics() {
1373 return new ReadStatistics(readStatistics);
1374 }
1375
1376 private synchronized void closeCurrentBlockReader() {
1377 if (blockReader == null) return;
1378 // Close the current block reader so that the new caching settings can
1379 // take effect immediately.
1380 try {
1381 blockReader.close();
1382 } catch (IOException e) {
1383 DFSClient.LOG.error("error closing blockReader", e);
1384 }
1385 blockReader = null;
1386 }
1387
1388 @Override
1389 public synchronized void setReadahead(Long readahead)
1390 throws IOException {
1391 this.cachingStrategy.setReadahead(readahead);
1392 closeCurrentBlockReader();
1393 }
1394
1395 @Override
1396 public synchronized void setDropBehind(Boolean dropBehind)
1397 throws IOException {
1398 this.cachingStrategy.setDropBehind(dropBehind);
1399 closeCurrentBlockReader();
1400 }
1401 }