Uploaded image for project: 'Hadoop HDFS'
  1. Hadoop HDFS
  2. HDFS-9146

HDFS forward seek() within a block shouldn't spawn new TCP Peer/RemoteBlockReader

Add voteVotersWatch issueWatchersCreate sub-taskLinkCloneUpdate Comment AuthorReplace String in CommentUpdate Comment VisibilityDelete Comments
    XMLWordPrintableJSON

Details

    • Bug
    • Status: Open
    • Major
    • Resolution: Unresolved
    • 2.6.0, 2.8.0, 2.7.1, 3.0.0
    • None
    • hdfs-client
    • None

    Description

      When a seek() + forward readFully() is triggered from a remote dfsclient, HDFS opens a new remote block reader even if the seek is within the same HDFS block.

      (analysis from Rajesh Balamohan)

      This is due to the fact that a simple read operation assumes that the user is going to read till the end of the block.

            try {
              blockReader = getBlockReader(targetBlock, offsetIntoBlock,
                  targetBlock.getBlockSize() - offsetIntoBlock, targetAddr,
                  storageType, chosenNode);
      

      https://github.com/apache/hadoop/blob/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java#L624

      Since the user hasn't read till the end of the block when the next seek happens, the BlockReader assumes this is an aborted read and tries to throw away the TCP peer it has got.

      https://github.com/apache/hadoop/blob/trunk/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java#L324

          // If we've now satisfied the whole client read, read one last packet
          // header, which should be empty
          if (bytesNeededToFinish <= 0) {
            readTrailingEmptyPacket(); 
           ...
                sendReadResult(Status.SUCCESS);
      

      Since that is not satisfied, the status code is unset & the peer is not returned to the cache.

          if (peerCache != null && sentStatusCode) {
            peerCache.put(datanodeID, peer);
          } else {
            peer.close();
          }
      

      Attachments

        Issue Links

        Activity

          This comment will be Viewable by All Users Viewable by All Users
          Cancel

          People

            Unassigned Unassigned
            gopalv Gopal Vijayaraghavan

            Dates

              Created:
              Updated:

              Slack

                Issue deployment