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

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

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

    https://github.com/apache/storm/pull/1081#discussion_r53369187
  
    --- Diff: 
storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java ---
    @@ -0,0 +1,687 @@
    +/**
    + * 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 org.apache.storm.cluster;
    +
    +import clojure.lang.*;
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.curator.framework.state.*;
    +import org.apache.curator.framework.state.ConnectionState;
    +import org.apache.storm.callback.ZKStateChangedCallback;
    +import org.apache.storm.generated.*;
    +import org.apache.storm.nimbus.NimbusInfo;
    +import org.apache.storm.utils.Time;
    +import org.apache.storm.utils.Utils;
    +import org.apache.storm.zookeeper.Zookeeper;
    +import org.apache.zookeeper.KeeperException;
    +import org.apache.zookeeper.Watcher;
    +import org.apache.zookeeper.data.ACL;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.PrintWriter;
    +import java.io.StringWriter;
    +import java.security.NoSuchAlgorithmException;
    +import java.util.*;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +public class StormClusterStateImpl implements IStormClusterState {
    +
    +    private static Logger LOG = 
LoggerFactory.getLogger(StormClusterStateImpl.class);
    +
    +    private IStateStorage stateStorage;
    +
    +    private ConcurrentHashMap<String, IFn> assignmentInfoCallback;
    +    private ConcurrentHashMap<String, IFn> 
assignmentInfoWithVersionCallback;
    +    private ConcurrentHashMap<String, IFn> assignmentVersionCallback;
    +    private AtomicReference<IFn> supervisorsCallback;
    +    // we want to reigister a topo directory getChildren callback for all 
workers of this dir
    +    private ConcurrentHashMap<String, IFn> backPressureCallback;
    +    private AtomicReference<IFn> assignmentsCallback;
    +    private ConcurrentHashMap<String, IFn> stormBaseCallback;
    +    private AtomicReference<IFn> blobstoreCallback;
    +    private ConcurrentHashMap<String, IFn> credentialsCallback;
    +    private ConcurrentHashMap<String, IFn> logConfigCallback;
    +
    +    private List<ACL> acls;
    +    private String stateId;
    +    private boolean solo;
    +
    +    public StormClusterStateImpl(IStateStorage StateStorage, List<ACL> 
acls, ClusterStateContext context, boolean solo) throws Exception {
    +
    +        this.stateStorage = StateStorage;
    +        this.solo = solo;
    +        this.acls = acls;
    +
    +        assignmentInfoCallback = new ConcurrentHashMap<>();
    +        assignmentInfoWithVersionCallback = new ConcurrentHashMap<>();
    +        assignmentVersionCallback = new ConcurrentHashMap<>();
    +        supervisorsCallback = new AtomicReference<>();
    +        backPressureCallback = new ConcurrentHashMap<>();
    +        assignmentsCallback = new AtomicReference<>();
    +        stormBaseCallback = new ConcurrentHashMap<>();
    +        credentialsCallback = new ConcurrentHashMap<>();
    +        logConfigCallback = new ConcurrentHashMap<>();
    +        blobstoreCallback = new AtomicReference<>();
    +
    +        stateId = this.stateStorage.register(new ZKStateChangedCallback() {
    +
    +            public void changed(Watcher.Event.EventType type, String path) 
{
    +                List<String> toks = Zookeeper.tokenizePath(path);
    +                int size = toks.size();
    +                if (size >= 1) {
    +                    String root = toks.get(0);
    +                    if (root.equals(ClusterUtils.ASSIGNMENTS_ROOT)) {
    +                        if (size == 1) {
    +                            // set null and get the old value
    +                            issueCallback(assignmentsCallback);
    +                        } else {
    +                            issueMapCallback(assignmentInfoCallback, 
toks.get(1));
    +                            issueMapCallback(assignmentVersionCallback, 
toks.get(1));
    +                            
issueMapCallback(assignmentInfoWithVersionCallback, toks.get(1));
    +                        }
    +
    +                    } else if (root.equals(ClusterUtils.SUPERVISORS_ROOT)) 
{
    +                        issueCallback(supervisorsCallback);
    +                    } else if (root.equals(ClusterUtils.BLOBSTORE_ROOT)) {
    +                        issueCallback(blobstoreCallback);
    +                    } else if (root.equals(ClusterUtils.STORMS_ROOT) && 
size > 1) {
    +                        issueMapCallback(stormBaseCallback, toks.get(1));
    +                    } else if (root.equals(ClusterUtils.CREDENTIALS_ROOT) 
&& size > 1) {
    +                        issueMapCallback(credentialsCallback, toks.get(1));
    +                    } else if (root.equals(ClusterUtils.LOGCONFIG_ROOT) && 
size > 1) {
    +                        issueMapCallback(logConfigCallback, toks.get(1));
    +                    } else if (root.equals(ClusterUtils.BACKPRESSURE_ROOT) 
&& size > 1) {
    +                        issueMapCallback(logConfigCallback, toks.get(1));
    +                    } else {
    +                        LOG.error("{} Unknown callback for subtree {}", 
new RuntimeException("Unknown callback for this path"), path);
    +                        Runtime.getRuntime().exit(30);
    +                    }
    +
    +                }
    +
    +                return;
    +            }
    +
    +        });
    +
    +        String[] pathlist = { ClusterUtils.ASSIGNMENTS_SUBTREE, 
ClusterUtils.STORMS_SUBTREE, ClusterUtils.SUPERVISORS_SUBTREE, 
ClusterUtils.WORKERBEATS_SUBTREE,
    +                ClusterUtils.ERRORS_SUBTREE, 
ClusterUtils.BLOBSTORE_SUBTREE, ClusterUtils.NIMBUSES_SUBTREE, 
ClusterUtils.LOGCONFIG_SUBTREE };
    +        for (String path : pathlist) {
    +            this.stateStorage.mkdirs(path, acls);
    +        }
    +
    +    }
    +
    +    protected void issueCallback(AtomicReference<IFn> cb) {
    +        IFn callback = cb.getAndSet(null);
    +        if (callback != null)
    +            callback.invoke();
    --- End diff --
    
    Anything that calls invoke with no parameters should be a Runnable not an 
IFn.


> port backtype.storm.cluster to java
> -----------------------------------
>
>                 Key: STORM-1273
>                 URL: https://issues.apache.org/jira/browse/STORM-1273
>             Project: Apache Storm
>          Issue Type: New Feature
>          Components: storm-core
>            Reporter: Robert Joseph Evans
>            Assignee: John Fang
>              Labels: java-migration, jstorm-merger
>
> current state of the cluster (Some of this moves to java as a part of 
> heartbeat server)
> https://github.com/apache/storm/tree/jstorm-import/jstorm-core/src/main/java/com/alibaba/jstorm/cluster
>  as an example



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

Reply via email to