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