read performance from synchronizing hfile.fddatainputstream
-----------------------------------------------------------

                 Key: HBASE-2180
                 URL: https://issues.apache.org/jira/browse/HBASE-2180
             Project: Hadoop HBase
          Issue Type: Bug
            Reporter: ryan rawson
             Fix For: 0.21.0


deep in the HFile read path, there is this code:

    synchronized (in) {
      in.seek(pos);
      ret = in.read(b, off, n);
    }


this makes it so that only 1 read per file per thread is active. this prevents 
the OS and hardware from being able to do IO scheduling by optimizing lots of 
concurrent reads. 

We need to either use a reentrant API (pread may be partially reentrant 
according to Todd) or use multiple stream objects, 1 per scanner/thread.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.

Reply via email to