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

Improve InputStream when read spans two blocks

    XMLWordPrintableJSON

Details

    • Improvement
    • Status: Patch Available
    • Major
    • Resolution: Unresolved
    • 2.4.0
    • None
    • hdfs-client
    • DFSInputStream

    Description

      In the current implementation of DFSInputStream, read(buffer, offset, length) is implemented as following:

      int realLen = (int) Math.min(len, (blockEnd - pos + 1L));
      if (locatedBlocks.isLastBlockComplete()) {
        realLen = (int) Math.min(realLen, locatedBlocks.getFileLength());
      }
      int result = readBuffer(strategy, off, realLen, corruptedBlockMap);
      

      From the above code, we can conclude that the read will return at most (blockEnd - pos + 1) bytes. As a result, when read spans two blocks, the caller must call read() second time to complete the request, and must wait second time to acquire the DFSInputStream lock(read() is synchronized for DFSInputStream). For latency sensitive applications, such as hbase, this will result in latency pain point when they under massive race conditions. So here we propose that we should loop internally in read() to do best effort read.

      In the current implementation of pread(read(position, buffer, offset, lenght)), it does loop internally to do best effort read. So we can refactor to support this on normal read.

      Attachments

        1. HDFS-6596.3.patch
          11 kB
          Zesheng Wu
        2. HDFS-6596.3.patch
          11 kB
          Zesheng Wu
        3. HDFS-6596.2.patch
          11 kB
          Zesheng Wu
        4. HDFS-6596.2.patch
          11 kB
          Zesheng Wu
        5. HDFS-6596.2.patch
          11 kB
          Zesheng Wu
        6. HDFS-6596.1.patch
          10 kB
          Zesheng Wu

        Activity

          People

            wuzesheng Zesheng Wu
            wuzesheng Zesheng Wu
            Votes:
            0 Vote for this issue
            Watchers:
            6 Start watching this issue

            Dates

              Created:
              Updated: