You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@hbase.apache.org by "ryan rawson (JIRA)" <ji...@apache.org> on 2010/02/02 23:12:18 UTC

[jira] Created: (HBASE-2180) read performance from synchronizing hfile.fddatainputstream

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.


[jira] Updated: (HBASE-2180) read performance from synchronizing hfile.fddatainputstream

Posted by "stack (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-2180?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

stack updated HBASE-2180:
-------------------------

    Attachment: 2180.patch

This patch has gets do preads fetching blocks and uses the old seek+read for scans.

Patch removes the old HFile.Reader.getScanner methods and replaces both with a getScanner that takes two arguments -- whether to cache blocks read and whether to use pread or not pulling in the block.  I got rid of the old getScanners to force all getScanners to be explicit about what they want regards caching and pread.

This patch does not include tests.  Its hard to test for this performance change.

A further improvement would recognize short scans -- i.e. scans that are < an hfile block size.  In this case, we'd want to pread rather than seek+scan (especially so when scan one row replaces get)



> 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
>            Assignee: ryan rawson
>             Fix For: 0.21.0
>
>         Attachments: 2180.patch
>
>
> 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.


[jira] Commented: (HBASE-2180) Bad random read performance from synchronizing hfile.fddatainputstream

Posted by "Erik Rozendaal (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-2180?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12838399#action_12838399 ] 

Erik Rozendaal commented on HBASE-2180:
---------------------------------------

I saw both CLOSE_WAIT and TIME_WAIT. Maybe CLOSE_WAIT was in the majority. Connections were mostly to the data node.

$ uname -a
Linux inrdb-worker1.ripe.net 2.6.18-164.11.1.el5 #1 SMP Wed Jan 20 07:32:21 EST 2010 x86_64 x86_64 x86_64 GNU/Linux

They do go after a while, since after a few of the "Cannot assign requested address" exceptions the server starts working again.

Unfortunately I'll be away for the weekend and won't be able to investigate further. I wonder why so many connections are being opened so quickly that the server runs out of ports within a few minutes of starting the gets/puts?

> Bad random 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
>            Assignee: stack
>             Fix For: 0.20.4, 0.21.0
>
>         Attachments: 2180-v2.patch, 2180.patch
>
>
> 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.


[jira] Commented: (HBASE-2180) read performance from synchronizing hfile.fddatainputstream

Posted by "stack (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-2180?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12829324#action_12829324 ] 

stack commented on HBASE-2180:
------------------------------

Using pread -- its present in the code already, its just commented out on the line following the above cited by Ryan -- I see doubled throughput when 16 clients concurrently random reading out of a single regionserver so it helps.  i'll try and get some more numbers in here (I see 'wa' in top running at about the same for both cases but the regionserver is definetly working harder for the pread case using about double the CPU).

Numbers are not that good though -- about 50ms latency doing a random read when 16 concurrent clients.  This is a RS carrying 16M rows on 92 regions where there is 1 storefile only in the family and 4DNs under it.

Way back when we were looking at pread, it improved the random read latency by some small percentage IIRC, about 11%, but then scan speed slowed some... but these would have been for the case of low numbers of concurrent clients.

Its scanning 27k rows/second before the pread change using single client.  And 21k/second after.

Let me get some more numbers... up the concurrent client count and get some other points on how pread changes throughput.


> 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
>            Assignee: 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.


[jira] Commented: (HBASE-2180) Bad random read performance from synchronizing hfile.fddatainputstream

Posted by "stack (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-2180?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12838394#action_12838394 ] 

stack commented on HBASE-2180:
------------------------------

I was thinking of a very old issue, HADOOP-2341, but that was about CLOSE_WAIT, not TIME_WAIT.  Erik I presume the TIME_WAIT are on the datanode side?  I suppose there could be an issue here if many random reads in a short amount of time and the minimum segment lifetime (MSL) time is long in your tcp/ip implementation.  Do you know what it is?  2minutes seems default reading up on the internets so could be in TIME_WAIT for 4 minutes.  This what you are seeing you think Erik?   They go away after a while?  Whats the OS?  This would seem to be a new issue then.  We need pread that does keep-alive reusing sockets (Todd!).

> Bad random 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
>            Assignee: stack
>             Fix For: 0.20.4, 0.21.0
>
>         Attachments: 2180-v2.patch, 2180.patch
>
>
> 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.


[jira] Updated: (HBASE-2180) Bad random read performance from synchronizing hfile.fddatainputstream

Posted by "stack (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-2180?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

stack updated HBASE-2180:
-------------------------

    Fix Version/s: 0.20.4
         Assignee: stack  (was: ryan rawson)
          Summary: Bad random read performance from synchronizing hfile.fddatainputstream  (was: read performance from synchronizing hfile.fddatainputstream)

Committed branch and trunk.

> Bad random 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
>            Assignee: stack
>             Fix For: 0.20.4, 0.21.0
>
>         Attachments: 2180-v2.patch, 2180.patch
>
>
> 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.


[jira] Assigned: (HBASE-2180) read performance from synchronizing hfile.fddatainputstream

Posted by "ryan rawson (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-2180?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

ryan rawson reassigned HBASE-2180:
----------------------------------

    Assignee: ryan rawson

> 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
>            Assignee: 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.


[jira] Updated: (HBASE-2180) read performance from synchronizing hfile.fddatainputstream

Posted by "stack (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-2180?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

stack updated HBASE-2180:
-------------------------

    Attachment: 2180-v2.patch

This patch includes fixes for tests making them use new getScanner method and includes small PE fix when --rows is small (We would NPE).  I might need a v3.  A test is failing (TestGetDeleteTracker).  Need to investigate.

In testing on something that tries to resemble the yahoo papers testing -- ~20M rows per server, 116 regions on a RS and only one replica -- this patch seems to double the throughput if ~20 concurrent clients on a RS.  I tested scans and scan speeds are what they were w/ this patch in place.  They have not deterioated.

One thing I noticed was that scanning when the data is not local -- i.e. the data is in a DN on another machine -- there is added latency for sure.... taking maybe 25% as long again for the test to complete.  I need to see if same is true of random reads.  Cosmin suggested that the yahoo test with its single replica only might be doing lots of remote accessing and could be incurring the extra latency.

> 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
>            Assignee: ryan rawson
>             Fix For: 0.21.0
>
>         Attachments: 2180-v2.patch, 2180.patch
>
>
> 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.


[jira] Commented: (HBASE-2180) Bad random read performance from synchronizing hfile.fddatainputstream

Posted by "Erik Rozendaal (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-2180?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12838302#action_12838302 ] 

Erik Rozendaal commented on HBASE-2180:
---------------------------------------

After applying this patch to 0.20.3 I got the following errors in my regionserver logs when doing high loads of gets and puts:

2010-02-25 11:44:08,243 INFO org.apache.hadoop.hbase.regionserver.HRegion: compaction completed on region inrdb_ticket,\x07R\x00\x00\x00\x00\x80\xFF\xFF\xFF\x7F\x00\x00\x00\x01,1267094341820 i
n 6sec
1177:java.net.BindException: Cannot assign requested address
        at sun.nio.ch.Net.connect(Native Method)
        at sun.nio.ch.SocketChannelImpl.connect(SocketChannelImpl.java:507)
        at org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java:192)
        at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:404)
        at org.apache.hadoop.hdfs.DFSClient$DFSInputStream.fetchBlockByteRange(DFSClient.java:1825)
        at org.apache.hadoop.hdfs.DFSClient$DFSInputStream.read(DFSClient.java:1898)
        at org.apache.hadoop.fs.FSDataInputStream.read(FSDataInputStream.java:46)
        at org.apache.hadoop.hbase.io.hfile.BoundedRangeFileInputStream.read(BoundedRangeFileInputStream.java:101)
        at org.apache.hadoop.hbase.io.hfile.BoundedRangeFileInputStream.read(BoundedRangeFileInputStream.java:88)
        at org.apache.hadoop.hbase.io.hfile.BoundedRangeFileInputStream.read(BoundedRangeFileInputStream.java:81)
        at org.apache.hadoop.io.compress.BlockDecompressorStream.rawReadInt(BlockDecompressorStream.java:121)
        at org.apache.hadoop.io.compress.BlockDecompressorStream.getCompressedData(BlockDecompressorStream.java:96)
        at org.apache.hadoop.io.compress.BlockDecompressorStream.decompress(BlockDecompressorStream.java:82)
        at org.apache.hadoop.io.compress.DecompressorStream.read(DecompressorStream.java:74)
        at java.io.BufferedInputStream.read1(BufferedInputStream.java:256)
        at java.io.BufferedInputStream.read(BufferedInputStream.java:317)
        at org.apache.hadoop.io.IOUtils.readFully(IOUtils.java:100)
        at org.apache.hadoop.hbase.io.hfile.HFile$Reader.decompress(HFile.java:1018)
        at org.apache.hadoop.hbase.io.hfile.HFile$Reader.readBlock(HFile.java:966)
        at org.apache.hadoop.hbase.io.hfile.HFile$Reader$Scanner.next(HFile.java:1159)
        at org.apache.hadoop.hbase.regionserver.StoreFileGetScan.getStoreFile(StoreFileGetScan.java:108)
        at org.apache.hadoop.hbase.regionserver.StoreFileGetScan.get(StoreFileGetScan.java:65)
        at org.apache.hadoop.hbase.regionserver.Store.get(Store.java:1463)
        at org.apache.hadoop.hbase.regionserver.HRegion.get(HRegion.java:2396)
        at org.apache.hadoop.hbase.regionserver.HRegion.get(HRegion.java:2385)
        at org.apache.hadoop.hbase.regionserver.HRegionServer.get(HRegionServer.java:1731)
        at sun.reflect.GeneratedMethodAccessor7.invoke(Unknown Source)
        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
        at java.lang.reflect.Method.invoke(Method.java:597)
        at org.apache.hadoop.hbase.ipc.HBaseRPC$Server.call(HBaseRPC.java:657)
        at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:915)

The DataNode logs are fine (no maximum xcievers exceeded errors). Turns out that the OS was running out of port numbers. netstat showed more than 20,000 connections in TIME_WAIT state. Reverting to the original hbase-0.20.3 jar solved the problem. Only very few (<10) TIME_WAIT connections even after running gets/puts for a while.

So it looks like this patch causes some network connection issues. Any ideas if that could be the case?

PS Running only gets seems to be fine, but I've mostly run tests with reads from the block cache.

> Bad random 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
>            Assignee: stack
>             Fix For: 0.20.4, 0.21.0
>
>         Attachments: 2180-v2.patch, 2180.patch
>
>
> 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.


[jira] Commented: (HBASE-2180) Bad random read performance from synchronizing hfile.fddatainputstream

Posted by "stack (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-2180?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12839250#action_12839250 ] 

stack commented on HBASE-2180:
------------------------------

.bq I wonder why so many connections are being opened so quickly that the server runs out of ports within a few minutes of starting the gets/puts?

Gets used hdfs pread.  pread opens a socket per access.  My guess is that high rate of gets soon overwhelms the time each socket takes to clean up after close.  What kinda rates are we talking here Erik?

> Bad random 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
>            Assignee: stack
>             Fix For: 0.20.4, 0.21.0
>
>         Attachments: 2180-v2.patch, 2180.patch
>
>
> 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.


[jira] Resolved: (HBASE-2180) Bad random read performance from synchronizing hfile.fddatainputstream

Posted by "stack (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-2180?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

stack resolved HBASE-2180.
--------------------------

    Fix Version/s:     (was: 0.21.0)
       Resolution: Fixed

Resolving against 0.20.4.  I opened hbase-2492 to cover underlying new socket per pread.

> Bad random 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
>            Assignee: stack
>             Fix For: 0.20.4
>
>         Attachments: 2180-v2.patch, 2180.patch
>
>
> 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.


[jira] Resolved: (HBASE-2180) Bad random read performance from synchronizing hfile.fddatainputstream

Posted by "stack (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-2180?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

stack resolved HBASE-2180.
--------------------------

      Resolution: Fixed
    Hadoop Flags: [Reviewed]

Committed a while back.  Resolving.

> Bad random 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
>            Assignee: stack
>             Fix For: 0.20.4, 0.21.0
>
>         Attachments: 2180-v2.patch, 2180.patch
>
>
> 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.


[jira] Commented: (HBASE-2180) read performance from synchronizing hfile.fddatainputstream

Posted by "Andrei Dragomir (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-2180?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12829800#action_12829800 ] 

Andrei Dragomir commented on HBASE-2180:
----------------------------------------

We ran some testing that show improved performance by commenting in the change in BoundedFileInputStream, by replacing the synchronized statement with the one commented out, that uses the PositionalReader interface

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

> 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
>            Assignee: 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.


[jira] Reopened: (HBASE-2180) Bad random read performance from synchronizing hfile.fddatainputstream

Posted by "stack (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-2180?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

stack reopened HBASE-2180:
--------------------------


Reopening to take a look.

I have a vague recollection of stuff not being closed down if not all is read out of the socket.  Thanks for reporting this Erik.

> Bad random 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
>            Assignee: stack
>             Fix For: 0.20.4, 0.21.0
>
>         Attachments: 2180-v2.patch, 2180.patch
>
>
> 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.


[jira] Commented: (HBASE-2180) read performance from synchronizing hfile.fddatainputstream

Posted by "ryan rawson (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-2180?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12830585#action_12830585 ] 

ryan rawson commented on HBASE-2180:
------------------------------------

+1 thanks for doing this!

> 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
>            Assignee: ryan rawson
>             Fix For: 0.21.0
>
>         Attachments: 2180-v2.patch, 2180.patch
>
>
> 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.


[jira] Commented: (HBASE-2180) Bad random read performance from synchronizing hfile.fddatainputstream

Posted by "stack (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-2180?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12831193#action_12831193 ] 

stack commented on HBASE-2180:
------------------------------

Really commit to TRUNK.

> Bad random 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
>            Assignee: stack
>             Fix For: 0.20.4, 0.21.0
>
>         Attachments: 2180-v2.patch, 2180.patch
>
>
> 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.


[jira] Commented: (HBASE-2180) Bad random read performance from synchronizing hfile.fddatainputstream

Posted by "Todd Lipcon (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-2180?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12840885#action_12840885 ] 

Todd Lipcon commented on HBASE-2180:
------------------------------------

In the absence of reusing sockets, I think the TIME_WAIT issue could be dealt with on the system level by toggling /proc/sys/net/ipv4/tcp_tw_recycle

> Bad random 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
>            Assignee: stack
>             Fix For: 0.20.4, 0.21.0
>
>         Attachments: 2180-v2.patch, 2180.patch
>
>
> 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.