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

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

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

    https://github.com/apache/storm/pull/838#discussion_r45512711
  
    --- Diff: 
storm-core/src/jvm/backtype/storm/messaging/netty/KerberosSaslServerHandler.java
 ---
    @@ -0,0 +1,131 @@
    +/**
    + * 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 java.util.List;
    +import java.util.Map;
    +import org.jboss.netty.channel.Channel;
    +import org.jboss.netty.channel.ChannelHandlerContext;
    +import org.jboss.netty.channel.Channels;
    +import org.jboss.netty.channel.ExceptionEvent;
    +import org.jboss.netty.channel.MessageEvent;
    +import org.jboss.netty.channel.SimpleChannelUpstreamHandler;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class KerberosSaslServerHandler extends 
SimpleChannelUpstreamHandler {
    +
    +    ISaslServer server;
    +    /** Used for client or server's token to send or receive from each 
other. */
    +    private Map storm_conf;
    +    private String jaas_section;
    +    private List<String> authorizedUsers;
    +
    +    private static final Logger LOG = LoggerFactory
    +        .getLogger(KerberosSaslServerHandler.class);
    +
    +    public KerberosSaslServerHandler(ISaslServer server, Map storm_conf, 
String jaas_section, List<String> authorizedUsers) throws IOException {
    +        this.server = server;
    +        this.storm_conf = storm_conf;
    +        this.jaas_section = jaas_section;
    +        this.authorizedUsers = authorizedUsers;
    +    }
    +
    +    @Override
    +    public void messageReceived(ChannelHandlerContext ctx, MessageEvent e)
    +        throws Exception {
    +        Object msg = e.getMessage();
    +        if (msg == null)
    +            return;
    +
    +        Channel channel = ctx.getChannel();
    +
    +
    +        if (msg instanceof SaslMessageToken) {
    +            // initialize server-side SASL functionality, if we haven't yet
    +            // (in which case we are looking at the first SASL message 
from the
    +            // client).
    +            try {
    +                LOG.debug("Got SaslMessageToken!");
    +
    +                KerberosSaslNettyServer saslNettyServer = 
KerberosSaslNettyServerState.getKerberosSaslNettyServer
    +                    .get(channel);
    +                if (saslNettyServer == null) {
    +                    LOG.debug("No saslNettyServer for {}  yet; creating 
now, with topology token: ", channel);
    +                    try {
    +                        saslNettyServer = new 
KerberosSaslNettyServer(storm_conf, jaas_section, authorizedUsers);
    +                    } catch (RuntimeException ioe) {
    +                        LOG.error("Error occurred while creating 
saslNettyServer on server {} for client {}",
    +                                  channel.getLocalAddress(), 
channel.getRemoteAddress());
    +                        saslNettyServer = null;
    +                    }
    +
    +                    
KerberosSaslNettyServerState.getKerberosSaslNettyServer.set(channel,
    +                                                                           
     saslNettyServer);
    +                } else {
    +                    LOG.debug("Found existing saslNettyServer on server: 
{} for client {}",
    +                              channel.getLocalAddress(), 
channel.getRemoteAddress());
    +                }
    +
    +                byte[] responseBytes = 
saslNettyServer.response(((SaslMessageToken) msg)
    +                                                                
.getSaslToken());
    --- End diff --
    
    If we catch the RTE above, we set `saslNettyServer = null` (even though it 
was already), and so at this line we would get a NullPointerException.


> Heartbeat Server (Pacemaker)
> ----------------------------
>
>                 Key: STORM-885
>                 URL: https://issues.apache.org/jira/browse/STORM-885
>             Project: Apache Storm
>          Issue Type: Improvement
>          Components: storm-core
>            Reporter: Robert Joseph Evans
>            Assignee: Kyle Nusbaum
>
> Large highly connected topologies and large clusters write a lot of data into 
> ZooKeeper.  The heartbeats, that make up the majority of this data, do not 
> need to be persisted to disk.  Pacemaker is intended to be a secure 
> replacement for storing the heartbeats without changing anything within the 
> heartbeats.  In the future as more metrics are added in, we may want to look 
> into switching it over to look more like Heron, where a metrics server is 
> running for each node/topology.  And can be used to aggregate/per-aggregate 
> them in a more scalable manor.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to