[
https://issues.apache.org/jira/browse/STORM-348?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14074518#comment-14074518
]
ASF GitHub Bot commented on STORM-348:
--------------------------------------
Github user revans2 commented on a diff in the pull request:
https://github.com/apache/incubator-storm/pull/202#discussion_r15409263
--- Diff:
storm-core/src/jvm/backtype/storm/messaging/netty/SaslStormClientHandler.java
---
@@ -0,0 +1,155 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package backtype.storm.messaging.netty;
+
+import java.io.IOException;
+
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.channel.ChannelStateEvent;
+import org.jboss.netty.channel.Channels;
+import org.jboss.netty.channel.MessageEvent;
+import org.jboss.netty.channel.SimpleChannelUpstreamHandler;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import backtype.storm.Config;
+
+public class SaslStormClientHandler extends SimpleChannelUpstreamHandler {
+
+ private static final Logger LOG = LoggerFactory
+ .getLogger(SaslStormClientHandler.class);
+ private Client client;
+ long start_time;
+ /** Used for client or server's token to send or receive from each
other. */
+ private byte[] token;
+ private String topologyName;
+
+ public SaslStormClientHandler(Client client) throws IOException {
+ this.client = client;
+ start_time = System.currentTimeMillis();
+ getSASLCredentials();
+ }
+
+ @Override
+ public void channelConnected(ChannelHandlerContext ctx,
+ ChannelStateEvent event) {
+ // register the newly established channel
+ Channel channel = ctx.getChannel();
+
+ LOG.info("Connection established from " +
channel.getLocalAddress()
+ + " to " + channel.getRemoteAddress());
+
+ try {
+ SaslNettyClient saslNettyClient =
SaslNettyClientState.getSaslNettyClient
+ .get(channel);
+
+ if (saslNettyClient == null) {
+ LOG.debug("Creating saslNettyClient now " +
"for channel: "
+ + channel);
+ saslNettyClient = new
SaslNettyClient(topologyName, token);
+
SaslNettyClientState.getSaslNettyClient.set(channel,
+ saslNettyClient);
+ }
+
channel.write(ControlMessage.SASL_TOKEN_MESSAGE_REQUEST);
+ } catch (Exception e) {
+ LOG.error("Failed to authenticate with server " + "due
to error: "
+ + e);
+ }
+ return;
+
+ }
+
+ @Override
+ public void messageReceived(ChannelHandlerContext ctx, MessageEvent
event)
+ throws Exception {
+ LOG.debug("send/recv time (ms): {}",
+ (System.currentTimeMillis() - start_time));
+
+ Channel channel = ctx.getChannel();
+
+ // Generate SASL response to server using Channel-local SASL
client.
+ SaslNettyClient saslNettyClient =
SaslNettyClientState.getSaslNettyClient
+ .get(channel);
+ if (saslNettyClient == null) {
+ throw new Exception("saslNettyClient was unexpectedly "
+ + "null for channel: " + channel);
+ }
+
+ // examine the response message from server
+ if (event.getMessage() instanceof ControlMessage) {
+ ControlMessage msg = (ControlMessage)
event.getMessage();
+ if (msg == ControlMessage.SASL_COMPLETE_REQUEST) {
+ LOG.debug("Server has sent us the SaslComplete "
+ + "message. Allowing normal
work to proceed.");
+
+ if (!saslNettyClient.isComplete()) {
+ LOG.error("Server returned a
Sasl-complete message, "
+ + "but as far as we can
tell, we are not authenticated yet.");
+ throw new Exception("Server returned a "
+ + "Sasl-complete
message, but as far as "
+ + "we can tell, we are
not authenticated yet.");
+ }
+ ctx.getPipeline().remove(this);
--- End diff --
I see now that the encoder/decoder are not SASL aware, so the wrap/unwrap
would have to happen at that level, making this a bigger change.
> (Security) Netty SASL Authentication
> ------------------------------------
>
> Key: STORM-348
> URL: https://issues.apache.org/jira/browse/STORM-348
> Project: Apache Storm (Incubating)
> Issue Type: Bug
> Reporter: Robert Joseph Evans
> Assignee: Raghavendra Nandagopal
> Labels: security
> Attachments: Storm-Netty Authentication.pdf
>
>
> Currently The Netty transport does no authentication at all. You can encrypt
> the tuples being sent, but that is a huge performance hit for many cases that
> do not need it. We should support simple SASL authentication when Netty
> first connects to an external process. We probably want to use something
> similar to what we do for ZK, and generate a random secret for each topology.
--
This message was sent by Atlassian JIRA
(v6.2#6252)