[jira] [Updated] (HBASE-5945) Reduce buffer copies in IPC server response path

2013-11-23 Thread stack (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-5945?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

stack updated HBASE-5945:
-

   Resolution: Fixed
Fix Version/s: 0.98.0
 Hadoop Flags: Reviewed
   Status: Resolved  (was: Patch Available)

Committed gathering patch.  Further cleanup to be done in other issues (e.g. 
the issue linked to this one).

> Reduce buffer copies in IPC server response path
> 
>
> Key: HBASE-5945
> URL: https://issues.apache.org/jira/browse/HBASE-5945
> Project: HBase
>  Issue Type: Improvement
>  Components: IPC/RPC
>Affects Versions: 0.95.2
>Reporter: Todd Lipcon
>Assignee: stack
> Fix For: 0.98.0, 0.96.1
>
> Attachments: 5945-in-progress.2.1.patch, 5945-in-progress.2.patch, 
> 5945-in-progress.patch, 5945v2.txt, 5945v4.txt, 5945v4.txt, 5945v5.txt, 
> buffer-copies.txt, even-fewer-copies.txt, hbase-5495.txt, with_patch.png, 
> without_patch.png
>
>
> The new PB code is sloppy with buffers and makes several needless copies. 
> This increases GC time a lot. A few simple changes can cut this back down.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Updated] (HBASE-5945) Reduce buffer copies in IPC server response path

2013-11-17 Thread stack (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-5945?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

stack updated HBASE-5945:
-

Attachment: 5945v5.txt

Fix the zombie created (was making a cell block stream though no cells in it)

> Reduce buffer copies in IPC server response path
> 
>
> Key: HBASE-5945
> URL: https://issues.apache.org/jira/browse/HBASE-5945
> Project: HBase
>  Issue Type: Improvement
>  Components: IPC/RPC
>Affects Versions: 0.95.2
>Reporter: Todd Lipcon
>Assignee: stack
> Fix For: 0.96.1
>
> Attachments: 5945-in-progress.2.1.patch, 5945-in-progress.2.patch, 
> 5945-in-progress.patch, 5945v2.txt, 5945v4.txt, 5945v4.txt, 5945v5.txt, 
> buffer-copies.txt, even-fewer-copies.txt, hbase-5495.txt, with_patch.png, 
> without_patch.png
>
>
> The new PB code is sloppy with buffers and makes several needless copies. 
> This increases GC time a lot. A few simple changes can cut this back down.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Updated] (HBASE-5945) Reduce buffer copies in IPC server response path

2013-11-15 Thread stack (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-5945?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

stack updated HBASE-5945:
-

Attachment: 5945v4.txt

Upload patch again so hadoopqa picks up this.

> Reduce buffer copies in IPC server response path
> 
>
> Key: HBASE-5945
> URL: https://issues.apache.org/jira/browse/HBASE-5945
> Project: HBase
>  Issue Type: Improvement
>  Components: IPC/RPC
>Affects Versions: 0.95.2
>Reporter: Todd Lipcon
>Assignee: stack
> Fix For: 0.96.1
>
> Attachments: 5945-in-progress.2.1.patch, 5945-in-progress.2.patch, 
> 5945-in-progress.patch, 5945v2.txt, 5945v4.txt, 5945v4.txt, 
> buffer-copies.txt, even-fewer-copies.txt, hbase-5495.txt, with_patch.png, 
> without_patch.png
>
>
> The new PB code is sloppy with buffers and makes several needless copies. 
> This increases GC time a lot. A few simple changes can cut this back down.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Updated] (HBASE-5945) Reduce buffer copies in IPC server response path

2013-11-15 Thread stack (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-5945?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

stack updated HBASE-5945:
-

Attachment: with_patch.png
without_patch.png

Here are results running little benchmark on end of TestIPC in its main.
It sets up the rpc doing a little echo protocol.  The echo copies the cells
it receives onto the response. On cmdline say how many cycles and how many 
columns.  I made Cell size be about 10k and ran the test adding 10 Cells per 
iteration so we are sending back and forth about 100k.  This approximates a 
small to medium-sized mult call.  I cycled 10k times.  Below are the test done 
twice.

With patch, the test finishes a little sooner... about 5-10% sooner.

I ran visualvm over a minute+ against each at about same stage in test.
Without patch we use more CPU and do more GC -- just over 36% CPU vs 33% or so 
and we do a bit more GC'ing... 4.1% or so vs 3.4% or so.  W/o the patch, more 
heap is used.  See pictures.  Patch seems to be improvement.

WITHOUT PATCH

durruti:hbase.git stack$ for i in 1 2 3 4 5; do time ./bin/hbase 
-Dhbase.defaults.for.version.skip=true org.apache.hadoop.hbase.ipc.TestIPC 
10 10 &> /tmp/wopatch.$i.txt; done

real0m42.843s
user0m43.902s
sys 0m17.495s

real0m43.357s
user0m46.050s
sys 0m17.712s

real0m42.595s
user0m44.179s
sys 0m17.448s

real0m43.320s
user0m45.578s
sys 0m17.736s

real0m42.647s
user0m44.845s
sys 0m17.583s

... and again

real0m45.868s
user0m46.522s
sys 0m18.776s

real0m42.764s
user0m44.505s
sys 0m17.447s

real0m43.080s
user0m45.445s
sys 0m17.585s

real0m43.261s
user0m45.246s
sys 0m17.722s

real0m42.592s
user0m44.102s
sys 0m17.333s


WITH PATCH

durruti:hbase.git stack$ for i in 1 2 3 4 5; do time ./bin/hbase 
-Dhbase.defaults.for.version.skip=true org.apache.hadoop.hbase.ipc.TestIPC 
10 10 &> /tmp/wpatch.$i.txt; done

real0m38.838s
user0m40.415s
sys 0m18.765s

real0m37.638s
user0m39.246s
sys 0m18.408s

real0m38.696s
user0m40.169s
sys 0m18.700s

real0m37.948s
user0m39.403s
sys 0m18.682s


real0m38.077s
user0m39.519s
sys 0m18.571s

...and again.

real0m43.888s
user0m44.394s
sys 0m21.427s

real0m40.311s
user0m42.553s
sys 0m19.460s

real0m38.489s
user0m41.097s
sys 0m18.761s

real0m38.252s
user0m39.603s
sys 0m18.618s

real0m38.066s
user0m39.656s
sys 0m18.621s

> Reduce buffer copies in IPC server response path
> 
>
> Key: HBASE-5945
> URL: https://issues.apache.org/jira/browse/HBASE-5945
> Project: HBase
>  Issue Type: Improvement
>  Components: IPC/RPC
>Affects Versions: 0.95.2
>Reporter: Todd Lipcon
>Assignee: stack
> Fix For: 0.96.1
>
> Attachments: 5945-in-progress.2.1.patch, 5945-in-progress.2.patch, 
> 5945-in-progress.patch, 5945v2.txt, 5945v4.txt, buffer-copies.txt, 
> even-fewer-copies.txt, hbase-5495.txt, with_patch.png, without_patch.png
>
>
> The new PB code is sloppy with buffers and makes several needless copies. 
> This increases GC time a lot. A few simple changes can cut this back down.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Updated] (HBASE-5945) Reduce buffer copies in IPC server response path

2013-11-15 Thread stack (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-5945?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

stack updated HBASE-5945:
-

Attachment: 5945v4.txt

Fixes the h2 build failure (forgot to change the h2 compat module for the 
metrics change).  Does a little fix up on the end of TestIPC so I can do a bit 
of a benchmark on this patch.

> Reduce buffer copies in IPC server response path
> 
>
> Key: HBASE-5945
> URL: https://issues.apache.org/jira/browse/HBASE-5945
> Project: HBase
>  Issue Type: Improvement
>  Components: IPC/RPC
>Affects Versions: 0.95.2
>Reporter: Todd Lipcon
>Assignee: stack
> Fix For: 0.96.1
>
> Attachments: 5945-in-progress.2.1.patch, 5945-in-progress.2.patch, 
> 5945-in-progress.patch, 5945v2.txt, 5945v4.txt, buffer-copies.txt, 
> even-fewer-copies.txt, hbase-5495.txt
>
>
> The new PB code is sloppy with buffers and makes several needless copies. 
> This increases GC time a lot. A few simple changes can cut this back down.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Updated] (HBASE-5945) Reduce buffer copies in IPC server response path

2013-11-14 Thread stack (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-5945?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

stack updated HBASE-5945:
-

Status: Patch Available  (was: Open)

> Reduce buffer copies in IPC server response path
> 
>
> Key: HBASE-5945
> URL: https://issues.apache.org/jira/browse/HBASE-5945
> Project: HBase
>  Issue Type: Improvement
>  Components: IPC/RPC
>Affects Versions: 0.95.2
>Reporter: Todd Lipcon
>Assignee: stack
> Fix For: 0.96.1
>
> Attachments: 5945-in-progress.2.1.patch, 5945-in-progress.2.patch, 
> 5945-in-progress.patch, 5945v2.txt, buffer-copies.txt, even-fewer-copies.txt, 
> hbase-5495.txt
>
>
> The new PB code is sloppy with buffers and makes several needless copies. 
> This increases GC time a lot. A few simple changes can cut this back down.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Updated] (HBASE-5945) Reduce buffer copies in IPC server response path

2013-11-13 Thread stack (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-5945?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

stack updated HBASE-5945:
-

Attachment: 5945v2.txt

Forward port Todd's old idea of saving copies by passing array of ByteBuffers 
to Channel rather than compose response in a buffer first before putting it on 
the wire.  Revives his BufferChain doohickey.

Here's some comment on the changes.

M hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java

Added getDelimitedMessageAsByteBuffer(final Message m)
Removed no longer used write method and getDelimitedMessageBytes.

M 
hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSource.java

Changed bytes sent size from int to long (it is long in implementation, just 
not in interfaces)

A hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/BufferChain.java

Class to carry an array of ByteBuffers and that can write the byte buffers in 
order
out to a GatheringByteChannel in chunks.

M hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java

Refactored response sending to use BufferChain.  We save on a composition of 
total
response in memory.  Instead we now pass the parts directly to the socket 
channel.

Stuff is a bit ugly in the sasl handling because have to compose the message
in memory completely so can do the sasl wrapping; this undoes any savings but
I believe we are still ahead because we removed the serializing of the resulting
sasl array to an output stream backed by a new buffer.

> Reduce buffer copies in IPC server response path
> 
>
> Key: HBASE-5945
> URL: https://issues.apache.org/jira/browse/HBASE-5945
> Project: HBase
>  Issue Type: Improvement
>  Components: IPC/RPC
>Affects Versions: 0.95.2
>Reporter: Todd Lipcon
>Assignee: stack
> Fix For: 0.96.1
>
> Attachments: 5945-in-progress.2.1.patch, 5945-in-progress.2.patch, 
> 5945-in-progress.patch, 5945v2.txt, buffer-copies.txt, even-fewer-copies.txt, 
> hbase-5495.txt
>
>
> The new PB code is sloppy with buffers and makes several needless copies. 
> This increases GC time a lot. A few simple changes can cut this back down.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Updated] (HBASE-5945) Reduce buffer copies in IPC server response path

2013-10-19 Thread stack (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-5945?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

stack updated HBASE-5945:
-

Fix Version/s: (was: 0.96.0)
   0.96.1

> Reduce buffer copies in IPC server response path
> 
>
> Key: HBASE-5945
> URL: https://issues.apache.org/jira/browse/HBASE-5945
> Project: HBase
>  Issue Type: Improvement
>  Components: IPC/RPC
>Affects Versions: 0.95.2
>Reporter: Todd Lipcon
>Assignee: stack
> Fix For: 0.96.1
>
> Attachments: 5945-in-progress.2.1.patch, 5945-in-progress.2.patch, 
> 5945-in-progress.patch, buffer-copies.txt, even-fewer-copies.txt, 
> hbase-5495.txt
>
>
> The new PB code is sloppy with buffers and makes several needless copies. 
> This increases GC time a lot. A few simple changes can cut this back down.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Updated] (HBASE-5945) Reduce buffer copies in IPC server response path

2013-08-15 Thread stack (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-5945?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

stack updated HBASE-5945:
-

Fix Version/s: (was: 0.95.2)
   0.96.0

> Reduce buffer copies in IPC server response path
> 
>
> Key: HBASE-5945
> URL: https://issues.apache.org/jira/browse/HBASE-5945
> Project: HBase
>  Issue Type: Improvement
>  Components: IPC/RPC
>Affects Versions: 0.95.2
>Reporter: Todd Lipcon
>Assignee: stack
> Fix For: 0.96.0
>
> Attachments: 5945-in-progress.2.1.patch, 5945-in-progress.2.patch, 
> 5945-in-progress.patch, buffer-copies.txt, even-fewer-copies.txt, 
> hbase-5495.txt
>
>
> The new PB code is sloppy with buffers and makes several needless copies. 
> This increases GC time a lot. A few simple changes can cut this back down.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (HBASE-5945) Reduce buffer copies in IPC server response path

2013-06-11 Thread stack (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-5945?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

stack updated HBASE-5945:
-

Fix Version/s: (was: 0.95.1)
   0.95.2

> Reduce buffer copies in IPC server response path
> 
>
> Key: HBASE-5945
> URL: https://issues.apache.org/jira/browse/HBASE-5945
> Project: HBase
>  Issue Type: Improvement
>  Components: IPC/RPC
>Affects Versions: 0.95.2
>Reporter: Todd Lipcon
>Assignee: stack
> Fix For: 0.95.2
>
> Attachments: 5945-in-progress.2.1.patch, 5945-in-progress.2.patch, 
> 5945-in-progress.patch, buffer-copies.txt, even-fewer-copies.txt, 
> hbase-5495.txt
>
>
> The new PB code is sloppy with buffers and makes several needless copies. 
> This increases GC time a lot. A few simple changes can cut this back down.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (HBASE-5945) Reduce buffer copies in IPC server response path

2013-03-27 Thread Devaraj Das (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-5945?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Devaraj Das updated HBASE-5945:
---

Status: Open  (was: Patch Available)

This patch is really stale. Canceling.

> Reduce buffer copies in IPC server response path
> 
>
> Key: HBASE-5945
> URL: https://issues.apache.org/jira/browse/HBASE-5945
> Project: HBase
>  Issue Type: Improvement
>  Components: IPC/RPC
>Affects Versions: 0.96.0
>Reporter: Todd Lipcon
>Assignee: stack
> Fix For: 0.95.0
>
> Attachments: 5945-in-progress.2.1.patch, 5945-in-progress.2.patch, 
> 5945-in-progress.patch, buffer-copies.txt, even-fewer-copies.txt, 
> hbase-5495.txt
>
>
> The new PB code is sloppy with buffers and makes several needless copies. 
> This increases GC time a lot. A few simple changes can cut this back down.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (HBASE-5945) Reduce buffer copies in IPC server response path

2013-03-27 Thread Devaraj Das (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-5945?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Devaraj Das updated HBASE-5945:
---

Priority: Major  (was: Blocker)

Downgrading priority since I don't think this is a relevant issue anymore. 
Please discuss if you feel otherwise.

> Reduce buffer copies in IPC server response path
> 
>
> Key: HBASE-5945
> URL: https://issues.apache.org/jira/browse/HBASE-5945
> Project: HBase
>  Issue Type: Improvement
>  Components: IPC/RPC
>Affects Versions: 0.96.0
>Reporter: Todd Lipcon
>Assignee: stack
> Fix For: 0.95.0
>
> Attachments: 5945-in-progress.2.1.patch, 5945-in-progress.2.patch, 
> 5945-in-progress.patch, buffer-copies.txt, even-fewer-copies.txt, 
> hbase-5495.txt
>
>
> The new PB code is sloppy with buffers and makes several needless copies. 
> This increases GC time a lot. A few simple changes can cut this back down.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (HBASE-5945) Reduce buffer copies in IPC server response path

2013-01-09 Thread Devaraj Das (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-5945?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Devaraj Das updated HBASE-5945:
---

Attachment: 5945-in-progress.2.1.patch

This is rebased patch (after HBASE-7479). I have updated RPC.proto in the patch 
with a description of what the RPC messages are. I think we can handle the 
KeyValue bytebuffers as well (in the RPC layer by defining a common interface 
for Message and ByteBuffer[]). For now, I haven't done it. I only introduced a 
responseBodyType blob that could be used to indicate the type of response. I 
also put in a TODO in RpcServer.java explaining the interface.

bq. Is this saving a buffer copy or is it just doing what writeDelimitedTo does?

Actually, in principle it is doing the same thing as writeDelimitedTo. But 
writeDelimitedTo creates a buffer internally which is of little use since we 
are already writing the output to a buffer. I don't know whether this will make 
any difference or not in practice but I thought why simply create a buffer 
unnecessarily..I create CodedOutputStream with a buffer size of '1' (buffer 
size of '0' makes it throw exceptions..).  

> Reduce buffer copies in IPC server response path
> 
>
> Key: HBASE-5945
> URL: https://issues.apache.org/jira/browse/HBASE-5945
> Project: HBase
>  Issue Type: Improvement
>  Components: IPC/RPC
>Affects Versions: 0.96.0
>Reporter: Todd Lipcon
>Assignee: stack
>Priority: Blocker
> Fix For: 0.96.0
>
> Attachments: 5945-in-progress.2.1.patch, 5945-in-progress.2.patch, 
> 5945-in-progress.patch, buffer-copies.txt, even-fewer-copies.txt, 
> hbase-5495.txt
>
>
> The new PB code is sloppy with buffers and makes several needless copies. 
> This increases GC time a lot. A few simple changes can cut this back down.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (HBASE-5945) Reduce buffer copies in IPC server response path

2013-01-04 Thread Devaraj Das (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-5945?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Devaraj Das updated HBASE-5945:
---

Attachment: 5945-in-progress.2.patch

Had uploaded an incomplete patch earlier.. This is the more appropriate one.

> Reduce buffer copies in IPC server response path
> 
>
> Key: HBASE-5945
> URL: https://issues.apache.org/jira/browse/HBASE-5945
> Project: HBase
>  Issue Type: Improvement
>  Components: IPC/RPC
>Affects Versions: 0.96.0
>Reporter: Todd Lipcon
>Assignee: stack
>Priority: Blocker
> Fix For: 0.96.0
>
> Attachments: 5945-in-progress.2.patch, 5945-in-progress.patch, 
> buffer-copies.txt, even-fewer-copies.txt, hbase-5495.txt
>
>
> The new PB code is sloppy with buffers and makes several needless copies. 
> This increases GC time a lot. A few simple changes can cut this back down.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (HBASE-5945) Reduce buffer copies in IPC server response path

2012-12-27 Thread Devaraj Das (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-5945?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Devaraj Das updated HBASE-5945:
---

Attachment: 5945-in-progress.patch

[~stack], hope it is okay with you that I am attaching a patch here... (I was 
tempted to do a patch for this issue..)

The main change in this patch is that it removes the RpcRequestBody from 
RPC.proto, and instead serializes the rpc method argument directly to the 
underlying output. There is a lot of change associated with this removal in the 
patch.

I didn't change the serialization of the rpc header fields yet (as was done by 
[~tlipcon] in his earlier patch). The reason being that I don't think the 
header part is a concern since they will be a few 10s of bytes and mostly will 
be noise. The ser/de of the rpc request-body/response-body is where I tried to 
improve on. The codebase compiles with this patch now but there is work still 
left (and maybe I missed some buffer copies as well; need to dig some more).

> Reduce buffer copies in IPC server response path
> 
>
> Key: HBASE-5945
> URL: https://issues.apache.org/jira/browse/HBASE-5945
> Project: HBase
>  Issue Type: Improvement
>  Components: IPC/RPC
>Affects Versions: 0.96.0
>Reporter: Todd Lipcon
>Assignee: stack
>Priority: Blocker
> Fix For: 0.96.0
>
> Attachments: 5945-in-progress.patch, buffer-copies.txt, 
> even-fewer-copies.txt, hbase-5495.txt
>
>
> The new PB code is sloppy with buffers and makes several needless copies. 
> This increases GC time a lot. A few simple changes can cut this back down.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (HBASE-5945) Reduce buffer copies in IPC server response path

2012-12-20 Thread stack (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-5945?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

stack updated HBASE-5945:
-

Fix Version/s: 0.96.0

> Reduce buffer copies in IPC server response path
> 
>
> Key: HBASE-5945
> URL: https://issues.apache.org/jira/browse/HBASE-5945
> Project: HBase
>  Issue Type: Improvement
>  Components: IPC/RPC
>Affects Versions: 0.96.0
>Reporter: Todd Lipcon
>Assignee: stack
>Priority: Blocker
> Fix For: 0.96.0
>
> Attachments: buffer-copies.txt, even-fewer-copies.txt, hbase-5495.txt
>
>
> The new PB code is sloppy with buffers and makes several needless copies. 
> This increases GC time a lot. A few simple changes can cut this back down.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (HBASE-5945) Reduce buffer copies in IPC server response path

2012-08-27 Thread Ted Yu (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-5945?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ted Yu updated HBASE-5945:
--

Priority: Blocker  (was: Critical)

> Reduce buffer copies in IPC server response path
> 
>
> Key: HBASE-5945
> URL: https://issues.apache.org/jira/browse/HBASE-5945
> Project: HBase
>  Issue Type: Improvement
>  Components: ipc
>Affects Versions: 0.96.0
>Reporter: Todd Lipcon
>Assignee: Todd Lipcon
>Priority: Blocker
> Attachments: buffer-copies.txt, even-fewer-copies.txt, hbase-5495.txt
>
>
> The new PB code is sloppy with buffers and makes several needless copies. 
> This increases GC time a lot. A few simple changes can cut this back down.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (HBASE-5945) Reduce buffer copies in IPC server response path

2012-05-16 Thread Todd Lipcon (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-5945?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Todd Lipcon updated HBASE-5945:
---

Status: Patch Available  (was: Open)

> Reduce buffer copies in IPC server response path
> 
>
> Key: HBASE-5945
> URL: https://issues.apache.org/jira/browse/HBASE-5945
> Project: HBase
>  Issue Type: Improvement
>  Components: ipc
>Affects Versions: 0.96.0
>Reporter: Todd Lipcon
>Assignee: Todd Lipcon
>Priority: Critical
> Attachments: buffer-copies.txt, even-fewer-copies.txt, hbase-5495.txt
>
>
> The new PB code is sloppy with buffers and makes several needless copies. 
> This increases GC time a lot. A few simple changes can cut this back down.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Updated] (HBASE-5945) Reduce buffer copies in IPC server response path

2012-05-06 Thread Todd Lipcon (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-5945?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Todd Lipcon updated HBASE-5945:
---

Attachment: hbase-5495.txt

attached does the optimization on the client side as well, and fixes a bug with 
serializing exception responses (forgot to length-prefix it)

> Reduce buffer copies in IPC server response path
> 
>
> Key: HBASE-5945
> URL: https://issues.apache.org/jira/browse/HBASE-5945
> Project: HBase
>  Issue Type: Improvement
>  Components: ipc
>Affects Versions: 0.96.0
>Reporter: Todd Lipcon
>Assignee: Todd Lipcon
>Priority: Critical
> Attachments: buffer-copies.txt, even-fewer-copies.txt, hbase-5495.txt
>
>
> The new PB code is sloppy with buffers and makes several needless copies. 
> This increases GC time a lot. A few simple changes can cut this back down.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Updated] (HBASE-5945) Reduce buffer copies in IPC server response path

2012-05-05 Thread stack (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-5945?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

stack updated HBASE-5945:
-

Priority: Critical  (was: Minor)

Making critical so we don't overlook this work.

> Reduce buffer copies in IPC server response path
> 
>
> Key: HBASE-5945
> URL: https://issues.apache.org/jira/browse/HBASE-5945
> Project: HBase
>  Issue Type: Improvement
>  Components: ipc
>Affects Versions: 0.96.0
>Reporter: Todd Lipcon
>Assignee: Todd Lipcon
>Priority: Critical
> Attachments: buffer-copies.txt, even-fewer-copies.txt
>
>
> The new PB code is sloppy with buffers and makes several needless copies. 
> This increases GC time a lot. A few simple changes can cut this back down.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Updated] (HBASE-5945) Reduce buffer copies in IPC server response path

2012-05-05 Thread Todd Lipcon (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-5945?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Todd Lipcon updated HBASE-5945:
---

Attachment: even-fewer-copies.txt

New rev gets rid of some more. This seems to make a noticeable difference in my 
oprofile output and YCSB results. Would appreciate if other folks could verify

(yes, patch still needs more work, please don't review for style/licenses/etc)

> Reduce buffer copies in IPC server response path
> 
>
> Key: HBASE-5945
> URL: https://issues.apache.org/jira/browse/HBASE-5945
> Project: HBase
>  Issue Type: Improvement
>  Components: ipc
>Affects Versions: 0.96.0
>Reporter: Todd Lipcon
>Assignee: Todd Lipcon
>Priority: Minor
> Attachments: buffer-copies.txt, even-fewer-copies.txt
>
>
> The new PB code is sloppy with buffers and makes several needless copies. 
> This increases GC time a lot. A few simple changes can cut this back down.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Updated] (HBASE-5945) Reduce buffer copies in IPC server response path

2012-05-04 Thread Todd Lipcon (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-5945?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Todd Lipcon updated HBASE-5945:
---

Attachment: buffer-copies.txt

Here's a first pass. I don't have great benchmark results, but it looked from 
my "jstat -gcutil" output like this cut the amount of allocation by a factor of 
2.5x or so for a benchmark that returned large responses.

Can probably get rid of at least one more copy in there with a bit more 
trickery.

> Reduce buffer copies in IPC server response path
> 
>
> Key: HBASE-5945
> URL: https://issues.apache.org/jira/browse/HBASE-5945
> Project: HBase
>  Issue Type: Improvement
>  Components: ipc
>Affects Versions: 0.96.0
>Reporter: Todd Lipcon
>Assignee: Todd Lipcon
>Priority: Minor
> Attachments: buffer-copies.txt
>
>
> The new PB code is sloppy with buffers and makes several needless copies. 
> This increases GC time a lot. A few simple changes can cut this back down.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira