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