Merge pull request #3169 from agresch/agresch_pacemaker_client
STORM-3540 fix Pacemaker connection issues
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 @@
Channel channel = ctx.channel();
LOG.info("Connection established from {} to {}",
channel.localAddress(), channel.remoteAddress());
+ client.channelReady(channel);
}
@Override
@@ -57,7 +58,7 @@
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 @@
throw new RuntimeException(e);
}
} else {
- client.channelReady(ch);
+ // no work for AuthMethod.NONE
}
pipeline.addLast("PacemakerClientHandler", new PacemakerClientHandler(client));