[
https://issues.apache.org/jira/browse/STORM-297?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14002311#comment-14002311
]
ASF GitHub Bot commented on STORM-297:
--------------------------------------
Github user revans2 commented on a diff in the pull request:
https://github.com/apache/incubator-storm/pull/103#discussion_r12813854
--- Diff:
storm-core/src/jvm/backtype/storm/messaging/netty/StormServerHandler.java ---
@@ -41,30 +45,22 @@ public void channelConnected(ChannelHandlerContext ctx,
ChannelStateEvent e) {
@Override
public void messageReceived(ChannelHandlerContext ctx, MessageEvent e)
{
- Object msg = e.getMessage();
- if (msg == null) return;
-
- //end of batch?
- if (msg==ControlMessage.EOB_MESSAGE) {
- Channel channel = ctx.getChannel();
- LOG.debug("Send back response ...");
- if (failure_count.get()==0)
- channel.write(ControlMessage.OK_RESPONSE);
- else channel.write(ControlMessage.FAILURE_RESPONSE);
- return;
- }
-
- //enqueue the received message for processing
- try {
- server.enqueue((TaskMessage)msg);
- } catch (InterruptedException e1) {
- LOG.info("failed to enqueue a request message", e);
- failure_count.incrementAndGet();
- }
+ List<TaskMessage> msgs = (List<TaskMessage>) e.getMessage();
+ if (msgs == null) {
+ return;
+ }
+
+ try {
+ server.enqueue(msgs);
+ } catch (InterruptedException e1) {
+ LOG.info("failed to enqueue a request message", e);
+ failure_count.incrementAndGet();
+ }
}
@Override
public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent
e) {
+ e.getCause().printStackTrace();
--- End diff --
Can we log this properly instead of just printing the stack trace to stderr?
> 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_Netty_receiver_diagram.png, 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)