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 try {
987 byte[] buf = bb.array();
988 int offset = bb.position();
989 actualGetFromOneDataNode(datanode, block, start, end, buf, offset,
990 corruptedBlockMap);
991 return bb;
992 } finally {
993 latch.countDown();
994 }
995 }
996 };
997 }
998
999 private void actualGetFromOneDataNode(final DNAddrPair datanode,
1000 LocatedBlock block, final long start, final long end, byte[] buf,
1001 int offset, Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
1002 throws IOException {
1003 DFSClientFaultInjector.get().startFetchFromDatanode();
1004 int refetchToken = 1; // only need to get a new access token once
1005 int refetchEncryptionKey = 1; // only need to get a new encryption key once
1006
1007 while (true) {
1008 // cached block locations may have been updated by chooseDataNode()
1009 // or fetchBlockAt(). Always get the latest list of locations at the
1010 // start of the loop.
1011 CachingStrategy curCachingStrategy;
1012 boolean allowShortCircuitLocalReads;
1013 synchronized (this) {
1014 block = getBlockAt(block.getStartOffset(), false);
1015 curCachingStrategy = cachingStrategy;
1016 allowShortCircuitLocalReads = !shortCircuitForbidden();
1017 }
1018 DatanodeInfo chosenNode = datanode.info;
1019 InetSocketAddress targetAddr = datanode.addr;
1020 BlockReader reader = null;
1021
1022 try {
1023 Token<BlockTokenIdentifier> blockToken = block.getBlockToken();
1024 int len = (int) (end - start + 1);
1025 reader = new BlockReaderFactory(dfsClient.getConf()).
1026 setInetSocketAddress(targetAddr).
1027 setRemotePeerFactory(dfsClient).
1028 setDatanodeInfo(chosenNode).
1029 setFileName(src).
1030 setBlock(block.getBlock()).
1031 setBlockToken(blockToken).
1032 setStartOffset(start).
1033 setVerifyChecksum(verifyChecksum).
1034 setClientName(dfsClient.clientName).
1035 setLength(len).
1036 setCachingStrategy(curCachingStrategy).
1037 setAllowShortCircuitLocalReads(allowShortCircuitLocalReads).
1038 setClientCacheContext(dfsClient.getClientContext()).
1039 setUserGroupInformation(dfsClient.ugi).
1040 setConfiguration(dfsClient.getConfiguration()).
1041 build();
1042 int nread = reader.readAll(buf, offset, len);
1043 if (nread != len) {
1044 throw new IOException("truncated return from reader.read(): " +
1045 "excpected " + len + ", got " + nread);
1046 }
1047 return;
1048 } catch (ChecksumException e) {
1049 String msg = "fetchBlockByteRange(). Got a checksum exception for "
1050 + src + " at " + block.getBlock() + ":" + e.getPos() + " from "
1051 + chosenNode;
1052 DFSClient.LOG.warn(msg);
1053 // we want to remember what we have tried
1054 addIntoCorruptedBlockMap(block.getBlock(), chosenNode, corruptedBlockMap);
1055 addToDeadNodes(chosenNode);
1056 throw new IOException(msg);
1057 } catch (IOException e) {
1058 if (e instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
1059 DFSClient.LOG.info("Will fetch a new encryption key and retry, "
1060 + "encryption key was invalid when connecting to " + targetAddr
1061 + " : " + e);
1062 // The encryption key used is invalid.
1063 refetchEncryptionKey--;
1064 dfsClient.clearDataEncryptionKey();
1065 continue;
1066 } else if (refetchToken > 0 && tokenRefetchNeeded(e, targetAddr)) {
1067 refetchToken--;
1068 try {
1069 fetchBlockAt(block.getStartOffset());
1070 } catch (IOException fbae) {
1071 // ignore IOE, since we can retry it later in a loop
1072 }
1073 continue;
1074 } else {
1075 String msg = "Failed to connect to " + targetAddr + " for file "
1076 + src + " for block " + block.getBlock() + ":" + e;
1077 DFSClient.LOG.warn("Connection failure: " + msg, e);
1078 addToDeadNodes(chosenNode);
1079 throw new IOException(msg);
1080 }
1081 } finally {
1082 if (reader != null) {
1083 reader.close();
1084 }
1085 }
1086 }
1087 }
1088
1089 /**
1090 * Like {@link #fetchBlockByteRange(LocatedBlock, long, long, byte[],
1091 * int, Map)} except we start up a second, parallel, 'hedged' read
1092 * if the first read is taking longer than configured amount of
1093 * time. We then wait on which ever read returns first.
1094 *
1095 * @param block
1096 * @param start
1097 * @param end
1098 * @param buf
1099 * @param offset
1100 * @param corruptedBlockMap
1101 * @throws IOException
1102 */
1103 private void hedgedFetchBlockByteRange(LocatedBlock block, long start,
1104 long end, byte[] buf, int offset,
1105 Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
1106 throws IOException {
1107 ArrayList<Future<ByteBuffer>> futures = new ArrayList<Future<ByteBuffer>>();
1108 ArrayList<DatanodeInfo> ignored = new ArrayList<DatanodeInfo>();
1109 ByteBuffer bb = null;
1110 int len = (int) (end - start + 1);
1111 block = getBlockAt(block.getStartOffset(), false);
1112 // Latch shared by all outstanding reads. First to finish closes
1113 CountDownLatch hasReceivedResult = new CountDownLatch(1);
1114 while (true) {
1115 DNAddrPair chosenNode = null;
1116 Future<ByteBuffer> future = null;
1117 // futures is null if there is no request already executing.
1118 if (futures.isEmpty()) {
1119 // chooseDataNode is a commitment. If no node, we go to
1120 // the NN to reget block locations. Only go here on first read.
1121 chosenNode = chooseDataNode(block, ignored);
1122 bb = ByteBuffer.wrap(buf, offset, len);
1123 future = getHedgedReadFuture(chosenNode, block, start, end, bb,
1124 corruptedBlockMap, hasReceivedResult);
1125 try {
1126 future.get(dfsClient.getHedgedReadTimeout(), TimeUnit.MILLISECONDS);
1127 return;
1128 } catch (TimeoutException e) {
1129 if (DFSClient.LOG.isDebugEnabled()) {
1130 DFSClient.LOG.debug("Waited " + dfsClient.getHedgedReadTimeout() +
1131 "ms to read from " + chosenNode.info + "; spawning hedged read");
1132 }
1133 // Ignore this node on next go around.
1134 ignored.add(chosenNode.info);
1135 dfsClient.getHedgedReadMetrics().incHedgedReadOps();
1136 futures.add(future);
1137 continue; // no need to refresh block locations
1138 } catch (InterruptedException e) {
1139 // Ignore
1140 } catch (ExecutionException e) {
1141 // Ignore already logged in the call.
1142 }
1143 } else {
1144 // We are starting up a 'hedged' read. We have a read already
1145 // ongoing. Call getBestNodeDNAddrPair instead of chooseDataNode.
1146 // If no nodes to do hedged reads against, pass.
1147 try {
1148 chosenNode = getBestNodeDNAddrPair(block.getLocations(), ignored);
1149 bb = ByteBuffer.allocate(len);
1150 future = getHedgedReadFuture(chosenNode, block, start, end, bb,
1151 corruptedBlockMap, hasReceivedResult);
1152 futures.add(future);
1153 } catch (IOException ioe) {
1154 if (DFSClient.LOG.isDebugEnabled()) {
1155 DFSClient.LOG.debug("Failed getting node for hedged read: " +
1156 ioe.getMessage());
1157 }
1158 }
1159 // if not succeeded. Submit callables for each datanode in a loop, wait
1160 // for a fixed interval and get the result from the fastest one.
1161 try {
1162 ByteBuffer result = getFirstToComplete(futures, hasReceivedResult);
1163 // cancel the rest.
1164 cancelAll(futures);
1165 if (result.array() != buf) { // compare the array pointers
1166 dfsClient.getHedgedReadMetrics().incHedgedReadWins();
1167 System.arraycopy(result.array(), result.position(), buf, offset,
1168 len);
1169 } else {
1170 dfsClient.getHedgedReadMetrics().incHedgedReadOps();
1171 }
1172 return;
1173 } catch (InterruptedException ie) {
1174 // Ignore
1175 } catch (ExecutionException e) {
1176 // exception already handled in the call method. getFirstToComplete
1177 // will remove the failing future from the list. nothing more to do.
1178 }
1179 // We got here if exception. Ignore this node on next go around IFF
1180 // we found a chosenNode to hedge read against.
1181 if (chosenNode != null && chosenNode.info != null) {
1182 ignored.add(chosenNode.info);
1183 }
1184 }
1185 // executed if we get an error from a data node
1186 block = getBlockAt(block.getStartOffset(), false);
1187 }
1188 }
1189
1190 private Future<ByteBuffer> getHedgedReadFuture(final DNAddrPair chosenNode,
1191 final LocatedBlock block, long start,
1192 final long end, final ByteBuffer bb,
1193 final Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap,
1194 final CountDownLatch hasReceivedResult) {
1195 Callable<ByteBuffer> getFromDataNodeCallable =
1196 getFromOneDataNode(chosenNode, block, start, end, bb,
1197 corruptedBlockMap, hasReceivedResult);
1198 return dfsClient.getHedgedReadsThreadPool().submit(getFromDataNodeCallable);
1199 }
1200
1201 private ByteBuffer getFirstToComplete(ArrayList<Future<ByteBuffer>> futures,
1202 CountDownLatch latch) throws ExecutionException, InterruptedException {
1203 latch.await();
1204 for (Future<ByteBuffer> future : futures) {
1205 if (future.isDone()) {
1206 try {
1207 return future.get();
1208 } catch (ExecutionException e) {
1209 // already logged in the Callable
1210 futures.remove(future);
1211 throw e;
1212 }
1213 }
1214 }
1215 throw new InterruptedException("latch has counted down to zero but no"
1216 + "result available yet, for safety try to request another one from"
1217 + "outside loop, this should be rare");
1218 }
1219
1220 private void cancelAll(List<Future<ByteBuffer>> futures) {
1221 for (Future<ByteBuffer> future : futures) {
1222 // Unfortunately, hdfs reads do not take kindly to interruption.
1223 // Threads return a variety of interrupted-type exceptions but
1224 // also complaints about invalid pbs -- likely because read
1225 // is interrupted before gets whole pb. Also verbose WARN
1226 // logging. So, for now, do not interrupt running read.
1227 future.cancel(false);
1228 }
1229 }
1230
1231 /**
1232 * Should the block access token be refetched on an exception
1233 *
1234 * @param ex Exception received
1235 * @param targetAddr Target datanode address from where exception was received
1236 * @return true if block access token has expired or invalid and it should be
1237 * refetched
1238 */
1239 private static boolean tokenRefetchNeeded(IOException ex,
1240 InetSocketAddress targetAddr) {
1241 /*
1242 * Get a new access token and retry. Retry is needed in 2 cases. 1)
1243 * When both NN and DN re-started while DFSClient holding a cached
1244 * access token. 2) In the case that NN fails to update its
1245 * access key at pre-set interval (by a wide margin) and
1246 * subsequently restarts. In this case, DN re-registers itself with
1247 * NN and receives a new access key, but DN will delete the old
1248 * access key from its memory since it's considered expired based on
1249 * the estimated expiration date.
1250 */
1251 if (ex instanceof InvalidBlockTokenException || ex instanceof InvalidToken) {
1252 DFSClient.LOG.info("Access token was invalid when connecting to "
1253 + targetAddr + " : " + ex);
1254 return true;
1255 }
1256 return false;
1257 }
1258
1259 /**
1260 * Read bytes starting from the specified position.
1261 *
1262 * @param position start read from this position
1263 * @param buffer read buffer
1264 * @param offset offset into buffer
1265 * @param length number of bytes to read
1266 *
1267 * @return actual number of bytes read
1268 */
1269 @Override
1270 public int read(long position, byte[] buffer, int offset, int length)
1271 throws IOException {
1272 // sanity checks
1273 dfsClient.checkOpen();
1274 if (closed) {
1275 throw new IOException("Stream closed");
1276 }
1277 failures = 0;
1278 long filelen = getFileLength();
1279 if ((position < 0) || (position >= filelen)) {
1280 return -1;
1281 }
1282 int realLen = length;
1283 if ((position + length) > filelen) {
1284 realLen = (int)(filelen - position);
1285 }
1286
1287 // determine the block and byte range within the block
1288 // corresponding to position and realLen
1289 List<LocatedBlock> blockRange = getBlockRange(position, realLen);
1290 int remaining = realLen;
1291 Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap
1292 = new HashMap<ExtendedBlock, Set<DatanodeInfo>>();
1293 for (LocatedBlock blk : blockRange) {
1294 long targetStart = position - blk.getStartOffset();
1295 long bytesToRead = Math.min(remaining, blk.getBlockSize() - targetStart);
1296 try {
1297 if (dfsClient.isHedgedReadsEnabled()) {
1298 hedgedFetchBlockByteRange(blk, targetStart, targetStart + bytesToRead
1299 - 1, buffer, offset, corruptedBlockMap);
1300 } else {
1301 fetchBlockByteRange(blk, targetStart, targetStart + bytesToRead - 1,
1302 buffer, offset, corruptedBlockMap);
1303 }
1304 } finally {
1305 // Check and report if any block replicas are corrupted.
1306 // BlockMissingException may be caught if all block replicas are
1307 // corrupted.
1308 reportCheckSumFailure(corruptedBlockMap, blk.getLocations().length);
1309 }
1310
1311 remaining -= bytesToRead;
1312 position += bytesToRead;
1313 offset += bytesToRead;
1314 }
1315 assert remaining == 0 : "Wrong number of bytes read.";
1316 if (dfsClient.stats != null) {
1317 dfsClient.stats.incrementBytesRead(realLen);
1318 }
1319 return realLen;
1320 }
1321
1322 /**
1323 * DFSInputStream reports checksum failure.
1324 * Case I : client has tried multiple data nodes and at least one of the
1325 * attempts has succeeded. We report the other failures as corrupted block to
1326 * namenode.
1327 * Case II: client has tried out all data nodes, but all failed. We
1328 * only report if the total number of replica is 1. We do not
1329 * report otherwise since this maybe due to the client is a handicapped client
1330 * (who can not read).
1331 * @param corruptedBlockMap map of corrupted blocks
1332 * @param dataNodeCount number of data nodes who contains the block replicas
1333 */
1334 private void reportCheckSumFailure(
1335 Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap,
1336 int dataNodeCount) {
1337 if (corruptedBlockMap.isEmpty()) {
1338 return;
1339 }
1340 Iterator<Entry<ExtendedBlock, Set<DatanodeInfo>>> it = corruptedBlockMap
1341 .entrySet().iterator();
1342 Entry<ExtendedBlock, Set<DatanodeInfo>> entry = it.next();
1343 ExtendedBlock blk = entry.getKey();
1344 Set<DatanodeInfo> dnSet = entry.getValue();
1345 if (((dnSet.size() < dataNodeCount) && (dnSet.size() > 0))
1346 || ((dataNodeCount == 1) && (dnSet.size() == dataNodeCount))) {
1347 DatanodeInfo[] locs = new DatanodeInfo[dnSet.size()];
1348 int i = 0;
1349 for (DatanodeInfo dn:dnSet) {
1350 locs[i++] = dn;
1351 }
1352 LocatedBlock [] lblocks = { new LocatedBlock(blk, locs) };
1353 dfsClient.reportChecksumFailure(src, lblocks);
1354 }
1355 corruptedBlockMap.clear();
1356 }
1357
1358 @Override
1359 public long skip(long n) throws IOException {
1360 if ( n > 0 ) {
1361 long curPos = getPos();
1362 long fileLen = getFileLength();
1363 if( n+curPos > fileLen ) {
1364 n = fileLen - curPos;
1365 }
1366 seek(curPos+n);
1367 return n;
1368 }
1369 return n < 0 ? -1 : 0;
1370 }
1371
1372 /**
1373 * Seek to a new arbitrary location
1374 */
1375 @Override
1376 public synchronized void seek(long targetPos) throws IOException {
1377 if (targetPos > getFileLength()) {
1378 throw new IOException("Cannot seek after EOF");
1379 }
1380 if (targetPos < 0) {
1381 throw new IOException("Cannot seek to negative offset");
1382 }
1383 if (closed) {
1384 throw new IOException("Stream is closed!");
1385 }
1386 boolean done = false;
1387 if (pos <= targetPos && targetPos <= blockEnd) {
1388 //
1389 // If this seek is to a positive position in the current
1390 // block, and this piece of data might already be lying in
1391 // the TCP buffer, then just eat up the intervening data.
1392 //
1393 int diff = (int)(targetPos - pos);
1394 if (diff <= blockReader.available()) {
1395 try {
1396 pos += blockReader.skip(diff);
1397 if (pos == targetPos) {
1398 done = true;
1399 } else {
1400 // The range was already checked. If the block reader returns
1401 // something unexpected instead of throwing an exception, it is
1402 // most likely a bug.
1403 String errMsg = "BlockReader failed to seek to " +
1404 targetPos + ". Instead, it seeked to " + pos + ".";
1405 DFSClient.LOG.warn(errMsg);
1406 throw new IOException(errMsg);
1407 }
1408 } catch (IOException e) {//make following read to retry
1409 if(DFSClient.LOG.isDebugEnabled()) {
1410 DFSClient.LOG.debug("Exception while seek to " + targetPos
1411 + " from " + getCurrentBlock() + " of " + src + " from "
1412 + currentNode, e);
1413 }
1414 }
1415 }
1416 }
1417 if (!done) {
1418 pos = targetPos;
1419 blockEnd = -1;
1420 }
1421 }
1422
1423 /**
1424 * Same as {@link #seekToNewSource(long)} except that it does not exclude
1425 * the current datanode and might connect to the same node.
1426 */
1427 private synchronized boolean seekToBlockSource(long targetPos)
1428 throws IOException {
1429 currentNode = blockSeekTo(targetPos);
1430 return true;
1431 }
1432
1433 /**
1434 * Seek to given position on a node other than the current node. If
1435 * a node other than the current node is found, then returns true.
1436 * If another node could not be found, then returns false.
1437 */
1438 @Override
1439 public synchronized boolean seekToNewSource(long targetPos) throws IOException {
1440 boolean markedDead = deadNodes.containsKey(currentNode);
1441 addToDeadNodes(currentNode);
1442 DatanodeInfo oldNode = currentNode;
1443 DatanodeInfo newNode = blockSeekTo(targetPos);
1444 if (!markedDead) {
1445 /* remove it from deadNodes. blockSeekTo could have cleared
1446 * deadNodes and added currentNode again. Thats ok. */
1447 deadNodes.remove(oldNode);
1448 }
1449 if (!oldNode.getDatanodeUuid().equals(newNode.getDatanodeUuid())) {
1450 currentNode = newNode;
1451 return true;
1452 } else {
1453 return false;
1454 }
1455 }
1456
1457 /**
1458 */
1459 @Override
1460 public synchronized long getPos() throws IOException {
1461 return pos;
1462 }
1463
1464 /** Return the size of the remaining available bytes
1465 * if the size is less than or equal to {@link Integer#MAX_VALUE},
1466 * otherwise, return {@link Integer#MAX_VALUE}.
1467 */
1468 @Override
1469 public synchronized int available() throws IOException {
1470 if (closed) {
1471 throw new IOException("Stream closed");
1472 }
1473
1474 final long remaining = getFileLength() - pos;
1475 return remaining <= Integer.MAX_VALUE? (int)remaining: Integer.MAX_VALUE;
1476 }
1477
1478 /**
1479 * We definitely don't support marks
1480 */
1481 @Override
1482 public boolean markSupported() {
1483 return false;
1484 }
1485 @Override
1486 public void mark(int readLimit) {
1487 }
1488 @Override
1489 public void reset() throws IOException {
1490 throw new IOException("Mark/reset not supported");
1491 }
1492
1493 /**
1494 * Pick the best node from which to stream the data.
1495 * Entries in <i>nodes</i> are already in the priority order
1496 */
1497 static DatanodeInfo bestNode(DatanodeInfo nodes[],
1498 AbstractMap<DatanodeInfo, DatanodeInfo> deadNodes,
1499 Collection<DatanodeInfo> ignoredNodes) throws IOException {
1500 if (nodes != null) {
1501 for (int i = 0; i < nodes.length; i++) {
1502 if (!deadNodes.containsKey(nodes[i])
1503 && (ignoredNodes == null || !ignoredNodes.contains(nodes[i]))) {
1504 return nodes[i];
1505 }
1506 }
1507 }
1508 throw new IOException("No live nodes contain current block");
1509 }
1510
1511 /** Utility class to encapsulate data node info and its address. */
1512 static class DNAddrPair {
1513 final DatanodeInfo info;
1514 final InetSocketAddress addr;
1515 DNAddrPair(DatanodeInfo info, InetSocketAddress addr) {
1516 this.info = info;
1517 this.addr = addr;
1518 }
1519 }
1520
1521 /**
1522 * Get statistics about the reads which this DFSInputStream has done.
1523 */
1524 public synchronized ReadStatistics getReadStatistics() {
1525 return new ReadStatistics(readStatistics);
1526 }
1527
1528 private synchronized void closeCurrentBlockReader() {
1529 if (blockReader == null) return;
1530 // Close the current block reader so that the new caching settings can
1531 // take effect immediately.
1532 try {
1533 blockReader.close();
1534 } catch (IOException e) {
1535 DFSClient.LOG.error("error closing blockReader", e);
1536 }
1537 blockReader = null;
1538 }
1539
1540 @Override
1541 public synchronized void setReadahead(Long readahead)
1542 throws IOException {
1543 this.cachingStrategy =
1544 new CachingStrategy.Builder(this.cachingStrategy).
1545 setReadahead(readahead).build();
1546 closeCurrentBlockReader();
1547 }
1548
1549 @Override
1550 public synchronized void setDropBehind(Boolean dropBehind)
1551 throws IOException {
1552 this.cachingStrategy =
1553 new CachingStrategy.Builder(this.cachingStrategy).
1554 setDropBehind(dropBehind).build();
1555 closeCurrentBlockReader();
1556 }
1557
1558 /**
1559 * The immutable empty buffer we return when we reach EOF when doing a
1560 * zero-copy read.
1561 */
1562 private static final ByteBuffer EMPTY_BUFFER =
1563 ByteBuffer.allocateDirect(0).asReadOnlyBuffer();
1564
1565 @Override
1566 public synchronized ByteBuffer read(ByteBufferPool bufferPool,
1567 int maxLength, EnumSet<ReadOption> opts)
1568 throws IOException, UnsupportedOperationException {
1569 if (maxLength == 0) {
1570 return EMPTY_BUFFER;
1571 } else if (maxLength < 0) {
1572 throw new IllegalArgumentException("can't read a negative " +
1573 "number of bytes.");
1574 }
1575 if ((blockReader == null) || (blockEnd == -1)) {
1576 if (pos >= getFileLength()) {
1577 return null;
1578 }
1579 /*
1580 * If we don't have a blockReader, or the one we have has no more bytes
1581 * left to read, we call seekToBlockSource to get a new blockReader and
1582 * recalculate blockEnd. Note that we assume we're not at EOF here
1583 * (we check this above).
1584 */
1585 if ((!seekToBlockSource(pos)) || (blockReader == null)) {
1586 throw new IOException("failed to allocate new BlockReader " +
1587 "at position " + pos);
1588 }
1589 }
1590 ByteBuffer buffer = null;
1591 if (dfsClient.getConf().shortCircuitMmapEnabled) {
1592 buffer = tryReadZeroCopy(maxLength, opts);
1593 }
1594 if (buffer != null) {
1595 return buffer;
1596 }
1597 buffer = ByteBufferUtil.fallbackRead(this, bufferPool, maxLength);
1598 if (buffer != null) {
1599 extendedReadBuffers.put(buffer, bufferPool);
1600 }
1601 return buffer;
1602 }
1603
1604 private synchronized ByteBuffer tryReadZeroCopy(int maxLength,
1605 EnumSet<ReadOption> opts) throws IOException {
1606 // Copy 'pos' and 'blockEnd' to local variables to make it easier for the
1607 // JVM to optimize this function.
1608 final long curPos = pos;
1609 final long curEnd = blockEnd;
1610 final long blockStartInFile = currentLocatedBlock.getStartOffset();
1611 final long blockPos = curPos - blockStartInFile;
1612
1613 // Shorten this read if the end of the block is nearby.
1614 long length63;
1615 if ((curPos + maxLength) <= (curEnd + 1)) {
1616 length63 = maxLength;
1617 } else {
1618 length63 = 1 + curEnd - curPos;
1619 if (length63 <= 0) {
1620 if (DFSClient.LOG.isDebugEnabled()) {
1621 DFSClient.LOG.debug("Unable to perform a zero-copy read from offset " +
1622 curPos + " of " + src + "; " + length63 + " bytes left in block. " +
1623 "blockPos=" + blockPos + "; curPos=" + curPos +
1624 "; curEnd=" + curEnd);
1625 }
1626 return null;
1627 }
1628 if (DFSClient.LOG.isDebugEnabled()) {
1629 DFSClient.LOG.debug("Reducing read length from " + maxLength +
1630 " to " + length63 + " to avoid going more than one byte " +
1631 "past the end of the block. blockPos=" + blockPos +
1632 "; curPos=" + curPos + "; curEnd=" + curEnd);
1633 }
1634 }
1635 // Make sure that don't go beyond 31-bit offsets in the MappedByteBuffer.
1636 int length;
1637 if (blockPos + length63 <= Integer.MAX_VALUE) {
1638 length = (int)length63;
1639 } else {
1640 long length31 = Integer.MAX_VALUE - blockPos;
1641 if (length31 <= 0) {
1642 // Java ByteBuffers can't be longer than 2 GB, because they use
1643 // 4-byte signed integers to represent capacity, etc.
1644 // So we can't mmap the parts of the block higher than the 2 GB offset.
1645 // FIXME: we could work around this with multiple memory maps.
1646 // See HDFS-5101.
1647 if (DFSClient.LOG.isDebugEnabled()) {
1648 DFSClient.LOG.debug("Unable to perform a zero-copy read from offset " +
1649 curPos + " of " + src + "; 31-bit MappedByteBuffer limit " +
1650 "exceeded. blockPos=" + blockPos + ", curEnd=" + curEnd);
1651 }
1652 return null;
1653 }
1654 length = (int)length31;
1655 if (DFSClient.LOG.isDebugEnabled()) {
1656 DFSClient.LOG.debug("Reducing read length from " + maxLength +
1657 " to " + length + " to avoid 31-bit limit. " +
1658 "blockPos=" + blockPos + "; curPos=" + curPos +
1659 "; curEnd=" + curEnd);
1660 }
1661 }
1662 final ClientMmap clientMmap = blockReader.getClientMmap(opts);
1663 if (clientMmap == null) {
1664 if (DFSClient.LOG.isDebugEnabled()) {
1665 DFSClient.LOG.debug("unable to perform a zero-copy read from offset " +
1666 curPos + " of " + src + "; BlockReader#getClientMmap returned " +
1667 "null.");
1668 }
1669 return null;
1670 }
1671 boolean success = false;
1672 ByteBuffer buffer;
1673 try {
1674 seek(curPos + length);
1675 buffer = clientMmap.getMappedByteBuffer().asReadOnlyBuffer();
1676 buffer.position((int)blockPos);
1677 buffer.limit((int)(blockPos + length));
1678 extendedReadBuffers.put(buffer, clientMmap);
1679 readStatistics.addZeroCopyBytes(length);
1680 if (DFSClient.LOG.isDebugEnabled()) {
1681 DFSClient.LOG.debug("readZeroCopy read " + length +
1682 " bytes from offset " + curPos + " via the zero-copy read " +
1683 "path. blockEnd = " + blockEnd);
1684 }
1685 success = true;
1686 } finally {
1687 if (!success) {
1688 IOUtils.closeQuietly(clientMmap);
1689 }
1690 }
1691 return buffer;
1692 }
1693
1694 @Override
1695 public synchronized void releaseBuffer(ByteBuffer buffer) {
1696 if (buffer == EMPTY_BUFFER) return;
1697 Object val = extendedReadBuffers.remove(buffer);
1698 if (val == null) {
1699 throw new IllegalArgumentException("tried to release a buffer " +
1700 "that was not created by this stream, " + buffer);
1701 }
1702 if (val instanceof ClientMmap) {
1703 IOUtils.closeQuietly((ClientMmap)val);
1704 } else if (val instanceof ByteBufferPool) {
1705 ((ByteBufferPool)val).putBuffer(buffer);
1706 }
1707 }
1708 }