You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@storm.apache.org by go...@apache.org on 2019/12/02 16:42:08 UTC

[storm] branch master updated: STORM-3540 fix Pacemaker connection issues

This is an automated email from the ASF dual-hosted git repository.

govind pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
     new 49c0948  STORM-3540 fix Pacemaker connection issues
     new 0f9f3fb  Merge pull request #3169 from agresch/agresch_pacemaker_client
49c0948 is described below

commit 49c0948b3696f5b9a7b04f33656fc0e1032e2c61
Author: Aaron Gresch <ag...@yahoo-inc.com>
AuthorDate: Thu Nov 7 15:23:00 2019 -0600

    STORM-3540 fix Pacemaker connection issues
---
 .../src/jvm/org/apache/storm/pacemaker/PacemakerClientHandler.java     | 3 ++-
 .../jvm/org/apache/storm/pacemaker/codec/ThriftNettyClientCodec.java   | 2 +-
 2 files changed, 3 insertions(+), 2 deletions(-)

diff --git a/storm-client/src/jvm/org/apache/storm/pacemaker/PacemakerClientHandler.java b/storm-client/src/jvm/org/apache/storm/pacemaker/PacemakerClientHandler.java
index b81f02d..31fabc1 100644
--- a/storm-client/src/jvm/org/apache/storm/pacemaker/PacemakerClientHandler.java
+++ b/storm-client/src/jvm/org/apache/storm/pacemaker/PacemakerClientHandler.java
@@ -36,6 +36,7 @@ public class PacemakerClientHandler extends ChannelInboundHandlerAdapter {
         Channel channel = ctx.channel();
         LOG.info("Connection established from {} to {}",
                  channel.localAddress(), channel.remoteAddress());
+        client.channelReady(channel);
     }
 
     @Override
@@ -57,7 +58,7 @@ public class PacemakerClientHandler extends ChannelInboundHandlerAdapter {
         if (cause instanceof ConnectException) {
             LOG.warn("Connection to pacemaker failed. Trying to reconnect {}", cause.getMessage());
         } else {
-            LOG.error("Exception occurred in Pacemaker.", cause);
+            LOG.error("Exception occurred in Pacemaker: " + cause);
         }
         client.reconnect();
     }
diff --git a/storm-client/src/jvm/org/apache/storm/pacemaker/codec/ThriftNettyClientCodec.java b/storm-client/src/jvm/org/apache/storm/pacemaker/codec/ThriftNettyClientCodec.java
index 8b9b0a2..b208834 100644
--- a/storm-client/src/jvm/org/apache/storm/pacemaker/codec/ThriftNettyClientCodec.java
+++ b/storm-client/src/jvm/org/apache/storm/pacemaker/codec/ThriftNettyClientCodec.java
@@ -72,7 +72,7 @@ public class ThriftNettyClientCodec extends ChannelInitializer<Channel> {
                 throw new RuntimeException(e);
             }
         } else {
-            client.channelReady(ch);
+            // no work for AuthMethod.NONE
         }
 
         pipeline.addLast("PacemakerClientHandler", new PacemakerClientHandler(client));