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.EOFException;
021 import java.io.IOException;
022 import java.net.InetSocketAddress;
023 import java.nio.ByteBuffer;
024 import java.util.AbstractMap;
025 import java.util.ArrayList;
026 import java.util.Arrays;
027 import java.util.Collection;
028 import java.util.EnumSet;
029 import java.util.HashMap;
030 import java.util.HashSet;
031 import java.util.Iterator;
032 import java.util.List;
033 import java.util.Map;
034 import java.util.Map.Entry;
035 import java.util.Set;
036 import java.util.concurrent.Callable;
037 import java.util.concurrent.CancellationException;
038 import java.util.concurrent.CompletionService;
039 import java.util.concurrent.ConcurrentHashMap;
040 import java.util.concurrent.ExecutionException;
041 import java.util.concurrent.ExecutorCompletionService;
042 import java.util.concurrent.Future;
043 import java.util.concurrent.TimeUnit;
044 import java.util.concurrent.atomic.AtomicLong;
045
046 import org.apache.commons.io.IOUtils;
047 import org.apache.hadoop.classification.InterfaceAudience;
048 import org.apache.hadoop.fs.ByteBufferReadable;
049 import org.apache.hadoop.fs.ByteBufferUtil;
050 import org.apache.hadoop.fs.CanSetDropBehind;
051 import org.apache.hadoop.fs.CanSetReadahead;
052 import org.apache.hadoop.fs.ChecksumException;
053 import org.apache.hadoop.fs.FSInputStream;
054 import org.apache.hadoop.fs.HasEnhancedByteBufferAccess;
055 import org.apache.hadoop.fs.ReadOption;
056 import org.apache.hadoop.fs.UnresolvedLinkException;
057 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
058 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
059 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
060 import org.apache.hadoop.fs.FileEncryptionInfo;
061 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
062 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
063 import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
064 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
065 import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
066 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
067 import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
068 import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
069 import org.apache.hadoop.io.ByteBufferPool;
070 import org.apache.hadoop.ipc.RPC;
071 import org.apache.hadoop.ipc.RemoteException;
072 import org.apache.hadoop.net.NetUtils;
073 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
074 import org.apache.hadoop.security.token.Token;
075 import org.apache.hadoop.util.IdentityHashStore;
076
077 import com.google.common.annotations.VisibleForTesting;
078
079 /****************************************************************
080 * DFSInputStream provides bytes from a named file. It handles
081 * negotiation of the namenode and various datanodes as necessary.
082 ****************************************************************/
083 @InterfaceAudience.Private
084 public class DFSInputStream extends FSInputStream
085 implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
086 HasEnhancedByteBufferAccess {
087 @VisibleForTesting
088 public static boolean tcpReadsDisabledForTesting = false;
089 private long hedgedReadOpsLoopNumForTesting = 0;
090 private final DFSClient dfsClient;
091 private boolean closed = false;
092 private final String src;
093 private BlockReader blockReader = null;
094 private final boolean verifyChecksum;
095 private LocatedBlocks locatedBlocks = null;
096 private long lastBlockBeingWrittenLength = 0;
097 private FileEncryptionInfo fileEncryptionInfo = null;
098 private DatanodeInfo currentNode = null;
099 private LocatedBlock currentLocatedBlock = null;
100 private long pos = 0;
101 private long blockEnd = -1;
102 private CachingStrategy cachingStrategy;
103 private final ReadStatistics readStatistics = new ReadStatistics();
104
105 /**
106 * Track the ByteBuffers that we have handed out to readers.
107 *
108 * The value type can be either ByteBufferPool or ClientMmap, depending on
109 * whether we this is a memory-mapped buffer or not.
110 */
111 private final IdentityHashStore<ByteBuffer, Object>
112 extendedReadBuffers = new IdentityHashStore<ByteBuffer, Object>(0);
113
114 public static class ReadStatistics {
115 public ReadStatistics() {
116 this.totalBytesRead = 0;
117 this.totalLocalBytesRead = 0;
118 this.totalShortCircuitBytesRead = 0;
119 this.totalZeroCopyBytesRead = 0;
120 }
121
122 public ReadStatistics(ReadStatistics rhs) {
123 this.totalBytesRead = rhs.getTotalBytesRead();
124 this.totalLocalBytesRead = rhs.getTotalLocalBytesRead();
125 this.totalShortCircuitBytesRead = rhs.getTotalShortCircuitBytesRead();
126 this.totalZeroCopyBytesRead = rhs.getTotalZeroCopyBytesRead();
127 }
128
129 /**
130 * @return The total bytes read. This will always be at least as
131 * high as the other numbers, since it includes all of them.
132 */
133 public long getTotalBytesRead() {
134 return totalBytesRead;
135 }
136
137 /**
138 * @return The total local bytes read. This will always be at least
139 * as high as totalShortCircuitBytesRead, since all short-circuit
140 * reads are also local.
141 */
142 public long getTotalLocalBytesRead() {
143 return totalLocalBytesRead;
144 }
145
146 /**
147 * @return The total short-circuit local bytes read.
148 */
149 public long getTotalShortCircuitBytesRead() {
150 return totalShortCircuitBytesRead;
151 }
152
153 /**
154 * @return The total number of zero-copy bytes read.
155 */
156 public long getTotalZeroCopyBytesRead() {
157 return totalZeroCopyBytesRead;
158 }
159
160 /**
161 * @return The total number of bytes read which were not local.
162 */
163 public long getRemoteBytesRead() {
164 return totalBytesRead - totalLocalBytesRead;
165 }
166
167 void addRemoteBytes(long amt) {
168 this.totalBytesRead += amt;
169 }
170
171 void addLocalBytes(long amt) {
172 this.totalBytesRead += amt;
173 this.totalLocalBytesRead += amt;
174 }
175
176 void addShortCircuitBytes(long amt) {
177 this.totalBytesRead += amt;
178 this.totalLocalBytesRead += amt;
179 this.totalShortCircuitBytesRead += amt;
180 }
181
182 void addZeroCopyBytes(long amt) {
183 this.totalBytesRead += amt;
184 this.totalLocalBytesRead += amt;
185 this.totalShortCircuitBytesRead += amt;
186 this.totalZeroCopyBytesRead += amt;
187 }
188
189 private long totalBytesRead;
190
191 private long totalLocalBytesRead;
192
193 private long totalShortCircuitBytesRead;
194
195 private long totalZeroCopyBytesRead;
196 }
197
198 /**
199 * This variable tracks the number of failures since the start of the
200 * most recent user-facing operation. That is to say, it should be reset
201 * whenever the user makes a call on this stream, and if at any point
202 * during the retry logic, the failure count exceeds a threshold,
203 * the errors will be thrown back to the operation.
204 *
205 * Specifically this counts the number of times the client has gone
206 * back to the namenode to get a new list of block locations, and is
207 * capped at maxBlockAcquireFailures
208 */
209 private int failures = 0;
210
211 /* XXX Use of CocurrentHashMap is temp fix. Need to fix
212 * parallel accesses to DFSInputStream (through ptreads) properly */
213 private final ConcurrentHashMap<DatanodeInfo, DatanodeInfo> deadNodes =
214 new ConcurrentHashMap<DatanodeInfo, DatanodeInfo>();
215 private int buffersize = 1;
216
217 private final byte[] oneByteBuf = new byte[1]; // used for 'int read()'
218
219 void addToDeadNodes(DatanodeInfo dnInfo) {
220 deadNodes.put(dnInfo, dnInfo);
221 }
222
223 DFSInputStream(DFSClient dfsClient, String src, int buffersize, boolean verifyChecksum
224 ) throws IOException, UnresolvedLinkException {
225 this.dfsClient = dfsClient;
226 this.verifyChecksum = verifyChecksum;
227 this.buffersize = buffersize;
228 this.src = src;
229 this.cachingStrategy =
230 dfsClient.getDefaultReadCachingStrategy();
231 openInfo();
232 }
233
234 /**
235 * Grab the open-file info from namenode
236 */
237 synchronized void openInfo() throws IOException, UnresolvedLinkException {
238 lastBlockBeingWrittenLength = fetchLocatedBlocksAndGetLastBlockLength();
239 int retriesForLastBlockLength = dfsClient.getConf().retryTimesForGetLastBlockLength;
240 while (retriesForLastBlockLength > 0) {
241 // Getting last block length as -1 is a special case. When cluster
242 // restarts, DNs may not report immediately. At this time partial block
243 // locations will not be available with NN for getting the length. Lets
244 // retry for 3 times to get the length.
245 if (lastBlockBeingWrittenLength == -1) {
246 DFSClient.LOG.warn("Last block locations not available. "
247 + "Datanodes might not have reported blocks completely."
248 + " Will retry for " + retriesForLastBlockLength + " times");
249 waitFor(dfsClient.getConf().retryIntervalForGetLastBlockLength);
250 lastBlockBeingWrittenLength = fetchLocatedBlocksAndGetLastBlockLength();
251 } else {
252 break;
253 }
254 retriesForLastBlockLength--;
255 }
256 if (retriesForLastBlockLength == 0) {
257 throw new IOException("Could not obtain the last block locations.");
258 }
259 }
260
261 private void waitFor(int waitTime) throws IOException {
262 try {
263 Thread.sleep(waitTime);
264 } catch (InterruptedException e) {
265 throw new IOException(
266 "Interrupted while getting the last block length.");
267 }
268 }
269
270 private long fetchLocatedBlocksAndGetLastBlockLength() throws IOException {
271 final LocatedBlocks newInfo = dfsClient.getLocatedBlocks(src, 0);
272 if (DFSClient.LOG.isDebugEnabled()) {
273 DFSClient.LOG.debug("newInfo = " + newInfo);
274 }
275 if (newInfo == null) {
276 throw new IOException("Cannot open filename " + src);
277 }
278
279 if (locatedBlocks != null) {
280 Iterator<LocatedBlock> oldIter = locatedBlocks.getLocatedBlocks().iterator();
281 Iterator<LocatedBlock> newIter = newInfo.getLocatedBlocks().iterator();
282 while (oldIter.hasNext() && newIter.hasNext()) {
283 if (! oldIter.next().getBlock().equals(newIter.next().getBlock())) {
284 throw new IOException("Blocklist for " + src + " has changed!");
285 }
286 }
287 }
288 locatedBlocks = newInfo;
289 long lastBlockBeingWrittenLength = 0;
290 if (!locatedBlocks.isLastBlockComplete()) {
291 final LocatedBlock last = locatedBlocks.getLastLocatedBlock();
292 if (last != null) {
293 if (last.getLocations().length == 0) {
294 if (last.getBlockSize() == 0) {
295 // if the length is zero, then no data has been written to
296 // datanode. So no need to wait for the locations.
297 return 0;
298 }
299 return -1;
300 }
301 final long len = readBlockLength(last);
302 last.getBlock().setNumBytes(len);
303 lastBlockBeingWrittenLength = len;
304 }
305 }
306
307 fileEncryptionInfo = locatedBlocks.getFileEncryptionInfo();
308
309 currentNode = null;
310 return lastBlockBeingWrittenLength;
311 }
312
313 /** Read the block length from one of the datanodes. */
314 private long readBlockLength(LocatedBlock locatedblock) throws IOException {
315 assert locatedblock != null : "LocatedBlock cannot be null";
316 int replicaNotFoundCount = locatedblock.getLocations().length;
317
318 for(DatanodeInfo datanode : locatedblock.getLocations()) {
319 ClientDatanodeProtocol cdp = null;
320
321 try {
322 cdp = DFSUtil.createClientDatanodeProtocolProxy(datanode,
323 dfsClient.getConfiguration(), dfsClient.getConf().socketTimeout,
324 dfsClient.getConf().connectToDnViaHostname, locatedblock);
325
326 final long n = cdp.getReplicaVisibleLength(locatedblock.getBlock());
327
328 if (n >= 0) {
329 return n;
330 }
331 }
332 catch(IOException ioe) {
333 if (ioe instanceof RemoteException &&
334 (((RemoteException) ioe).unwrapRemoteException() instanceof
335 ReplicaNotFoundException)) {
336 // special case : replica might not be on the DN, treat as 0 length
337 replicaNotFoundCount--;
338 }
339
340 if (DFSClient.LOG.isDebugEnabled()) {
341 DFSClient.LOG.debug("Failed to getReplicaVisibleLength from datanode "
342 + datanode + " for block " + locatedblock.getBlock(), ioe);
343 }
344 } finally {
345 if (cdp != null) {
346 RPC.stopProxy(cdp);
347 }
348 }
349 }
350
351 // Namenode told us about these locations, but none know about the replica
352 // means that we hit the race between pipeline creation start and end.
353 // we require all 3 because some other exception could have happened
354 // on a DN that has it. we want to report that error
355 if (replicaNotFoundCount == 0) {
356 return 0;
357 }
358
359 throw new IOException("Cannot obtain block length for " + locatedblock);
360 }
361
362 public synchronized long getFileLength() {
363 return locatedBlocks == null? 0:
364 locatedBlocks.getFileLength() + lastBlockBeingWrittenLength;
365 }
366
367 // Short circuit local reads are forbidden for files that are
368 // under construction. See HDFS-2757.
369 synchronized boolean shortCircuitForbidden() {
370 return locatedBlocks.isUnderConstruction();
371 }
372
373 /**
374 * Returns the datanode from which the stream is currently reading.
375 */
376 public DatanodeInfo getCurrentDatanode() {
377 return currentNode;
378 }
379
380 /**
381 * Returns the block containing the target position.
382 */
383 synchronized public ExtendedBlock getCurrentBlock() {
384 if (currentLocatedBlock == null){
385 return null;
386 }
387 return currentLocatedBlock.getBlock();
388 }
389
390 /**
391 * Return collection of blocks that has already been located.
392 */
393 public synchronized List<LocatedBlock> getAllBlocks() throws IOException {
394 return getBlockRange(0, getFileLength());
395 }
396
397 /**
398 * Get block at the specified position.
399 * Fetch it from the namenode if not cached.
400 *
401 * @param offset block corresponding to this offset in file is returned
402 * @param updatePosition whether to update current position
403 * @return located block
404 * @throws IOException
405 */
406 private synchronized LocatedBlock getBlockAt(long offset,
407 boolean updatePosition) throws IOException {
408 assert (locatedBlocks != null) : "locatedBlocks is null";
409
410 final LocatedBlock blk;
411
412 //check offset
413 if (offset < 0 || offset >= getFileLength()) {
414 throw new IOException("offset < 0 || offset >= getFileLength(), offset="
415 + offset
416 + ", updatePosition=" + updatePosition
417 + ", locatedBlocks=" + locatedBlocks);
418 }
419 else if (offset >= locatedBlocks.getFileLength()) {
420 // offset to the portion of the last block,
421 // which is not known to the name-node yet;
422 // getting the last block
423 blk = locatedBlocks.getLastLocatedBlock();
424 }
425 else {
426 // search cached blocks first
427 int targetBlockIdx = locatedBlocks.findBlock(offset);
428 if (targetBlockIdx < 0) { // block is not cached
429 targetBlockIdx = LocatedBlocks.getInsertIndex(targetBlockIdx);
430 // fetch more blocks
431 final LocatedBlocks newBlocks = dfsClient.getLocatedBlocks(src, offset);
432 assert (newBlocks != null) : "Could not find target position " + offset;
433 locatedBlocks.insertRange(targetBlockIdx, newBlocks.getLocatedBlocks());
434 }
435 blk = locatedBlocks.get(targetBlockIdx);
436 }
437
438 // update current position
439 if (updatePosition) {
440 pos = offset;
441 blockEnd = blk.getStartOffset() + blk.getBlockSize() - 1;
442 currentLocatedBlock = blk;
443 }
444 return blk;
445 }
446
447 /** Fetch a block from namenode and cache it */
448 private synchronized void fetchBlockAt(long offset) throws IOException {
449 int targetBlockIdx = locatedBlocks.findBlock(offset);
450 if (targetBlockIdx < 0) { // block is not cached
451 targetBlockIdx = LocatedBlocks.getInsertIndex(targetBlockIdx);
452 }
453 // fetch blocks
454 final LocatedBlocks newBlocks = dfsClient.getLocatedBlocks(src, offset);
455 if (newBlocks == null) {
456 throw new IOException("Could not find target position " + offset);
457 }
458 locatedBlocks.insertRange(targetBlockIdx, newBlocks.getLocatedBlocks());
459 }
460
461 /**
462 * Get blocks in the specified range.
463 * Fetch them from the namenode if not cached. This function
464 * will not get a read request beyond the EOF.
465 * @param offset starting offset in file
466 * @param length length of data
467 * @return consequent segment of located blocks
468 * @throws IOException
469 */
470 private synchronized List<LocatedBlock> getBlockRange(long offset,
471 long length) throws IOException {
472 // getFileLength(): returns total file length
473 // locatedBlocks.getFileLength(): returns length of completed blocks
474 if (offset >= getFileLength()) {
475 throw new IOException("Offset: " + offset +
476 " exceeds file length: " + getFileLength());
477 }
478
479 final List<LocatedBlock> blocks;
480 final long lengthOfCompleteBlk = locatedBlocks.getFileLength();
481 final boolean readOffsetWithinCompleteBlk = offset < lengthOfCompleteBlk;
482 final boolean readLengthPastCompleteBlk = offset + length > lengthOfCompleteBlk;
483
484 if (readOffsetWithinCompleteBlk) {
485 //get the blocks of finalized (completed) block range
486 blocks = getFinalizedBlockRange(offset,
487 Math.min(length, lengthOfCompleteBlk - offset));
488 } else {
489 blocks = new ArrayList<LocatedBlock>(1);
490 }
491
492 // get the blocks from incomplete block range
493 if (readLengthPastCompleteBlk) {
494 blocks.add(locatedBlocks.getLastLocatedBlock());
495 }
496
497 return blocks;
498 }
499
500 /**
501 * Get blocks in the specified range.
502 * Includes only the complete blocks.
503 * Fetch them from the namenode if not cached.
504 */
505 private synchronized List<LocatedBlock> getFinalizedBlockRange(
506 long offset, long length) throws IOException {
507 assert (locatedBlocks != null) : "locatedBlocks is null";
508 List<LocatedBlock> blockRange = new ArrayList<LocatedBlock>();
509 // search cached blocks first
510 int blockIdx = locatedBlocks.findBlock(offset);
511 if (blockIdx < 0) { // block is not cached
512 blockIdx = LocatedBlocks.getInsertIndex(blockIdx);
513 }
514 long remaining = length;
515 long curOff = offset;
516 while(remaining > 0) {
517 LocatedBlock blk = null;
518 if(blockIdx < locatedBlocks.locatedBlockCount())
519 blk = locatedBlocks.get(blockIdx);
520 if (blk == null || curOff < blk.getStartOffset()) {
521 LocatedBlocks newBlocks;
522 newBlocks = dfsClient.getLocatedBlocks(src, curOff, remaining);
523 locatedBlocks.insertRange(blockIdx, newBlocks.getLocatedBlocks());
524 continue;
525 }
526 assert curOff >= blk.getStartOffset() : "Block not found";
527 blockRange.add(blk);
528 long bytesRead = blk.getStartOffset() + blk.getBlockSize() - curOff;
529 remaining -= bytesRead;
530 curOff += bytesRead;
531 blockIdx++;
532 }
533 return blockRange;
534 }
535
536 /**
537 * Open a DataInputStream to a DataNode so that it can be read from.
538 * We get block ID and the IDs of the destinations at startup, from the namenode.
539 */
540 private synchronized DatanodeInfo blockSeekTo(long target) throws IOException {
541 if (target >= getFileLength()) {
542 throw new IOException("Attempted to read past end of file");
543 }
544
545 // Will be getting a new BlockReader.
546 if (blockReader != null) {
547 blockReader.close();
548 blockReader = null;
549 }
550
551 //
552 // Connect to best DataNode for desired Block, with potential offset
553 //
554 DatanodeInfo chosenNode = null;
555 int refetchToken = 1; // only need to get a new access token once
556 int refetchEncryptionKey = 1; // only need to get a new encryption key once
557
558 boolean connectFailedOnce = false;
559
560 while (true) {
561 //
562 // Compute desired block
563 //
564 LocatedBlock targetBlock = getBlockAt(target, true);
565 assert (target==pos) : "Wrong postion " + pos + " expect " + target;
566 long offsetIntoBlock = target - targetBlock.getStartOffset();
567
568 DNAddrPair retval = chooseDataNode(targetBlock, null);
569 chosenNode = retval.info;
570 InetSocketAddress targetAddr = retval.addr;
571 StorageType storageType = retval.storageType;
572
573 try {
574 ExtendedBlock blk = targetBlock.getBlock();
575 Token<BlockTokenIdentifier> accessToken = targetBlock.getBlockToken();
576 blockReader = new BlockReaderFactory(dfsClient.getConf()).
577 setInetSocketAddress(targetAddr).
578 setRemotePeerFactory(dfsClient).
579 setDatanodeInfo(chosenNode).
580 setStorageType(storageType).
581 setFileName(src).
582 setBlock(blk).
583 setBlockToken(accessToken).
584 setStartOffset(offsetIntoBlock).
585 setVerifyChecksum(verifyChecksum).
586 setClientName(dfsClient.clientName).
587 setLength(blk.getNumBytes() - offsetIntoBlock).
588 setCachingStrategy(cachingStrategy).
589 setAllowShortCircuitLocalReads(!shortCircuitForbidden()).
590 setClientCacheContext(dfsClient.getClientContext()).
591 setUserGroupInformation(dfsClient.ugi).
592 setConfiguration(dfsClient.getConfiguration()).
593 build();
594 if(connectFailedOnce) {
595 DFSClient.LOG.info("Successfully connected to " + targetAddr +
596 " for " + blk);
597 }
598 return chosenNode;
599 } catch (IOException ex) {
600 if (ex instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
601 DFSClient.LOG.info("Will fetch a new encryption key and retry, "
602 + "encryption key was invalid when connecting to " + targetAddr
603 + " : " + ex);
604 // The encryption key used is invalid.
605 refetchEncryptionKey--;
606 dfsClient.clearDataEncryptionKey();
607 } else if (refetchToken > 0 && tokenRefetchNeeded(ex, targetAddr)) {
608 refetchToken--;
609 fetchBlockAt(target);
610 } else {
611 connectFailedOnce = true;
612 DFSClient.LOG.warn("Failed to connect to " + targetAddr + " for block"
613 + ", add to deadNodes and continue. " + ex, ex);
614 // Put chosen node into dead list, continue
615 addToDeadNodes(chosenNode);
616 }
617 }
618 }
619 }
620
621 /**
622 * Close it down!
623 */
624 @Override
625 public synchronized void close() throws IOException {
626 if (closed) {
627 return;
628 }
629 dfsClient.checkOpen();
630
631 if (!extendedReadBuffers.isEmpty()) {
632 final StringBuilder builder = new StringBuilder();
633 extendedReadBuffers.visitAll(new IdentityHashStore.Visitor<ByteBuffer, Object>() {
634 private String prefix = "";
635 @Override
636 public void accept(ByteBuffer k, Object v) {
637 builder.append(prefix).append(k);
638 prefix = ", ";
639 }
640 });
641 DFSClient.LOG.warn("closing file " + src + ", but there are still " +
642 "unreleased ByteBuffers allocated by read(). " +
643 "Please release " + builder.toString() + ".");
644 }
645 if (blockReader != null) {
646 blockReader.close();
647 blockReader = null;
648 }
649 super.close();
650 closed = true;
651 }
652
653 @Override
654 public synchronized int read() throws IOException {
655 int ret = read( oneByteBuf, 0, 1 );
656 return ( ret <= 0 ) ? -1 : (oneByteBuf[0] & 0xff);
657 }
658
659 /**
660 * Wraps different possible read implementations so that readBuffer can be
661 * strategy-agnostic.
662 */
663 private interface ReaderStrategy {
664 public int doRead(BlockReader blockReader, int off, int len,
665 ReadStatistics readStatistics) throws ChecksumException, IOException;
666 }
667
668 private static void updateReadStatistics(ReadStatistics readStatistics,
669 int nRead, BlockReader blockReader) {
670 if (nRead <= 0) return;
671 if (blockReader.isShortCircuit()) {
672 readStatistics.addShortCircuitBytes(nRead);
673 } else if (blockReader.isLocal()) {
674 readStatistics.addLocalBytes(nRead);
675 } else {
676 readStatistics.addRemoteBytes(nRead);
677 }
678 }
679
680 /**
681 * Used to read bytes into a byte[]
682 */
683 private static class ByteArrayStrategy implements ReaderStrategy {
684 final byte[] buf;
685
686 public ByteArrayStrategy(byte[] buf) {
687 this.buf = buf;
688 }
689
690 @Override
691 public int doRead(BlockReader blockReader, int off, int len,
692 ReadStatistics readStatistics) throws ChecksumException, IOException {
693 int nRead = blockReader.read(buf, off, len);
694 updateReadStatistics(readStatistics, nRead, blockReader);
695 return nRead;
696 }
697 }
698
699 /**
700 * Used to read bytes into a user-supplied ByteBuffer
701 */
702 private static class ByteBufferStrategy implements ReaderStrategy {
703 final ByteBuffer buf;
704 ByteBufferStrategy(ByteBuffer buf) {
705 this.buf = buf;
706 }
707
708 @Override
709 public int doRead(BlockReader blockReader, int off, int len,
710 ReadStatistics readStatistics) throws ChecksumException, IOException {
711 int oldpos = buf.position();
712 int oldlimit = buf.limit();
713 boolean success = false;
714 try {
715 int ret = blockReader.read(buf);
716 success = true;
717 updateReadStatistics(readStatistics, ret, blockReader);
718 return ret;
719 } finally {
720 if (!success) {
721 // Reset to original state so that retries work correctly.
722 buf.position(oldpos);
723 buf.limit(oldlimit);
724 }
725 }
726 }
727 }
728
729 /* This is a used by regular read() and handles ChecksumExceptions.
730 * name readBuffer() is chosen to imply similarity to readBuffer() in
731 * ChecksumFileSystem
732 */
733 private synchronized int readBuffer(ReaderStrategy reader, int off, int len,
734 Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
735 throws IOException {
736 IOException ioe;
737
738 /* we retry current node only once. So this is set to true only here.
739 * Intention is to handle one common case of an error that is not a
740 * failure on datanode or client : when DataNode closes the connection
741 * since client is idle. If there are other cases of "non-errors" then
742 * then a datanode might be retried by setting this to true again.
743 */
744 boolean retryCurrentNode = true;
745
746 while (true) {
747 // retry as many times as seekToNewSource allows.
748 try {
749 return reader.doRead(blockReader, off, len, readStatistics);
750 } catch ( ChecksumException ce ) {
751 DFSClient.LOG.warn("Found Checksum error for "
752 + getCurrentBlock() + " from " + currentNode
753 + " at " + ce.getPos());
754 ioe = ce;
755 retryCurrentNode = false;
756 // we want to remember which block replicas we have tried
757 addIntoCorruptedBlockMap(getCurrentBlock(), currentNode,
758 corruptedBlockMap);
759 } catch ( IOException e ) {
760 if (!retryCurrentNode) {
761 DFSClient.LOG.warn("Exception while reading from "
762 + getCurrentBlock() + " of " + src + " from "
763 + currentNode, e);
764 }
765 ioe = e;
766 }
767 boolean sourceFound = false;
768 if (retryCurrentNode) {
769 /* possibly retry the same node so that transient errors don't
770 * result in application level failures (e.g. Datanode could have
771 * closed the connection because the client is idle for too long).
772 */
773 sourceFound = seekToBlockSource(pos);
774 } else {
775 addToDeadNodes(currentNode);
776 sourceFound = seekToNewSource(pos);
777 }
778 if (!sourceFound) {
779 throw ioe;
780 }
781 retryCurrentNode = false;
782 }
783 }
784
785 private int readWithStrategy(ReaderStrategy strategy, int off, int len) throws IOException {
786 dfsClient.checkOpen();
787 if (closed) {
788 throw new IOException("Stream closed");
789 }
790 Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap
791 = new HashMap<ExtendedBlock, Set<DatanodeInfo>>();
792 failures = 0;
793 if (pos < getFileLength()) {
794 int retries = 2;
795 while (retries > 0) {
796 try {
797 // currentNode can be left as null if previous read had a checksum
798 // error on the same block. See HDFS-3067
799 if (pos > blockEnd || currentNode == null) {
800 currentNode = blockSeekTo(pos);
801 }
802 int realLen = (int) Math.min(len, (blockEnd - pos + 1L));
803 if (locatedBlocks.isLastBlockComplete()) {
804 realLen = (int) Math.min(realLen, locatedBlocks.getFileLength());
805 }
806 int result = readBuffer(strategy, off, realLen, corruptedBlockMap);
807
808 if (result >= 0) {
809 pos += result;
810 } else {
811 // got a EOS from reader though we expect more data on it.
812 throw new IOException("Unexpected EOS from the reader");
813 }
814 if (dfsClient.stats != null) {
815 dfsClient.stats.incrementBytesRead(result);
816 }
817 return result;
818 } catch (ChecksumException ce) {
819 throw ce;
820 } catch (IOException e) {
821 if (retries == 1) {
822 DFSClient.LOG.warn("DFS Read", e);
823 }
824 blockEnd = -1;
825 if (currentNode != null) { addToDeadNodes(currentNode); }
826 if (--retries == 0) {
827 throw e;
828 }
829 } finally {
830 // Check if need to report block replicas corruption either read
831 // was successful or ChecksumException occured.
832 reportCheckSumFailure(corruptedBlockMap,
833 currentLocatedBlock.getLocations().length);
834 }
835 }
836 }
837 return -1;
838 }
839
840 /**
841 * Read the entire buffer.
842 */
843 @Override
844 public synchronized int read(final byte buf[], int off, int len) throws IOException {
845 ReaderStrategy byteArrayReader = new ByteArrayStrategy(buf);
846
847 return readWithStrategy(byteArrayReader, off, len);
848 }
849
850 @Override
851 public synchronized int read(final ByteBuffer buf) throws IOException {
852 ReaderStrategy byteBufferReader = new ByteBufferStrategy(buf);
853
854 return readWithStrategy(byteBufferReader, 0, buf.remaining());
855 }
856
857
858 /**
859 * Add corrupted block replica into map.
860 */
861 private void addIntoCorruptedBlockMap(ExtendedBlock blk, DatanodeInfo node,
862 Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
863 Set<DatanodeInfo> dnSet = null;
864 if((corruptedBlockMap.containsKey(blk))) {
865 dnSet = corruptedBlockMap.get(blk);
866 }else {
867 dnSet = new HashSet<DatanodeInfo>();
868 }
869 if (!dnSet.contains(node)) {
870 dnSet.add(node);
871 corruptedBlockMap.put(blk, dnSet);
872 }
873 }
874
875 private DNAddrPair chooseDataNode(LocatedBlock block,
876 Collection<DatanodeInfo> ignoredNodes) throws IOException {
877 while (true) {
878 try {
879 return getBestNodeDNAddrPair(block, ignoredNodes);
880 } catch (IOException ie) {
881 String errMsg = getBestNodeDNAddrPairErrorString(block.getLocations(),
882 deadNodes, ignoredNodes);
883 String blockInfo = block.getBlock() + " file=" + src;
884 if (failures >= dfsClient.getMaxBlockAcquireFailures()) {
885 String description = "Could not obtain block: " + blockInfo;
886 DFSClient.LOG.warn(description + errMsg
887 + ". Throwing a BlockMissingException");
888 throw new BlockMissingException(src, description,
889 block.getStartOffset());
890 }
891
892 DatanodeInfo[] nodes = block.getLocations();
893 if (nodes == null || nodes.length == 0) {
894 DFSClient.LOG.info("No node available for " + blockInfo);
895 }
896 DFSClient.LOG.info("Could not obtain " + block.getBlock()
897 + " from any node: " + ie + errMsg
898 + ". Will get new block locations from namenode and retry...");
899 try {
900 // Introducing a random factor to the wait time before another retry.
901 // The wait time is dependent on # of failures and a random factor.
902 // At the first time of getting a BlockMissingException, the wait time
903 // is a random number between 0..3000 ms. If the first retry
904 // still fails, we will wait 3000 ms grace period before the 2nd retry.
905 // Also at the second retry, the waiting window is expanded to 6000 ms
906 // alleviating the request rate from the server. Similarly the 3rd retry
907 // will wait 6000ms grace period before retry and the waiting window is
908 // expanded to 9000ms.
909 final int timeWindow = dfsClient.getConf().timeWindow;
910 double waitTime = timeWindow * failures + // grace period for the last round of attempt
911 timeWindow * (failures + 1) * DFSUtil.getRandom().nextDouble(); // expanding time window for each failure
912 DFSClient.LOG.warn("DFS chooseDataNode: got # " + (failures + 1) + " IOException, will wait for " + waitTime + " msec.");
913 Thread.sleep((long)waitTime);
914 } catch (InterruptedException iex) {
915 }
916 deadNodes.clear(); //2nd option is to remove only nodes[blockId]
917 openInfo();
918 block = getBlockAt(block.getStartOffset(), false);
919 failures++;
920 continue;
921 }
922 }
923 }
924
925 /**
926 * Get the best node from which to stream the data.
927 * @param block LocatedBlock, containing nodes in priority order.
928 * @param ignoredNodes Do not choose nodes in this array (may be null)
929 * @return The DNAddrPair of the best node.
930 * @throws IOException
931 */
932 private DNAddrPair getBestNodeDNAddrPair(LocatedBlock block,
933 Collection<DatanodeInfo> ignoredNodes) throws IOException {
934 DatanodeInfo[] nodes = block.getLocations();
935 StorageType[] storageTypes = block.getStorageTypes();
936 DatanodeInfo chosenNode = null;
937 StorageType storageType = null;
938 if (nodes != null) {
939 for (int i = 0; i < nodes.length; i++) {
940 if (!deadNodes.containsKey(nodes[i])
941 && (ignoredNodes == null || !ignoredNodes.contains(nodes[i]))) {
942 chosenNode = nodes[i];
943 // Storage types are ordered to correspond with nodes, so use the same
944 // index to get storage type.
945 if (storageTypes != null && i < storageTypes.length) {
946 storageType = storageTypes[i];
947 }
948 break;
949 }
950 }
951 }
952 if (chosenNode == null) {
953 throw new IOException("No live nodes contain block " + block.getBlock() +
954 " after checking nodes = " + Arrays.toString(nodes) +
955 ", ignoredNodes = " + ignoredNodes);
956 }
957 final String dnAddr =
958 chosenNode.getXferAddr(dfsClient.getConf().connectToDnViaHostname);
959 if (DFSClient.LOG.isDebugEnabled()) {
960 DFSClient.LOG.debug("Connecting to datanode " + dnAddr);
961 }
962 InetSocketAddress targetAddr = NetUtils.createSocketAddr(dnAddr);
963 return new DNAddrPair(chosenNode, targetAddr, storageType);
964 }
965
966 private static String getBestNodeDNAddrPairErrorString(
967 DatanodeInfo nodes[], AbstractMap<DatanodeInfo,
968 DatanodeInfo> deadNodes, Collection<DatanodeInfo> ignoredNodes) {
969 StringBuilder errMsgr = new StringBuilder(
970 " No live nodes contain current block ");
971 errMsgr.append("Block locations:");
972 for (DatanodeInfo datanode : nodes) {
973 errMsgr.append(" ");
974 errMsgr.append(datanode.toString());
975 }
976 errMsgr.append(" Dead nodes: ");
977 for (DatanodeInfo datanode : deadNodes.keySet()) {
978 errMsgr.append(" ");
979 errMsgr.append(datanode.toString());
980 }
981 if (ignoredNodes != null) {
982 errMsgr.append(" Ignored nodes: ");
983 for (DatanodeInfo datanode : ignoredNodes) {
984 errMsgr.append(" ");
985 errMsgr.append(datanode.toString());
986 }
987 }
988 return errMsgr.toString();
989 }
990
991 private void fetchBlockByteRange(LocatedBlock block, long start, long end,
992 byte[] buf, int offset,
993 Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
994 throws IOException {
995 block = getBlockAt(block.getStartOffset(), false);
996 while (true) {
997 DNAddrPair addressPair = chooseDataNode(block, null);
998 try {
999 actualGetFromOneDataNode(addressPair, block, start, end, buf, offset,
1000 corruptedBlockMap);
1001 return;
1002 } catch (IOException e) {
1003 // Ignore. Already processed inside the function.
1004 // Loop through to try the next node.
1005 }
1006 }
1007 }
1008
1009 private Callable<ByteBuffer> getFromOneDataNode(final DNAddrPair datanode,
1010 final LocatedBlock block, final long start, final long end,
1011 final ByteBuffer bb,
1012 final Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
1013 return new Callable<ByteBuffer>() {
1014 @Override
1015 public ByteBuffer call() throws Exception {
1016 byte[] buf = bb.array();
1017 int offset = bb.position();
1018 actualGetFromOneDataNode(datanode, block, start, end, buf, offset,
1019 corruptedBlockMap);
1020 return bb;
1021 }
1022 };
1023 }
1024
1025 private void actualGetFromOneDataNode(final DNAddrPair datanode,
1026 LocatedBlock block, final long start, final long end, byte[] buf,
1027 int offset, Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
1028 throws IOException {
1029 DFSClientFaultInjector.get().startFetchFromDatanode();
1030 int refetchToken = 1; // only need to get a new access token once
1031 int refetchEncryptionKey = 1; // only need to get a new encryption key once
1032
1033 while (true) {
1034 // cached block locations may have been updated by chooseDataNode()
1035 // or fetchBlockAt(). Always get the latest list of locations at the
1036 // start of the loop.
1037 CachingStrategy curCachingStrategy;
1038 boolean allowShortCircuitLocalReads;
1039 synchronized (this) {
1040 block = getBlockAt(block.getStartOffset(), false);
1041 curCachingStrategy = cachingStrategy;
1042 allowShortCircuitLocalReads = !shortCircuitForbidden();
1043 }
1044 DatanodeInfo chosenNode = datanode.info;
1045 InetSocketAddress targetAddr = datanode.addr;
1046 StorageType storageType = datanode.storageType;
1047 BlockReader reader = null;
1048
1049 try {
1050 DFSClientFaultInjector.get().fetchFromDatanodeException();
1051 Token<BlockTokenIdentifier> blockToken = block.getBlockToken();
1052 int len = (int) (end - start + 1);
1053 reader = new BlockReaderFactory(dfsClient.getConf()).
1054 setInetSocketAddress(targetAddr).
1055 setRemotePeerFactory(dfsClient).
1056 setDatanodeInfo(chosenNode).
1057 setStorageType(storageType).
1058 setFileName(src).
1059 setBlock(block.getBlock()).
1060 setBlockToken(blockToken).
1061 setStartOffset(start).
1062 setVerifyChecksum(verifyChecksum).
1063 setClientName(dfsClient.clientName).
1064 setLength(len).
1065 setCachingStrategy(curCachingStrategy).
1066 setAllowShortCircuitLocalReads(allowShortCircuitLocalReads).
1067 setClientCacheContext(dfsClient.getClientContext()).
1068 setUserGroupInformation(dfsClient.ugi).
1069 setConfiguration(dfsClient.getConfiguration()).
1070 build();
1071 int nread = reader.readAll(buf, offset, len);
1072 updateReadStatistics(readStatistics, nread, reader);
1073
1074 if (nread != len) {
1075 throw new IOException("truncated return from reader.read(): " +
1076 "excpected " + len + ", got " + nread);
1077 }
1078 DFSClientFaultInjector.get().readFromDatanodeDelay();
1079 return;
1080 } catch (ChecksumException e) {
1081 String msg = "fetchBlockByteRange(). Got a checksum exception for "
1082 + src + " at " + block.getBlock() + ":" + e.getPos() + " from "
1083 + chosenNode;
1084 DFSClient.LOG.warn(msg);
1085 // we want to remember what we have tried
1086 addIntoCorruptedBlockMap(block.getBlock(), chosenNode, corruptedBlockMap);
1087 addToDeadNodes(chosenNode);
1088 throw new IOException(msg);
1089 } catch (IOException e) {
1090 if (e instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
1091 DFSClient.LOG.info("Will fetch a new encryption key and retry, "
1092 + "encryption key was invalid when connecting to " + targetAddr
1093 + " : " + e);
1094 // The encryption key used is invalid.
1095 refetchEncryptionKey--;
1096 dfsClient.clearDataEncryptionKey();
1097 continue;
1098 } else if (refetchToken > 0 && tokenRefetchNeeded(e, targetAddr)) {
1099 refetchToken--;
1100 try {
1101 fetchBlockAt(block.getStartOffset());
1102 } catch (IOException fbae) {
1103 // ignore IOE, since we can retry it later in a loop
1104 }
1105 continue;
1106 } else {
1107 String msg = "Failed to connect to " + targetAddr + " for file "
1108 + src + " for block " + block.getBlock() + ":" + e;
1109 DFSClient.LOG.warn("Connection failure: " + msg, e);
1110 addToDeadNodes(chosenNode);
1111 throw new IOException(msg);
1112 }
1113 } finally {
1114 if (reader != null) {
1115 reader.close();
1116 }
1117 }
1118 }
1119 }
1120
1121 /**
1122 * Like {@link #fetchBlockByteRange(LocatedBlock, long, long, byte[],
1123 * int, Map)} except we start up a second, parallel, 'hedged' read
1124 * if the first read is taking longer than configured amount of
1125 * time. We then wait on which ever read returns first.
1126 */
1127 private void hedgedFetchBlockByteRange(LocatedBlock block, long start,
1128 long end, byte[] buf, int offset,
1129 Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
1130 throws IOException {
1131 ArrayList<Future<ByteBuffer>> futures = new ArrayList<Future<ByteBuffer>>();
1132 CompletionService<ByteBuffer> hedgedService =
1133 new ExecutorCompletionService<ByteBuffer>(
1134 dfsClient.getHedgedReadsThreadPool());
1135 ArrayList<DatanodeInfo> ignored = new ArrayList<DatanodeInfo>();
1136 ByteBuffer bb = null;
1137 int len = (int) (end - start + 1);
1138 block = getBlockAt(block.getStartOffset(), false);
1139 while (true) {
1140 // see HDFS-6591, this metric is used to verify/catch unnecessary loops
1141 hedgedReadOpsLoopNumForTesting++;
1142 DNAddrPair chosenNode = null;
1143 // there is no request already executing.
1144 if (futures.isEmpty()) {
1145 // chooseDataNode is a commitment. If no node, we go to
1146 // the NN to reget block locations. Only go here on first read.
1147 chosenNode = chooseDataNode(block, ignored);
1148 bb = ByteBuffer.wrap(buf, offset, len);
1149 Callable<ByteBuffer> getFromDataNodeCallable = getFromOneDataNode(
1150 chosenNode, block, start, end, bb, corruptedBlockMap);
1151 Future<ByteBuffer> firstRequest = hedgedService
1152 .submit(getFromDataNodeCallable);
1153 futures.add(firstRequest);
1154 try {
1155 Future<ByteBuffer> future = hedgedService.poll(
1156 dfsClient.getHedgedReadTimeout(), TimeUnit.MILLISECONDS);
1157 if (future != null) {
1158 future.get();
1159 return;
1160 }
1161 if (DFSClient.LOG.isDebugEnabled()) {
1162 DFSClient.LOG.debug("Waited " + dfsClient.getHedgedReadTimeout()
1163 + "ms to read from " + chosenNode.info
1164 + "; spawning hedged read");
1165 }
1166 // Ignore this node on next go around.
1167 ignored.add(chosenNode.info);
1168 dfsClient.getHedgedReadMetrics().incHedgedReadOps();
1169 continue; // no need to refresh block locations
1170 } catch (InterruptedException e) {
1171 // Ignore
1172 } catch (ExecutionException e) {
1173 // Ignore already logged in the call.
1174 }
1175 } else {
1176 // We are starting up a 'hedged' read. We have a read already
1177 // ongoing. Call getBestNodeDNAddrPair instead of chooseDataNode.
1178 // If no nodes to do hedged reads against, pass.
1179 try {
1180 try {
1181 chosenNode = getBestNodeDNAddrPair(block, ignored);
1182 } catch (IOException ioe) {
1183 chosenNode = chooseDataNode(block, ignored);
1184 }
1185 bb = ByteBuffer.allocate(len);
1186 Callable<ByteBuffer> getFromDataNodeCallable = getFromOneDataNode(
1187 chosenNode, block, start, end, bb, corruptedBlockMap);
1188 Future<ByteBuffer> oneMoreRequest = hedgedService
1189 .submit(getFromDataNodeCallable);
1190 futures.add(oneMoreRequest);
1191 } catch (IOException ioe) {
1192 if (DFSClient.LOG.isDebugEnabled()) {
1193 DFSClient.LOG.debug("Failed getting node for hedged read: "
1194 + ioe.getMessage());
1195 }
1196 }
1197 // if not succeeded. Submit callables for each datanode in a loop, wait
1198 // for a fixed interval and get the result from the fastest one.
1199 try {
1200 ByteBuffer result = getFirstToComplete(hedgedService, futures);
1201 // cancel the rest.
1202 cancelAll(futures);
1203 if (result.array() != buf) { // compare the array pointers
1204 dfsClient.getHedgedReadMetrics().incHedgedReadWins();
1205 System.arraycopy(result.array(), result.position(), buf, offset,
1206 len);
1207 } else {
1208 dfsClient.getHedgedReadMetrics().incHedgedReadOps();
1209 }
1210 return;
1211 } catch (InterruptedException ie) {
1212 // Ignore and retry
1213 }
1214 // We got here if exception. Ignore this node on next go around IFF
1215 // we found a chosenNode to hedge read against.
1216 if (chosenNode != null && chosenNode.info != null) {
1217 ignored.add(chosenNode.info);
1218 }
1219 }
1220 }
1221 }
1222
1223 @VisibleForTesting
1224 public long getHedgedReadOpsLoopNumForTesting() {
1225 return hedgedReadOpsLoopNumForTesting;
1226 }
1227
1228 private ByteBuffer getFirstToComplete(
1229 CompletionService<ByteBuffer> hedgedService,
1230 ArrayList<Future<ByteBuffer>> futures) throws InterruptedException {
1231 if (futures.isEmpty()) {
1232 throw new InterruptedException("let's retry");
1233 }
1234 Future<ByteBuffer> future = null;
1235 try {
1236 future = hedgedService.take();
1237 ByteBuffer bb = future.get();
1238 futures.remove(future);
1239 return bb;
1240 } catch (ExecutionException e) {
1241 // already logged in the Callable
1242 futures.remove(future);
1243 } catch (CancellationException ce) {
1244 // already logged in the Callable
1245 futures.remove(future);
1246 }
1247
1248 throw new InterruptedException("let's retry");
1249 }
1250
1251 private void cancelAll(List<Future<ByteBuffer>> futures) {
1252 for (Future<ByteBuffer> future : futures) {
1253 // Unfortunately, hdfs reads do not take kindly to interruption.
1254 // Threads return a variety of interrupted-type exceptions but
1255 // also complaints about invalid pbs -- likely because read
1256 // is interrupted before gets whole pb. Also verbose WARN
1257 // logging. So, for now, do not interrupt running read.
1258 future.cancel(false);
1259 }
1260 }
1261
1262 /**
1263 * Should the block access token be refetched on an exception
1264 *
1265 * @param ex Exception received
1266 * @param targetAddr Target datanode address from where exception was received
1267 * @return true if block access token has expired or invalid and it should be
1268 * refetched
1269 */
1270 private static boolean tokenRefetchNeeded(IOException ex,
1271 InetSocketAddress targetAddr) {
1272 /*
1273 * Get a new access token and retry. Retry is needed in 2 cases. 1)
1274 * When both NN and DN re-started while DFSClient holding a cached
1275 * access token. 2) In the case that NN fails to update its
1276 * access key at pre-set interval (by a wide margin) and
1277 * subsequently restarts. In this case, DN re-registers itself with
1278 * NN and receives a new access key, but DN will delete the old
1279 * access key from its memory since it's considered expired based on
1280 * the estimated expiration date.
1281 */
1282 if (ex instanceof InvalidBlockTokenException || ex instanceof InvalidToken) {
1283 DFSClient.LOG.info("Access token was invalid when connecting to "
1284 + targetAddr + " : " + ex);
1285 return true;
1286 }
1287 return false;
1288 }
1289
1290 /**
1291 * Read bytes starting from the specified position.
1292 *
1293 * @param position start read from this position
1294 * @param buffer read buffer
1295 * @param offset offset into buffer
1296 * @param length number of bytes to read
1297 *
1298 * @return actual number of bytes read
1299 */
1300 @Override
1301 public int read(long position, byte[] buffer, int offset, int length)
1302 throws IOException {
1303 // sanity checks
1304 dfsClient.checkOpen();
1305 if (closed) {
1306 throw new IOException("Stream closed");
1307 }
1308 failures = 0;
1309 long filelen = getFileLength();
1310 if ((position < 0) || (position >= filelen)) {
1311 return -1;
1312 }
1313 int realLen = length;
1314 if ((position + length) > filelen) {
1315 realLen = (int)(filelen - position);
1316 }
1317
1318 // determine the block and byte range within the block
1319 // corresponding to position and realLen
1320 List<LocatedBlock> blockRange = getBlockRange(position, realLen);
1321 int remaining = realLen;
1322 Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap
1323 = new HashMap<ExtendedBlock, Set<DatanodeInfo>>();
1324 for (LocatedBlock blk : blockRange) {
1325 long targetStart = position - blk.getStartOffset();
1326 long bytesToRead = Math.min(remaining, blk.getBlockSize() - targetStart);
1327 try {
1328 if (dfsClient.isHedgedReadsEnabled()) {
1329 hedgedFetchBlockByteRange(blk, targetStart, targetStart + bytesToRead
1330 - 1, buffer, offset, corruptedBlockMap);
1331 } else {
1332 fetchBlockByteRange(blk, targetStart, targetStart + bytesToRead - 1,
1333 buffer, offset, corruptedBlockMap);
1334 }
1335 } finally {
1336 // Check and report if any block replicas are corrupted.
1337 // BlockMissingException may be caught if all block replicas are
1338 // corrupted.
1339 reportCheckSumFailure(corruptedBlockMap, blk.getLocations().length);
1340 }
1341
1342 remaining -= bytesToRead;
1343 position += bytesToRead;
1344 offset += bytesToRead;
1345 }
1346 assert remaining == 0 : "Wrong number of bytes read.";
1347 if (dfsClient.stats != null) {
1348 dfsClient.stats.incrementBytesRead(realLen);
1349 }
1350 return realLen;
1351 }
1352
1353 /**
1354 * DFSInputStream reports checksum failure.
1355 * Case I : client has tried multiple data nodes and at least one of the
1356 * attempts has succeeded. We report the other failures as corrupted block to
1357 * namenode.
1358 * Case II: client has tried out all data nodes, but all failed. We
1359 * only report if the total number of replica is 1. We do not
1360 * report otherwise since this maybe due to the client is a handicapped client
1361 * (who can not read).
1362 * @param corruptedBlockMap map of corrupted blocks
1363 * @param dataNodeCount number of data nodes who contains the block replicas
1364 */
1365 private void reportCheckSumFailure(
1366 Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap,
1367 int dataNodeCount) {
1368 if (corruptedBlockMap.isEmpty()) {
1369 return;
1370 }
1371 Iterator<Entry<ExtendedBlock, Set<DatanodeInfo>>> it = corruptedBlockMap
1372 .entrySet().iterator();
1373 Entry<ExtendedBlock, Set<DatanodeInfo>> entry = it.next();
1374 ExtendedBlock blk = entry.getKey();
1375 Set<DatanodeInfo> dnSet = entry.getValue();
1376 if (((dnSet.size() < dataNodeCount) && (dnSet.size() > 0))
1377 || ((dataNodeCount == 1) && (dnSet.size() == dataNodeCount))) {
1378 DatanodeInfo[] locs = new DatanodeInfo[dnSet.size()];
1379 int i = 0;
1380 for (DatanodeInfo dn:dnSet) {
1381 locs[i++] = dn;
1382 }
1383 LocatedBlock [] lblocks = { new LocatedBlock(blk, locs) };
1384 dfsClient.reportChecksumFailure(src, lblocks);
1385 }
1386 corruptedBlockMap.clear();
1387 }
1388
1389 @Override
1390 public long skip(long n) throws IOException {
1391 if ( n > 0 ) {
1392 long curPos = getPos();
1393 long fileLen = getFileLength();
1394 if( n+curPos > fileLen ) {
1395 n = fileLen - curPos;
1396 }
1397 seek(curPos+n);
1398 return n;
1399 }
1400 return n < 0 ? -1 : 0;
1401 }
1402
1403 /**
1404 * Seek to a new arbitrary location
1405 */
1406 @Override
1407 public synchronized void seek(long targetPos) throws IOException {
1408 if (targetPos > getFileLength()) {
1409 throw new EOFException("Cannot seek after EOF");
1410 }
1411 if (targetPos < 0) {
1412 throw new EOFException("Cannot seek to negative offset");
1413 }
1414 if (closed) {
1415 throw new IOException("Stream is closed!");
1416 }
1417 boolean done = false;
1418 if (pos <= targetPos && targetPos <= blockEnd) {
1419 //
1420 // If this seek is to a positive position in the current
1421 // block, and this piece of data might already be lying in
1422 // the TCP buffer, then just eat up the intervening data.
1423 //
1424 int diff = (int)(targetPos - pos);
1425 if (diff <= blockReader.available()) {
1426 try {
1427 pos += blockReader.skip(diff);
1428 if (pos == targetPos) {
1429 done = true;
1430 } else {
1431 // The range was already checked. If the block reader returns
1432 // something unexpected instead of throwing an exception, it is
1433 // most likely a bug.
1434 String errMsg = "BlockReader failed to seek to " +
1435 targetPos + ". Instead, it seeked to " + pos + ".";
1436 DFSClient.LOG.warn(errMsg);
1437 throw new IOException(errMsg);
1438 }
1439 } catch (IOException e) {//make following read to retry
1440 if(DFSClient.LOG.isDebugEnabled()) {
1441 DFSClient.LOG.debug("Exception while seek to " + targetPos
1442 + " from " + getCurrentBlock() + " of " + src + " from "
1443 + currentNode, e);
1444 }
1445 }
1446 }
1447 }
1448 if (!done) {
1449 pos = targetPos;
1450 blockEnd = -1;
1451 }
1452 }
1453
1454 /**
1455 * Same as {@link #seekToNewSource(long)} except that it does not exclude
1456 * the current datanode and might connect to the same node.
1457 */
1458 private synchronized boolean seekToBlockSource(long targetPos)
1459 throws IOException {
1460 currentNode = blockSeekTo(targetPos);
1461 return true;
1462 }
1463
1464 /**
1465 * Seek to given position on a node other than the current node. If
1466 * a node other than the current node is found, then returns true.
1467 * If another node could not be found, then returns false.
1468 */
1469 @Override
1470 public synchronized boolean seekToNewSource(long targetPos) throws IOException {
1471 boolean markedDead = deadNodes.containsKey(currentNode);
1472 addToDeadNodes(currentNode);
1473 DatanodeInfo oldNode = currentNode;
1474 DatanodeInfo newNode = blockSeekTo(targetPos);
1475 if (!markedDead) {
1476 /* remove it from deadNodes. blockSeekTo could have cleared
1477 * deadNodes and added currentNode again. Thats ok. */
1478 deadNodes.remove(oldNode);
1479 }
1480 if (!oldNode.getDatanodeUuid().equals(newNode.getDatanodeUuid())) {
1481 currentNode = newNode;
1482 return true;
1483 } else {
1484 return false;
1485 }
1486 }
1487
1488 /**
1489 */
1490 @Override
1491 public synchronized long getPos() throws IOException {
1492 return pos;
1493 }
1494
1495 /** Return the size of the remaining available bytes
1496 * if the size is less than or equal to {@link Integer#MAX_VALUE},
1497 * otherwise, return {@link Integer#MAX_VALUE}.
1498 */
1499 @Override
1500 public synchronized int available() throws IOException {
1501 if (closed) {
1502 throw new IOException("Stream closed");
1503 }
1504
1505 final long remaining = getFileLength() - pos;
1506 return remaining <= Integer.MAX_VALUE? (int)remaining: Integer.MAX_VALUE;
1507 }
1508
1509 /**
1510 * We definitely don't support marks
1511 */
1512 @Override
1513 public boolean markSupported() {
1514 return false;
1515 }
1516 @Override
1517 public void mark(int readLimit) {
1518 }
1519 @Override
1520 public void reset() throws IOException {
1521 throw new IOException("Mark/reset not supported");
1522 }
1523
1524 /** Utility class to encapsulate data node info and its address. */
1525 private static final class DNAddrPair {
1526 final DatanodeInfo info;
1527 final InetSocketAddress addr;
1528 final StorageType storageType;
1529
1530 DNAddrPair(DatanodeInfo info, InetSocketAddress addr,
1531 StorageType storageType) {
1532 this.info = info;
1533 this.addr = addr;
1534 this.storageType = storageType;
1535 }
1536 }
1537
1538 /**
1539 * Get statistics about the reads which this DFSInputStream has done.
1540 */
1541 public synchronized ReadStatistics getReadStatistics() {
1542 return new ReadStatistics(readStatistics);
1543 }
1544
1545 public synchronized FileEncryptionInfo getFileEncryptionInfo() {
1546 return fileEncryptionInfo;
1547 }
1548
1549 private synchronized void closeCurrentBlockReader() {
1550 if (blockReader == null) return;
1551 // Close the current block reader so that the new caching settings can
1552 // take effect immediately.
1553 try {
1554 blockReader.close();
1555 } catch (IOException e) {
1556 DFSClient.LOG.error("error closing blockReader", e);
1557 }
1558 blockReader = null;
1559 }
1560
1561 @Override
1562 public synchronized void setReadahead(Long readahead)
1563 throws IOException {
1564 this.cachingStrategy =
1565 new CachingStrategy.Builder(this.cachingStrategy).
1566 setReadahead(readahead).build();
1567 closeCurrentBlockReader();
1568 }
1569
1570 @Override
1571 public synchronized void setDropBehind(Boolean dropBehind)
1572 throws IOException {
1573 this.cachingStrategy =
1574 new CachingStrategy.Builder(this.cachingStrategy).
1575 setDropBehind(dropBehind).build();
1576 closeCurrentBlockReader();
1577 }
1578
1579 /**
1580 * The immutable empty buffer we return when we reach EOF when doing a
1581 * zero-copy read.
1582 */
1583 private static final ByteBuffer EMPTY_BUFFER =
1584 ByteBuffer.allocateDirect(0).asReadOnlyBuffer();
1585
1586 @Override
1587 public synchronized ByteBuffer read(ByteBufferPool bufferPool,
1588 int maxLength, EnumSet<ReadOption> opts)
1589 throws IOException, UnsupportedOperationException {
1590 if (maxLength == 0) {
1591 return EMPTY_BUFFER;
1592 } else if (maxLength < 0) {
1593 throw new IllegalArgumentException("can't read a negative " +
1594 "number of bytes.");
1595 }
1596 if ((blockReader == null) || (blockEnd == -1)) {
1597 if (pos >= getFileLength()) {
1598 return null;
1599 }
1600 /*
1601 * If we don't have a blockReader, or the one we have has no more bytes
1602 * left to read, we call seekToBlockSource to get a new blockReader and
1603 * recalculate blockEnd. Note that we assume we're not at EOF here
1604 * (we check this above).
1605 */
1606 if ((!seekToBlockSource(pos)) || (blockReader == null)) {
1607 throw new IOException("failed to allocate new BlockReader " +
1608 "at position " + pos);
1609 }
1610 }
1611 ByteBuffer buffer = null;
1612 if (dfsClient.getConf().shortCircuitMmapEnabled) {
1613 buffer = tryReadZeroCopy(maxLength, opts);
1614 }
1615 if (buffer != null) {
1616 return buffer;
1617 }
1618 buffer = ByteBufferUtil.fallbackRead(this, bufferPool, maxLength);
1619 if (buffer != null) {
1620 extendedReadBuffers.put(buffer, bufferPool);
1621 }
1622 return buffer;
1623 }
1624
1625 private synchronized ByteBuffer tryReadZeroCopy(int maxLength,
1626 EnumSet<ReadOption> opts) throws IOException {
1627 // Copy 'pos' and 'blockEnd' to local variables to make it easier for the
1628 // JVM to optimize this function.
1629 final long curPos = pos;
1630 final long curEnd = blockEnd;
1631 final long blockStartInFile = currentLocatedBlock.getStartOffset();
1632 final long blockPos = curPos - blockStartInFile;
1633
1634 // Shorten this read if the end of the block is nearby.
1635 long length63;
1636 if ((curPos + maxLength) <= (curEnd + 1)) {
1637 length63 = maxLength;
1638 } else {
1639 length63 = 1 + curEnd - curPos;
1640 if (length63 <= 0) {
1641 if (DFSClient.LOG.isDebugEnabled()) {
1642 DFSClient.LOG.debug("Unable to perform a zero-copy read from offset " +
1643 curPos + " of " + src + "; " + length63 + " bytes left in block. " +
1644 "blockPos=" + blockPos + "; curPos=" + curPos +
1645 "; curEnd=" + curEnd);
1646 }
1647 return null;
1648 }
1649 if (DFSClient.LOG.isDebugEnabled()) {
1650 DFSClient.LOG.debug("Reducing read length from " + maxLength +
1651 " to " + length63 + " to avoid going more than one byte " +
1652 "past the end of the block. blockPos=" + blockPos +
1653 "; curPos=" + curPos + "; curEnd=" + curEnd);
1654 }
1655 }
1656 // Make sure that don't go beyond 31-bit offsets in the MappedByteBuffer.
1657 int length;
1658 if (blockPos + length63 <= Integer.MAX_VALUE) {
1659 length = (int)length63;
1660 } else {
1661 long length31 = Integer.MAX_VALUE - blockPos;
1662 if (length31 <= 0) {
1663 // Java ByteBuffers can't be longer than 2 GB, because they use
1664 // 4-byte signed integers to represent capacity, etc.
1665 // So we can't mmap the parts of the block higher than the 2 GB offset.
1666 // FIXME: we could work around this with multiple memory maps.
1667 // See HDFS-5101.
1668 if (DFSClient.LOG.isDebugEnabled()) {
1669 DFSClient.LOG.debug("Unable to perform a zero-copy read from offset " +
1670 curPos + " of " + src + "; 31-bit MappedByteBuffer limit " +
1671 "exceeded. blockPos=" + blockPos + ", curEnd=" + curEnd);
1672 }
1673 return null;
1674 }
1675 length = (int)length31;
1676 if (DFSClient.LOG.isDebugEnabled()) {
1677 DFSClient.LOG.debug("Reducing read length from " + maxLength +
1678 " to " + length + " to avoid 31-bit limit. " +
1679 "blockPos=" + blockPos + "; curPos=" + curPos +
1680 "; curEnd=" + curEnd);
1681 }
1682 }
1683 final ClientMmap clientMmap = blockReader.getClientMmap(opts);
1684 if (clientMmap == null) {
1685 if (DFSClient.LOG.isDebugEnabled()) {
1686 DFSClient.LOG.debug("unable to perform a zero-copy read from offset " +
1687 curPos + " of " + src + "; BlockReader#getClientMmap returned " +
1688 "null.");
1689 }
1690 return null;
1691 }
1692 boolean success = false;
1693 ByteBuffer buffer;
1694 try {
1695 seek(curPos + length);
1696 buffer = clientMmap.getMappedByteBuffer().asReadOnlyBuffer();
1697 buffer.position((int)blockPos);
1698 buffer.limit((int)(blockPos + length));
1699 extendedReadBuffers.put(buffer, clientMmap);
1700 readStatistics.addZeroCopyBytes(length);
1701 if (DFSClient.LOG.isDebugEnabled()) {
1702 DFSClient.LOG.debug("readZeroCopy read " + length +
1703 " bytes from offset " + curPos + " via the zero-copy read " +
1704 "path. blockEnd = " + blockEnd);
1705 }
1706 success = true;
1707 } finally {
1708 if (!success) {
1709 IOUtils.closeQuietly(clientMmap);
1710 }
1711 }
1712 return buffer;
1713 }
1714
1715 @Override
1716 public synchronized void releaseBuffer(ByteBuffer buffer) {
1717 if (buffer == EMPTY_BUFFER) return;
1718 Object val = extendedReadBuffers.remove(buffer);
1719 if (val == null) {
1720 throw new IllegalArgumentException("tried to release a buffer " +
1721 "that was not created by this stream, " + buffer);
1722 }
1723 if (val instanceof ClientMmap) {
1724 IOUtils.closeQuietly((ClientMmap)val);
1725 } else if (val instanceof ByteBufferPool) {
1726 ((ByteBufferPool)val).putBuffer(buffer);
1727 }
1728 }
1729 }