[ 
https://issues.apache.org/jira/browse/STORM-297?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13991586#comment-13991586
 ] 

ASF GitHub Bot commented on STORM-297:
--------------------------------------

Github user clockfly commented on a diff in the pull request:

    https://github.com/apache/incubator-storm/pull/103#discussion_r12362595
  
    --- Diff: 
storm-core/src/jvm/backtype/storm/messaging/netty/MessageDecoder.java ---
    @@ -34,52 +37,78 @@
          */
         protected Object decode(ChannelHandlerContext ctx, Channel channel, 
ChannelBuffer buf) throws Exception {
             // Make sure that we have received at least a short 
    -        if (buf.readableBytes() < 2) {
    +        long available = buf.readableBytes();
    +        if (available < 2) {
                 //need more data
                 return null;
             }
     
    -        // Mark the current buffer position before reading task/len field
    -        // because the whole frame might not be in the buffer yet.
    -        // We will reset the buffer position to the marked position if
    -        // there's not enough bytes in the buffer.
    -        buf.markReaderIndex();
    -
    -        //read the short field
    -        short code = buf.readShort();
    -        
    -        //case 1: Control message
    -        ControlMessage ctrl_msg = ControlMessage.mkMessage(code);
    -        if (ctrl_msg != null) return ctrl_msg;
    -        
    -        //case 2: task Message
    -        short task = code;
    -        
    -        // Make sure that we have received at least an integer (length) 
    -        if (buf.readableBytes() < 4) {
    -            //need more data
    -            buf.resetReaderIndex();
    -            return null;
    -        }
    +        List<Object> ret = new ArrayList<Object>();
    +
    +        while (available >= 2) {
    --- End diff --
    
    while loop, try to decode as more messages as possible.


> Storm Performance cannot be scaled up by adding more CPU cores
> --------------------------------------------------------------
>
>                 Key: STORM-297
>                 URL: https://issues.apache.org/jira/browse/STORM-297
>             Project: Apache Storm (Incubating)
>          Issue Type: Bug
>            Reporter: Sean Zhong
>              Labels: Performance, netty
>             Fix For: 0.9.2-incubating
>
>         Attachments: Storm_performance_fix.pdf, storm_performance_fix.patch
>
>
> We cannot scale up the performance by adding more CPU cores and increasing 
> parallelism.
> For a 2 layer topology Spout ---shuffle grouping--> bolt, when message size 
> is small (around 100 bytes), we can find in the below picture that neither 
> the CPU nor the network is saturated. When message size is 100 bytes, only 
> 40% of CPU is used, only 18% of network is used, although we have a high 
> parallelism (overall we have 144 executors)



--
This message was sent by Atlassian JIRA
(v6.2#6252)

Reply via email to