[ 
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.

Reply via email to