hbase-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Vladimir Rodionov (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HBASE-7336) HFileBlock.readAtOffset does not work well with multiple threads
Date Wed, 02 Jul 2014 18:37:25 GMT

    [ https://issues.apache.org/jira/browse/HBASE-7336?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14050538#comment-14050538
] 

Vladimir Rodionov commented on HBASE-7336:
------------------------------------------

I am looking into this stuff now, trying to figure out how to make parallel scan on a region
more efficient. The code in AbstractFSReader looks dangerous and does not provide any benefits
in terms of MT performance.
{code}
    protected int readAtOffset(FSDataInputStream istream,
        byte[] dest, int destOffset, int size,
        boolean peekIntoNextBlock, long fileOffset, boolean pread)
        throws IOException {
      if (peekIntoNextBlock &&
          destOffset + size + hdrSize > dest.length) {
        // We are asked to read the next block's header as well, but there is
        // not enough room in the array.
        throw new IOException("Attempted to read " + size + " bytes and " +
            hdrSize + " bytes of next header into a " + dest.length +
            "-byte array at offset " + destOffset);
      }

      if (!pread && streamLock.tryLock()) {
        // Seek + read. Better for scanning.
        try {
          istream.seek(fileOffset);

          long realOffset = istream.getPos();
          if (realOffset != fileOffset) {
            throw new IOException("Tried to seek to " + fileOffset + " to "
                + "read " + size + " bytes, but pos=" + realOffset
                + " after seek");
          }

          if (!peekIntoNextBlock) {
            IOUtils.readFully(istream, dest, destOffset, size);
            return -1;
          }

          // Try to read the next block header.
          if (!readWithExtra(istream, dest, destOffset, size, hdrSize))
            return -1;
        } finally {
          streamLock.unlock();
        }
      } else {
        // Positional read. Better for random reads; or when the streamLock is already locked.
        int extraSize = peekIntoNextBlock ? hdrSize : 0;

        int ret = istream.read(fileOffset, dest, destOffset, size + extraSize);
        if (ret < size) {
          throw new IOException("Positional read of " + size + " bytes " +
              "failed at offset " + fileOffset + " (returned " + ret + ")");
        }

        if (ret == size || ret < size + extraSize) {
          // Could not read the next block's header, or did not try.
          return -1;
        }
      }

      assert peekIntoNextBlock;
      return Bytes.toInt(dest, destOffset + size + BlockType.MAGIC_LENGTH) +
          hdrSize;
    }
{code}

Positional reads in FSInputStream (DFSInputStream) are heavily synchronized. It is lock on
stream than seek and read, unlock. Here is the code for FSInputStream:
{code}
  @Override
  public int read(long position, byte[] buffer, int offset, int length)
    throws IOException {
    synchronized (this) {
      long oldPos = getPos();
      int nread = -1;
      try {
        seek(position);
        nread = read(buffer, offset, length);
      } finally {
        seek(oldPos);
      }
      return nread;
    }
  }
{code}

DFSInputStream extends FSInputStream but does not override the above method. Taking into account
that code is synchronized, it is hard to explain observed performance improvement published
in this JIRA.


  


> HFileBlock.readAtOffset does not work well with multiple threads
> ----------------------------------------------------------------
>
>                 Key: HBASE-7336
>                 URL: https://issues.apache.org/jira/browse/HBASE-7336
>             Project: HBase
>          Issue Type: Sub-task
>          Components: Performance
>            Reporter: Lars Hofhansl
>            Assignee: Lars Hofhansl
>            Priority: Critical
>             Fix For: 0.94.4, 0.95.0
>
>         Attachments: 7336-0.94.txt, 7336-0.96.txt
>
>
> HBase grinds to a halt when many threads scan along the same set of blocks and neither
read short circuit is nor block caching is enabled for the dfs client ... disabling the block
cache makes sense on very large scans.
> It turns out that synchronizing in istream in HFileBlock.readAtOffset is the culprit.



--
This message was sent by Atlassian JIRA
(v6.2#6252)

Mime
View raw message