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

stack commented on HBASE-1394:
------------------------------

Was thinking the log should be compressed.  Now I think otherwise.  If gzip for 
instance, after the header, data is compressed in "blocks" of 32k (IIRC).  That 
means that since the gzip block is not tied off properly, we will lose up to 
32k edits regardless of whether we hdfs sync.

> Uploads sometimes fall to 0 requests/second (Binding up on HLog#append?)
> ------------------------------------------------------------------------
>
>                 Key: HBASE-1394
>                 URL: https://issues.apache.org/jira/browse/HBASE-1394
>             Project: Hadoop HBase
>          Issue Type: Bug
>            Reporter: stack
>            Priority: Critical
>             Fix For: 0.20.0
>
>         Attachments: loghowlongittookappendinghlog.patch
>
>
> Trying to figure why rate sometimes goes to zero.
> Studying the reginoserver, HLog#append looks like a possible culprit.
> {code}
> "IPC Server handler 7 on 60021" daemon prio=10 tid=0x000000004057dc00 
> nid=0x1bc4 in Object.wait() [0x0000000043393000..0x0000000043393b80]
>    java.lang.Thread.State: WAITING (on object monitor)
>         at java.lang.Object.wait(Native Method)
>         at java.lang.Object.wait(Object.java:485)
>         at 
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.writeChunk(DFSClient.java:2964)
>         - locked <0x00007f9e3e449ff0> (a java.util.LinkedList)
>         - locked <0x00007f9e3e449e18> (a 
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream)
>         at 
> org.apache.hadoop.fs.FSOutputSummer.writeChecksumChunk(FSOutputSummer.java:150)
>         at org.apache.hadoop.fs.FSOutputSummer.write1(FSOutputSummer.java:100)
>         at org.apache.hadoop.fs.FSOutputSummer.write(FSOutputSummer.java:86)
>         - locked <0x00007f9e3e449e18> (a 
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream)
>         at 
> org.apache.hadoop.fs.FSDataOutputStream$PositionCache.write(FSDataOutputStream.java:49)
>         at java.io.DataOutputStream.write(DataOutputStream.java:90)
>         - locked <0x00007f9e434e5588> (a 
> org.apache.hadoop.fs.FSDataOutputStream)
>         at 
> org.apache.hadoop.io.SequenceFile$Writer.append(SequenceFile.java:1020)
>         - locked <0x00007f9e434e55c0> (a 
> org.apache.hadoop.io.SequenceFile$Writer)
>         at 
> org.apache.hadoop.io.SequenceFile$Writer.append(SequenceFile.java:984)
>         - locked <0x00007f9e434e55c0> (a 
> org.apache.hadoop.io.SequenceFile$Writer)
>         at org.apache.hadoop.hbase.regionserver.HLog.doWrite(HLog.java:565)
>         at org.apache.hadoop.hbase.regionserver.HLog.append(HLog.java:521)
>         - locked <0x00007f9dfa376f70> (a java.lang.Object)
>         at 
> org.apache.hadoop.hbase.regionserver.HRegion.update(HRegion.java:1777)
>         at 
> org.apache.hadoop.hbase.regionserver.HRegion.batchUpdate(HRegion.java:1348)
>         at 
> org.apache.hadoop.hbase.regionserver.HRegion.batchUpdate(HRegion.java:1289)
>         at 
> org.apache.hadoop.hbase.regionserver.HRegionServer.batchUpdates(HRegionServer.java:1727)
>         at sun.reflect.GeneratedMethodAccessor9.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:642)
>         at 
> org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:911)
> {code}

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