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