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