You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@storm.apache.org by danny0405 <gi...@git.apache.org> on 2017/11/22 11:08:51 UTC

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

GitHub user danny0405 opened a pull request:

    https://github.com/apache/storm/pull/2433

    [STORM-2693] Heartbeats and assignments promotion for storm2.0

    For large cluster support, good scheduling performance for scheduling and assignments distribution.
    
    For nimbus heartbeats pressure test:
    heartbeats response time:
    ![e4a1e216-0a08-4e08-90c0-334e00db962c](https://user-images.githubusercontent.com/7644508/33124588-7f5aa776-cfb8-11e7-883c-4f17001052dd.png)
    
    For topology submission time cost:
    topology scheduling time:
    
    cluster slots | newly submit slots | after submission slots | time cost milliseconds
    -- | -- | -- | --
    3700 | 100 | 3800 | 1181ms
    3600 | 100 | 3700 | 886ms
    3500 | 100 | 3600 | 925ms
    3400 | 100 | 3500 | 1520ms
    3300 | 100 | 3400 | 930ms
    2000 | 100 | 2100 |  
    1500 | 100 | 1600 |  
    1000 | 100 | 1100 |  
    500 | 100 | 600 |  
    100 | 100 | 200 |  
    
    This is PR for 1.1.x-branch of heartbeats promotion
    #2389
    
    This is PR for 1.1.x-branch of assignments promotion
    #2319

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/danny0405/storm heartbeats-promotion-v2

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/storm/pull/2433.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #2433
    
----
commit 3f58b1fcdbee39798a8fe49a2d32f07e3ed05640
Author: chenyuzhao <ch...@meituan.com>
Date:   2017-11-21T09:54:23Z

    heartbeats and assignments promotion for storm2.0

----


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @danny0405 
    Could you rebase?
    
    And I'm sorry but I'd like to ask a favor of including option for compatibility mode for supporting Storm 1.x and 0.10.x. We have applied [STORM-2448](https://issues.apache.org/jira/browse/STORM-2448) which guarantees Storm 2.0.0 daemons can interact Storm 1.x and 0.10.x workers. I have initialized discussion to revisit that issue, but we may still want to have STORM-2448, so we need to have a compatibility mode for now.
    
    Could we let Nimbus also reads the ZK heartbeat like before when the option is turned on?


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152902376
  
    --- Diff: storm-server/src/main/java/org/apache/storm/nimbus/TimeOutWorkerHeartbeatsRecoveryStrategy.java ---
    @@ -0,0 +1,86 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.nimbus;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.utils.ObjectReader;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.HashSet;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * Wait for a node to report worker heartbeats until a configured timeout. For cases below we have strategies:
    + * <p>
    + * 1: When nimbus gains leader ship, it will decide if the heartbeats are ready based on the reported node ids,
    + * supervisors/nodes will take care of the worker heartbeats recovery, a reported node id means all the workers heartbeats on the node are reported.
    + * <p>
    + * 2: If several supervisor also crush and will never recover[or all crush for some unknown reason],
    + * workers will report their heartbeats directly to master, so it has not any effect.
    + */
    +public class TimeOutWorkerHeartbeatsRecoveryStrategy implements IWorkerHeartbeatsRecoveryStrategy {
    +    private static final Logger LOG = LoggerFactory.getLogger(TimeOutWorkerHeartbeatsRecoveryStrategy.class);
    +
    +    private static int NODE_MAX_TIMEOUT_SECS = 600;
    +
    +    private long startTimeSecs;
    +
    +    private Set<String> reportedIDs;
    +
    +    @Override
    +    public void prepare(Map conf) {
    +        NODE_MAX_TIMEOUT_SECS = ObjectReader.getInt(conf.get(Config.SUPERVISOR_WORKER_HEARTBEATS_MAX_TIMEOUT_SECS), 600);
    +        this.startTimeSecs = System.currentTimeMillis() / 1000L;
    +        this.reportedIDs = new HashSet<>();
    +    }
    +
    +    @Override
    +    public boolean isReady(Set<String> nodeIds) {
    +        if (isMaxTimeOut()) {
    +            HashSet<String> tmp = new HashSet<>();
    +            for(String nodeID : nodeIds) {
    +                if (!this.reportedIDs.contains(nodeID))
    +                tmp.add(nodeID);
    +            }
    +            LOG.warn("Failed to recover heartbeats for nodes: {} with timeout {}s", tmp, NODE_MAX_TIMEOUT_SECS);
    +            return true;
    +        }
    +        for (String nodeID : nodeIds) {
    --- End diff --
    
    add empty line to separate logic


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r158688366
  
    --- Diff: storm-server/src/main/java/org/apache/storm/daemon/supervisor/ReadClusterState.java ---
    @@ -69,10 +65,8 @@
         public ReadClusterState(Supervisor supervisor) throws Exception {
             this.superConf = supervisor.getConf();
             this.stormClusterState = supervisor.getStormClusterState();
    -        this.syncSupEventManager = supervisor.getEventManger();
    -        this.assignmentVersions = new AtomicReference<>(new HashMap<>());
             this.assignmentId = supervisor.getAssignmentId();
    -        this.iSuper = supervisor.getiSupervisor();
    +        this.iSupervisor = supervisor.getiSupervisor();
    --- End diff --
    
    done


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @HeartSaVioR
    I have applied you suggestions and it worked as desired, thx very much.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152716882
  
    --- Diff: storm-client/src/jvm/org/apache/storm/grouping/LoadAwareShuffleGrouping.java ---
    @@ -291,7 +291,10 @@ private Scope calculateScope(Map<Integer, NodeInfo> taskToNodePort, Map<String,
         private Map<String, String> getHostToRackMapping(Map<Integer, NodeInfo> taskToNodePort) {
             Set<String> hosts = new HashSet();
             for (int task: targetTasks) {
    -            hosts.add(taskToNodePort.get(task).get_node());
    +            //if is a kill, taskToNodePort will be an empty map which is refreshed by WorkerState
    --- End diff --
    
    i means that the taskToNodePort cache is refreshed by worker as a timer task, if we kill the worker/or rebalance, the local task NodePort maybe a null, so we should have a check here, or it may throw a NPE


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @HeartSaVioR 
    
    Perhaps we should have a call on the Metrics V2 work and what is its status for storm 2.0 because I don't totally know myself.
    
    What I think is mandatory for this patch is.
    
    1) wait for delegation token work to go in so we can authenticate connections between the worker and nimbus/supervisors.
    2) Add in some form of authorization to the newly added APIs. 
       - getSupervisorAssignments should only be allowed from a supervisor
       - sendSupervisorWorkerHeartbeats should only be allowed from a supervisor.
       - sendSupervisorWorkerHeartbeat should verify that it has come from the owner of the topology the heartbeat is for
       - sendSupervisorAssignments needs to verify that it came from nimbus.
       - getLocalAssignmentForStorm needs to verify that it came from the owner of the topology.
       - sendSupervisorWorkerHeartbeat needs to verify that it came from the owner of the topology.
    3) supervisor needs to pick a port from a configured allowable range of ports and get that information to everyone who is going to need it.
    
    If someone wants to drop the old heartbeat mechanisms for talking to nimbus and the supervisors that is fine with me.  However, if we do drop it I really would love to have a way to maintain backwards compatibility because otherwise I will have to add it back in myself.
    
    If we don't drop it now I would like to see a follow on JIRA to remove it, but for that to work as part of a rolling upgrade we would need to support both mechanisms at the same time.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152903549
  
    --- Diff: storm-server/src/main/java/org/apache/storm/nimbus/LeaderListenerCallback.java ---
    @@ -0,0 +1,213 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.nimbus;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.collect.Sets;
    +import org.apache.commons.io.IOUtils;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.curator.framework.recipes.leader.LeaderLatch;
    +import org.apache.storm.Config;
    +import org.apache.storm.blobstore.BlobStore;
    +import org.apache.storm.blobstore.InputStreamWithMeta;
    +import org.apache.storm.cluster.ClusterUtils;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.daemon.nimbus.TopoCache;
    +import org.apache.storm.generated.AuthorizationException;
    +import org.apache.storm.generated.KeyNotFoundException;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.security.auth.ReqContext;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.Utils;
    +import org.apache.storm.zookeeper.ClientZookeeper;
    +import org.apache.zookeeper.CreateMode;
    +import org.apache.zookeeper.data.ACL;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.io.IOException;
    +import java.util.*;
    +
    +/**
    + * A callback function when nimbus gains leadership
    + */
    +public class LeaderListenerCallback {
    +    private static final Logger LOG = LoggerFactory.getLogger(LeaderListenerCallback.class);
    +
    +    private final BlobStore blobStore;
    +    private final TopoCache tc;
    +    private final IStormClusterState clusterState;
    +
    +    private final CuratorFramework zk;
    +    private final LeaderLatch leaderLatch;
    +
    +    private final Map conf;
    +    private final List<ACL> acls;
    +
    +    private static final String STORM_JAR_SUFFIX = "-stormjar.jar";
    +    private static final String STORM_CODE_SUFFIX = "-stormcode.ser";
    +    private static final String STORM_CONF_SUFFIX = "-stormconf.ser";
    +
    +    public LeaderListenerCallback(Map conf, CuratorFramework zk, LeaderLatch leaderLatch, BlobStore blobStore, TopoCache tc, IStormClusterState clusterState, List<ACL> acls) {
    +        this.blobStore = blobStore;
    +        this.tc = tc;
    +        this.clusterState = clusterState;
    +        this.zk = zk;
    +        this.leaderLatch = leaderLatch;
    +        this.conf = conf;
    +        this.acls = acls;
    +    }
    +
    +    /**
    +     * Invoke when gains leadership.
    +     */
    +    public void leaderCallBack() {
    +        //in local mode, only one leader exist
    +        if (ConfigUtils.isLocalMode(conf)) {
    +            return;
    +        }
    +        //set up nimbus-info to zk
    +        setUpNimbusInfo(acls);
    +        //sync zk assignments/id-info to local
    +        LOG.info("Sync remote assignments and id-info to local");
    +        clusterState.syncRemoteAssignments(null);
    +        clusterState.syncRemoteIds(null);
    +
    +        Set<String> activeTopologyIds = new TreeSet<>(ClientZookeeper.getChildren(zk,
    +                conf.get(Config.STORM_ZOOKEEPER_ROOT) + ClusterUtils.STORMS_SUBTREE, false));
    +
    +        Set<String> activeTopologyBlobKeys = populateTopologyBlobKeys(activeTopologyIds);
    +        Set<String> activeTopologyCodeKeys = filterTopologyCodeKeys(activeTopologyBlobKeys);
    +        Set<String> allLocalBlobKeys = Sets.newHashSet(blobStore.listKeys());
    +        Set<String> allLocalTopologyBlobKeys = filterTopologyBlobKeys(allLocalBlobKeys);
    +
    +        // this finds all active topologies blob keys from all local topology blob keys
    +        Sets.SetView<String> diffTopology = Sets.difference(activeTopologyBlobKeys, allLocalTopologyBlobKeys);
    +        LOG.info("active-topology-blobs [{}] local-topology-blobs [{}] diff-topology-blobs [{}]",
    +                generateJoinedString(activeTopologyIds), generateJoinedString(allLocalTopologyBlobKeys),
    +                generateJoinedString(diffTopology));
    +
    +        if (diffTopology.isEmpty()) {
    +            Set<String> activeTopologyDependencies = getTopologyDependencyKeys(activeTopologyCodeKeys);
    +
    +            // this finds all dependency blob keys from active topologies from all local blob keys
    +            Sets.SetView<String> diffDependencies = Sets.difference(activeTopologyDependencies, allLocalBlobKeys);
    +            LOG.info("active-topology-dependencies [{}] local-blobs [{}] diff-topology-dependencies [{}]",
    +                    generateJoinedString(activeTopologyDependencies), generateJoinedString(allLocalBlobKeys),
    +                    generateJoinedString(diffDependencies));
    +
    +            if (diffDependencies.isEmpty()) {
    +                LOG.info("Accepting leadership, all active topologies and corresponding dependencies found locally.");
    +                tc.clear();
    +            } else {
    +                LOG.info("Code for all active topologies is available locally, but some dependencies are not found locally, "
    +                        + "giving up leadership.");
    +                closeLatch();
    +            }
    +        } else {
    +            LOG.info("code for all active topologies not available locally, giving up leadership.");
    +            closeLatch();
    +        }
    +    }
    +
    +    /**
    +     * Invoke when lost leadership.
    +     */
    +    public void notLeaderCallback() {
    --- End diff --
    
    Oh there's another place logging this. Please ignore this.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @danny0405 
    We have SUPERVISOR_WORKER_VERSION_CLASSPATH_MAP in Config, which is a map representing supported versions in supervisor. Assuming all the nodes have consistent configuration, Nimbus could check the value of map while initiating and determine whether the cluster supports Storm versions under 2.0.0.
    
    https://github.com/apache/storm/blob/466a7ad74da27c1250eedf412a487db409e42c19/storm-client/src/jvm/org/apache/storm/Config.java#L1529-L1551
    
    And updated topology structure has Storm version information:
    
    https://github.com/apache/storm/blob/466a7ad74da27c1250eedf412a487db409e42c19/storm-client/src/storm.thrift#L265-L283
    
    we could treat `no value` as same as cluster's version.
    
    So if Nimbus could determine topology's Storm version and apply its heartbeat behavior based on the version, we could read heartbeat from ZK for topology version under Storm 2.0.0, and we could treat heartbeats only from RPC for topology version same or higher than Storm 2.0.0. Not sure how much the work would be needed, but with this approach we don't even need to concern with SUPERVISOR_WORKER_VERSION_CLASSPATH_MAP.
    



---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152695062
  
    --- Diff: storm-client/src/jvm/org/apache/storm/assignments/LocalAssignmentsBackendFactory.java ---
    @@ -0,0 +1,48 @@
    +/**
    + * 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.assignments;
    +
    +import com.google.common.base.Preconditions;
    +import org.apache.storm.Config;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.ReflectionUtils;
    +
    +import java.util.Map;
    +
    +/**
    + * Factory class for creating local assignments.
    + */
    +public class LocalAssignmentsBackendFactory {
    +
    +    public static ILocalAssignmentsBackend getBackend(Map conf) {
    --- End diff --
    
    Please expose generic types for the `conf`. And if `conf` is storm conf map, why not passing it to getDefault() for reusing?


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152703493
  
    --- Diff: storm-client/src/jvm/org/apache/storm/daemon/worker/WorkerState.java ---
    @@ -647,6 +625,22 @@ public static boolean isConnectionReady(IConnection connection) {
             return executorsAssignedToThisWorker;
         }
     
    +    private Assignment getLocalAssignment(Map<String, Object> conf, IStormClusterState stormClusterState, String topologyId) {
    +        if (!ConfigUtils.isLocalMode(conf)) {
    +            try{
    --- End diff --
    
    space between y and {


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152716336
  
    --- Diff: storm-client/src/jvm/org/apache/storm/assignments/ILocalAssignmentsBackend.java ---
    @@ -0,0 +1,103 @@
    +/**
    + * 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.assignments;
    +
    +import org.apache.storm.generated.Assignment;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +/**
    + * Interface for storing local assignments.
    + */
    +public interface ILocalAssignmentsBackend {
    +
    +    /**
    +     * initial function for creating backend.
    +     * @param conf
    +     */
    +    void prepare(Map conf);
    +
    +    /**
    +     * Keep a storm assignment to local state or update old assignment.
    +     * @param stormID storm runtime id
    +     * @param assignment assignment as thrift
    +     */
    +    void keepOrUpdateAssignment(String stormID, Assignment assignment);
    +
    +    /**
    +     * Get assignment as byte[] for a storm
    +     * @param stormID storm runtime id
    +     * @return
    +     */
    +    Assignment getAssignment(String stormID);
    +
    +    void removeAssignment(String stormID);
    +
    +    /**
    +     * List all the storm runtime ids of local assignments
    --- End diff --
    
    okey, i will fix the style issue


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152702076
  
    --- Diff: storm-client/src/jvm/org/apache/storm/assignments/InMemoryAssignmentBackend.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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.assignments;
    +
    +import org.apache.storm.cluster.ClusterUtils;
    +import org.apache.storm.generated.Assignment;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +/**
    + * An assignment backend which will keep all assignments and id-info in memory. Only used if no backend is specified internal.
    + */
    +public class InMemoryAssignmentBackend implements ILocalAssignmentsBackend {
    +    private static final Logger LOG = LoggerFactory.getLogger(InMemoryAssignmentBackend.class);
    +
    +    protected Map<String, Assignment> idToAssignment;
    +    protected Map<String, String> idToName;
    +    protected Map<String, String> nameToId;
    +    /**
    +     * Used for assignments set/get, assignments set/get should be kept thread safe
    +     */
    +    private final Object assignmentsLock = new Object();
    +
    +    public InMemoryAssignmentBackend() {}
    +
    +    @Override
    +    public void prepare(Map conf) {
    +        // do nothing for conf now
    +        this.idToAssignment = new HashMap<>();
    +        this.idToName = new HashMap<>();
    +        this.nameToId = new HashMap<>();
    +    }
    +
    +    @Override
    +    public void keepOrUpdateAssignment(String stormID, Assignment assignment) {
    +        synchronized (assignmentsLock) {
    +            this.idToAssignment.put(stormID, assignment);
    +        }
    +    }
    +
    +    @Override
    +    public Assignment getAssignment(String stormID) {
    +        synchronized (assignmentsLock) {
    +            return this.idToAssignment.get(stormID);
    +        }
    +    }
    +
    +    @Override
    +    public void removeAssignment(String stormID) {
    +        synchronized (assignmentsLock) {
    +            this.idToAssignment.remove(stormID);
    +        }
    +    }
    +
    +    @Override
    +    public List<String> assignments() {
    +        if(idToAssignment == null) {
    +            return new ArrayList<>();
    +        }
    +        List<String> ret = new ArrayList<>();
    +        synchronized (assignmentsLock) {
    +            ret.addAll(this.idToAssignment.keySet());
    +            return ret;
    +        }
    +    }
    +
    +    @Override
    +    public Map<String, Assignment> assignmentsInfo() {
    +        Map<String, Assignment> ret = new HashMap<>();
    +        synchronized (assignmentsLock) {
    +            ret.putAll(this.idToAssignment);
    +        }
    +
    +        return ret;
    +    }
    +
    +    @Override
    +    public void syncRemoteAssignments(Map<String, byte[]> remote) {
    +        Map<String, Assignment> tmp = new HashMap<>();
    +        for(Map.Entry<String, byte[]> entry: remote.entrySet()) {
    +            tmp.put(entry.getKey(), ClusterUtils.maybeDeserialize(entry.getValue(), Assignment.class));
    +        }
    +        this.idToAssignment = tmp;
    +    }
    +
    +    @Override
    +    public void keepStormId(String stormName, String stormID) {
    +        this.nameToId.put(stormName, stormID);
    +        this.idToName.put(stormID, stormName);
    +    }
    +
    +    @Override
    +    public String getStormId(String stormName) {
    +        return this.nameToId.get(stormName);
    +    }
    +
    +    @Override
    +    public void syncRemoteIDS(Map<String, String> remote) {
    --- End diff --
    
    Same here: syncRemoteIDS -> syncRemoteIds


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152903015
  
    --- Diff: storm-server/src/main/java/org/apache/storm/daemon/supervisor/timer/SynchronizeAssignments.java ---
    @@ -0,0 +1,133 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.daemon.supervisor.timer;
    +
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.daemon.supervisor.ReadClusterState;
    +import org.apache.storm.daemon.supervisor.Supervisor;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.SupervisorAssignments;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.NimbusClient;
    +import org.apache.storm.utils.Utils;
    +import org.apache.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.HashMap;
    +import java.util.Map;
    +
    +/**
    + * A runnable which will synchronize assignments to node local and then worker processes.
    + */
    +public class SynchronizeAssignments implements Runnable {
    +    private static final Logger LOG = LoggerFactory.getLogger(SynchronizeAssignments.class);
    +
    +    private Supervisor supervisor;
    +    private SupervisorAssignments assignments;
    +    private ReadClusterState readClusterState;
    +
    +    public SynchronizeAssignments(Supervisor supervisor, SupervisorAssignments assignments, ReadClusterState readClusterState) {
    +        this.supervisor = supervisor;
    +        this.assignments = assignments;
    +        this.readClusterState = readClusterState;
    +    }
    +
    +    @Override
    +    public void run() {
    +        // first sync assignments to local, then sync processes.
    +        if (null == assignments) {
    +            getAssignmentsFromMaster(this.supervisor.getConf(), this.supervisor.getStormClusterState(), this.supervisor.getAssignmentId());
    +        } else {
    +            assignedAssignmentsToLocal(this.supervisor.getStormClusterState(), assignments);
    +        }
    +        this.readClusterState.run();
    +    }
    +
    +    /**
    +     * Used by {@link Supervisor} to fetch assignments when start up.
    +     * @param supervisor
    +     */
    +    public void getAssignmentsFromMasterUntilSuccess(Supervisor supervisor) {
    +        boolean success = false;
    +        NimbusClient master;
    +        while (!success) {
    +            try {
    +                master = NimbusClient.getConfiguredClient(supervisor.getConf());
    +                SupervisorAssignments assignments = master.getClient().getSupervisorAssignments(supervisor.getAssignmentId());
    +                assignedAssignmentsToLocal(supervisor.getStormClusterState(), assignments);
    +                success = true;
    +                try {
    +                    master.close();
    +                } catch (Throwable t) {
    +                    LOG.warn("Close master client exception", t);
    +                }
    +            } catch (Exception t) {
    +                // just ignore the exception
    +            }
    +            if (!success) {
    +                LOG.info("Waiting for a success sync of assignments from master...");
    +                try {
    +                    Thread.sleep(5000l);
    --- End diff --
    
    Let's use `Time.sleep()`.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152704003
  
    --- Diff: storm-client/src/jvm/org/apache/storm/stats/StatsUtil.java ---
    @@ -1360,11 +1329,11 @@ public static ComponentPageInfo aggCompExecsStats(
          * Convenience overload when called from the topology page code (in that case we want data
          * for all workers in the topology, not filtered by supervisor)
          *
    -     * @param topologyId       topology id
    -     * @param topology         storm topology
    -     * @param task2component   a Map of {task id -> component}
    +     * @param stormId       topology id
    --- End diff --
    
    Nice catch!


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r177594095
  
    --- Diff: storm-client/src/jvm/org/apache/storm/assignments/ILocalAssignmentsBackend.java ---
    @@ -26,7 +26,7 @@
     /**
      * Interface for storing local assignments.
      */
    -public interface ILocalAssignmentsBackend {
    +public interface ILocalAssignmentsBackend extends AutoCloseable{
    --- End diff --
    
    nit: space between `e` and `{`


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @HeartSaVioR I think I must have mixed this patch up with another patch so yes I am wrong if this goes in backwards compatibility is in trouble.
    
    @danny0405 
    I have been looking through the code and this is a rather large change, could you explain to me at a high level how the architecture is changing?  I see that there is a SupervisorClient now. 
    
    It looks like the worker tries to talk to the supervisor through this client sending heartbeats and asking for its current assignment.  Also the worker can fall back to talking to nimbus for the heartbeats.
    
    I also see that nimbus will use it to try and send out new assignments to the Supervisors as well.
    
    Why do we heartbeat to the supervisor in 2 different ways?  We still go through the local file system, but also through thrift with a fallback to nimbus if it fails.  The implementation of this thrift heartbeat handler in the Supervisor is a noop.  Is the plan to eventually stop going through the file system?
    
    My biggest concern is around security.  In the current security model the workers have no way to communicate with Nimbus, or for that matter the new Supervisor Thrift Server.  Nimbus/Supervisor only supports Kerberos authentication and topologies are not guaranteed to have kerberos credentials available to them.  For Zookeeper we use a username/password that is randomly generated per topology to protect access.  When going through the file system to the supervisor we explicitly setup the file system permissions to only allow the owner of the topology + the supervisor user to be able to read and write to the heartbeat directory.  This does none of those.  The new Thrift APIs do nothing to restrict who has access to the them, so anyone can download assignments and anyone can heartbeat in for anyone else.  I can think of a few malicious things that an attacker could do with this power. Even if we do have proper authorization if I turn on Kerberos authentication those communication c
 hannels are now shut down to the worker because it cannot authenticate through Kerberos and hence cannot get past SASL to do anything.
    
    I am also concerned about Storm on Mesos with this patch.  We have broken them a lot of times because we don't think about them enough, and this is going to be another one of those cases.  Each supervisor must be on a given port for the cluster to work in this setup.  On Mesos and probably on Yarn too, there is the possibility of multiple supervisors running on a single physical host.  Storm on mesos already jumps through all kinds of hoops to make this work by resetting the directories where the supervisors store things to avoid any collisions. With a port that cannot change because nimbus does not support talking to it on an ephemeral port I don't see a way to make 2 supervisors happy on the same box.
    
    Now please don't think I am beating on this patch because I would much rather have pacemaker.  I think the overall direction that this patch is going in is much much cleaner than what we are doing with pacemaker/zookeeper.  There is just potentially a lot of work to really get this working in a secure and compatible way.  We would have to support something like hadoop delegation tokens in storm so the workers could talk to nimbus and we could guarantee in all cases that they would be there an up to date.  We would also have to have a way for the supervisors to support something similar.
    
    But I would encourage you to take another look at pacemaker in the short-term, at least until we have all of these issues worked out.  We currently have clusters running using pacemaker with 900 supervisors and over 120 topologies in a secure and stable way.  We did run into stability issues with pacemaker at the beginning, but we believe that we have fixed all of them.  If you are having crashes please let us know and we will do our best to fix them.  We also are currently experiencing issues with the time it takes to download heartbeats.  We have a plan to parallelize it, but just have not put it in place yet.
    
    Another thing to think about is that we are likely going to be able to remove the metrics from the ZK heartbeats once we have an alternative path for them.  This should let us reduce the load on ZK/pacemaker massively and let us scale even better.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r172934007
  
    --- Diff: conf/defaults.yaml ---
    @@ -182,7 +195,7 @@ worker.log.level.reset.poll.secs: 30
     # control how many worker receiver threads we need per worker
     topology.worker.receiver.thread.count: 1
     
    -task.heartbeat.frequency.secs: 3
    +task.heartbeat.frequency.secs: 60
    --- End diff --
    
    This is a bit scary.  Why is the default now 20 times bigger than it was before?  If this is just for scale we should drop it back to the default and write up documentation on how to scale a cluster much larger.  Even in that case I don't feel comfortable with waiting 10+ mins for a worker to time out in the worst case.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152704189
  
    --- Diff: storm-client/src/jvm/org/apache/storm/stats/StatsUtil.java ---
    @@ -1547,26 +1533,21 @@ public static ComponentPageInfo aggCompExecsStats(
          * @param timeout       timeout
          * @return a HashMap of updated executor heart beats
          */
    -    public static Map<List<Integer>, Map<String, Object>> updateHeartbeatCache(Map<List<Integer>, Map<String, Object>> cache,
    +    public static void updateHeartbeatCache(Map<List<Integer>, Map<String, Object>> cache,
    --- End diff --
    
    We're removing the return, so we need to remove it from javadoc too.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152902340
  
    --- Diff: storm-server/src/main/java/org/apache/storm/nimbus/TimeOutWorkerHeartbeatsRecoveryStrategy.java ---
    @@ -0,0 +1,86 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.nimbus;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.utils.ObjectReader;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.HashSet;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * Wait for a node to report worker heartbeats until a configured timeout. For cases below we have strategies:
    + * <p>
    + * 1: When nimbus gains leader ship, it will decide if the heartbeats are ready based on the reported node ids,
    + * supervisors/nodes will take care of the worker heartbeats recovery, a reported node id means all the workers heartbeats on the node are reported.
    + * <p>
    + * 2: If several supervisor also crush and will never recover[or all crush for some unknown reason],
    + * workers will report their heartbeats directly to master, so it has not any effect.
    + */
    +public class TimeOutWorkerHeartbeatsRecoveryStrategy implements IWorkerHeartbeatsRecoveryStrategy {
    +    private static final Logger LOG = LoggerFactory.getLogger(TimeOutWorkerHeartbeatsRecoveryStrategy.class);
    +
    +    private static int NODE_MAX_TIMEOUT_SECS = 600;
    +
    +    private long startTimeSecs;
    +
    +    private Set<String> reportedIDs;
    +
    +    @Override
    +    public void prepare(Map conf) {
    +        NODE_MAX_TIMEOUT_SECS = ObjectReader.getInt(conf.get(Config.SUPERVISOR_WORKER_HEARTBEATS_MAX_TIMEOUT_SECS), 600);
    +        this.startTimeSecs = System.currentTimeMillis() / 1000L;
    +        this.reportedIDs = new HashSet<>();
    +    }
    +
    +    @Override
    +    public boolean isReady(Set<String> nodeIds) {
    +        if (isMaxTimeOut()) {
    +            HashSet<String> tmp = new HashSet<>();
    +            for(String nodeID : nodeIds) {
    +                if (!this.reportedIDs.contains(nodeID))
    +                tmp.add(nodeID);
    +            }
    +            LOG.warn("Failed to recover heartbeats for nodes: {} with timeout {}s", tmp, NODE_MAX_TIMEOUT_SECS);
    +            return true;
    --- End diff --
    
    So after max timeout we just consider it as ready, right?


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @danny0405 
    Thanks for updating your great effort!
    The change is huge, and unfortunately I can't find time to review this at once. I'll try to review partially and multiple times.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152701420
  
    --- Diff: storm-client/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java ---
    @@ -147,21 +153,52 @@ protected void issueMapCallback(ConcurrentHashMap<String, Runnable> callbackConc
     
         @Override
         public List<String> assignments(Runnable callback) {
    +        //deprecated
             if (callback != null) {
                 assignmentsCallback.set(callback);
             }
    -        return stateStorage.get_children(ClusterUtils.ASSIGNMENTS_SUBTREE, callback != null);
    +        return this.backend.assignments();
         }
     
         @Override
         public Assignment assignmentInfo(String stormId, Runnable callback) {
    +        //deprecated
    +        if (callback != null) {
    +            assignmentInfoCallback.put(stormId, callback);
    +        }
    +        return this.backend.getAssignment(stormId);
    +    }
    +
    +    @Override
    +    public Assignment remoteAssignmentInfo(String stormId, Runnable callback) {
    +        //deprecated
             if (callback != null) {
                 assignmentInfoCallback.put(stormId, callback);
             }
             byte[] serialized = stateStorage.get_data(ClusterUtils.assignmentPath(stormId), callback != null);
             return ClusterUtils.maybeDeserialize(serialized, Assignment.class);
         }
     
    +    @Override
    +    public Map<String, Assignment> assignmentsInfo() {
    +        return this.backend.assignmentsInfo();
    +    }
    +
    +    @Override
    +    public void syncRemoteAssignments(Map<String, byte[]> remote) {
    +        if (null != remote) {
    +            this.backend.syncRemoteAssignments(remote);
    +        } else {
    +            Map<String, byte[]> tmp = new HashMap<>();
    +            List<String> stormIDS = this.stateStorage.get_children(ClusterUtils.ASSIGNMENTS_SUBTREE, false);
    --- End diff --
    
    `stormIDS` -> `stormIds` due to checkstyle, `stormID` -> `stormId` as well


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @danny0405 @revans2 just posted the patch of STORM-2898: #2531. Please participate reviewing when you have free time. Thanks in advance.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r173217779
  
    --- Diff: storm-client/src/jvm/org/apache/storm/utils/SupervisorClient.java ---
    @@ -0,0 +1,80 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.utils;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.generated.Supervisor;
    +import org.apache.storm.security.auth.ThriftClient;
    +import org.apache.storm.security.auth.ThriftConnectionType;
    +import org.apache.thrift.transport.TTransportException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.Map;
    +
    +public class SupervisorClient extends ThriftClient {
    +    private Supervisor.Client _client;
    +    private static final Logger LOG = LoggerFactory.getLogger(SupervisorClient.class);
    +
    +    public static SupervisorClient getConfiguredClient(Map conf, String host) {
    +        int port = Integer.parseInt(conf.get(Config.SUPERVISOR_THRIFT_PORT).toString());
    +        return getConfiguredClientAs(conf, host, port, null);
    +    }
    +
    +    public static SupervisorClient getConfiguredClient(Map conf, String host, int port) {
    +        return getConfiguredClientAs(conf, host, port, null);
    +    }
    +
    +    public static SupervisorClient getConfiguredClientAs(Map conf, String host, int port, String asUser) {
    --- End diff --
    
    I don't think we want to support this right now as we don't have a use case for it.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @erikdw
    > For storm-on-mesos, the Supervisor + Workers run in a per-topology container on each host. The > Supervisor is the container's init process, so if it dies then the Workers die with it. So the
    > problematic scenario you outlined doesn't exist for that use-case.
    
    This sounds great, maybe a specified port is a better choice compared to range of ports.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152693735
  
    --- Diff: storm-client/src/jvm/org/apache/storm/assignments/ILocalAssignmentsBackend.java ---
    @@ -0,0 +1,103 @@
    +/**
    + * 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.assignments;
    +
    +import org.apache.storm.generated.Assignment;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +/**
    + * Interface for storing local assignments.
    + */
    +public interface ILocalAssignmentsBackend {
    +
    +    /**
    +     * initial function for creating backend.
    +     * @param conf
    +     */
    +    void prepare(Map conf);
    +
    +    /**
    +     * Keep a storm assignment to local state or update old assignment.
    +     * @param stormID storm runtime id
    +     * @param assignment assignment as thrift
    +     */
    +    void keepOrUpdateAssignment(String stormID, Assignment assignment);
    +
    +    /**
    +     * Get assignment as byte[] for a storm
    --- End diff --
    
    We are returning Assignment, the instance of thrift generated class, not a byte[].
    And please add . to the end of comment. Checkstyle also checks that.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152898161
  
    --- Diff: storm-server/src/main/java/org/apache/storm/daemon/supervisor/timer/SynchronizeAssignments.java ---
    @@ -0,0 +1,133 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.daemon.supervisor.timer;
    +
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.daemon.supervisor.ReadClusterState;
    +import org.apache.storm.daemon.supervisor.Supervisor;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.SupervisorAssignments;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.NimbusClient;
    +import org.apache.storm.utils.Utils;
    +import org.apache.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.HashMap;
    +import java.util.Map;
    +
    +/**
    + * A runnable which will synchronize assignments to node local and then worker processes.
    + */
    +public class SynchronizeAssignments implements Runnable {
    +    private static final Logger LOG = LoggerFactory.getLogger(SynchronizeAssignments.class);
    +
    +    private Supervisor supervisor;
    +    private SupervisorAssignments assignments;
    +    private ReadClusterState readClusterState;
    +
    +    public SynchronizeAssignments(Supervisor supervisor, SupervisorAssignments assignments, ReadClusterState readClusterState) {
    +        this.supervisor = supervisor;
    +        this.assignments = assignments;
    +        this.readClusterState = readClusterState;
    +    }
    +
    +    @Override
    +    public void run() {
    +        // first sync assignments to local, then sync processes.
    +        if (null == assignments) {
    +            getAssignmentsFromMaster(this.supervisor.getConf(), this.supervisor.getStormClusterState(), this.supervisor.getAssignmentId());
    +        } else {
    +            assignedAssignmentsToLocal(this.supervisor.getStormClusterState(), assignments);
    +        }
    +        this.readClusterState.run();
    +    }
    +
    +    /**
    +     * Used by {@link Supervisor} to fetch assignments when start up.
    +     * @param supervisor
    +     */
    +    public void getAssignmentsFromMasterUntilSuccess(Supervisor supervisor) {
    +        boolean success = false;
    +        NimbusClient master;
    +        while (!success) {
    +            try {
    +                master = NimbusClient.getConfiguredClient(supervisor.getConf());
    +                SupervisorAssignments assignments = master.getClient().getSupervisorAssignments(supervisor.getAssignmentId());
    +                assignedAssignmentsToLocal(supervisor.getStormClusterState(), assignments);
    +                success = true;
    +                try {
    +                    master.close();
    +                } catch (Throwable t) {
    +                    LOG.warn("Close master client exception", t);
    +                }
    +            } catch (Exception t) {
    +                // just ignore the exception
    +            }
    +            if (!success) {
    +                LOG.info("Waiting for a success sync of assignments from master...");
    +                try {
    +                    Thread.sleep(5000l);
    +                } catch (InterruptedException e) {
    +                    throw new RuntimeException(e);
    +                }
    +            }
    +
    +        }
    +
    +    }
    +
    +    public void getAssignmentsFromMaster(Map conf, IStormClusterState clusterState, String node) {
    +        if(ConfigUtils.isLocalMode(conf)) {
    +            try {
    +                SupervisorAssignments assignments = this.supervisor.getLocalNimbus().getSupervisorAssignments(node);
    +                assignedAssignmentsToLocal(clusterState, assignments);
    +            } catch (TException e) {
    +                LOG.error("Get assignments from local master exception", e);
    +            }
    +        } else {
    +            NimbusClient master;
    +            try {
    +                master = NimbusClient.getConfiguredClient(conf);
    +                SupervisorAssignments assignments = master.getClient().getSupervisorAssignments(node);
    +                LOG.debug("Sync an assignments from master, will start to sync with assignments: {}", assignments);
    +                assignedAssignmentsToLocal(clusterState, assignments);
    +                try {
    +                    master.close();
    +                } catch (Throwable t) {
    +                    LOG.warn("Close master client exception", t);
    +                }
    +            } catch (Exception t) {
    +                LOG.error("Get assignments from master exception", t);
    +            }
    +        }
    +    }
    +
    +    private static void assignedAssignmentsToLocal(IStormClusterState clusterState, SupervisorAssignments assignments) {
    +        if (null == assignments) {
    +            //unknown error, just skip
    +            return;
    +        }
    +        Map<String, byte[]> serAssignments = new HashMap<>();
    --- End diff --
    
    Maybe we could simplify here:
    
    ```
    Map<String, byte[]> serAssignments = assignments.get_storm_assignment().entrySet().stream()
    .collect(toMap(e -> e.getKey(), e -> Utils.serialize(e.getValue())));
    ```
    
    That's just suggestion, either is fine.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r153401446
  
    --- Diff: storm-client/src/jvm/org/apache/storm/stats/StatsUtil.java ---
    @@ -1589,18 +1570,16 @@ public static ComponentPageInfo aggCompExecsStats(
                 if (lastReportedTime != null) {
                     reportedTime = lastReportedTime;
                 } else {
    -                reportedTime = 0;
    +                reportedTime = lastReportedTime = 0;
    --- End diff --
    
    I was confused that lastReportedTime is not used afterwards. My bad.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152703334
  
    --- Diff: storm-client/src/jvm/org/apache/storm/daemon/worker/WorkerState.java ---
    @@ -51,12 +51,7 @@
     import org.apache.storm.task.WorkerTopologyContext;
     import org.apache.storm.tuple.AddressedTuple;
     import org.apache.storm.tuple.Fields;
    -import org.apache.storm.utils.ConfigUtils;
    -import org.apache.storm.utils.Utils;
    -import org.apache.storm.utils.DisruptorQueue;
    -import org.apache.storm.utils.ObjectReader;
    -import org.apache.storm.utils.ThriftTopologyUtils;
    -import org.apache.storm.utils.TransferDrainer;
    +import org.apache.storm.utils.*;
    --- End diff --
    
    avoid wildcard import


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152717501
  
    --- Diff: storm-core/test/clj/org/apache/storm/nimbus_test.clj ---
    @@ -792,15 +795,18 @@
           (bind _ (log-message "ass1, t76, pre beat: " (pr-str ass1)))
           (bind _ (log-message "ass2, t76, pre beat: " (pr-str ass2)))
           (is (= ass1 (executor-assignment cluster storm-id executor-id1)))
    -      (is (not= ass2 (executor-assignment cluster storm-id executor-id2)))
    +      ;TODO: fix local test of worker reports
    --- End diff --
    
    it is not necessary, i will fire a JIRA to fix this


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152707672
  
    --- Diff: storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java ---
    @@ -1395,10 +1374,59 @@ private void updateAllHeartbeats(Map<String, Assignment> existingAssignments, Ma
                 updateHeartbeats(topoId, topologyToExecutors.get(topoId), entry.getValue());
             }
         }
    +
    +    private void updateCachedHeartbeatsFromWorker(SupervisorWorkerHeartbeat workerHeartbeat) {
    +        Map<List<Integer>, Map<String, Object>> executorBeats = StatsUtil.convertWorkerBeats(workerHeartbeat);
    +        String topoId = workerHeartbeat.get_storm_id();
    +        Map<List<Integer>, Map<String, Object>> cache = heartbeatsCache.get().get(topoId);
    +        if(cache == null) {
    +            cache = new HashMap<>();
    +            heartbeatsCache.getAndUpdate(new Assoc<>(topoId, cache));
    +        }
    +        Set<List<Integer>> executors = new HashSet<>();
    +        for(ExecutorInfo executorInfo : workerHeartbeat.get_executors()) {
    +            executors.add(Arrays.asList(executorInfo.get_task_start(), executorInfo.get_task_end()));
    +        }
    +
    +        StatsUtil.updateHeartbeatCache(heartbeatsCache.get().get(topoId), executorBeats, executors,
    +                ObjectReader.getInt(conf.get(DaemonConfig.NIMBUS_TASK_TIMEOUT_SECS)));
    +
    +    }
    +
    +    private void updateCachedHeartbeatsFromSupervisor(SupervisorWorkerHeartbeats workerHeartbeats) {
    +        workerHeartbeats.get_worker_heartbeats().forEach(this::updateCachedHeartbeatsFromWorker);
    +        if(!heartbeatsReadyFlag.get() && !Strings.isNullOrEmpty(workerHeartbeats.get_supervisor_id())) {
    +            heartbeatsRecoveryStrategy.reportNodeId(workerHeartbeats.get_supervisor_id());
    +        }
    +    }
    +
    +    /**
    +     * decide if the heartbeats is recovered for a master, will wait for all the assignments nodes to recovery,
    +     * every node will take care its node heartbeats reporting
    +     * @return
    +     */
    +    private boolean isHeartbeatsRecovered() {
    +        if(heartbeatsReadyFlag.get()) {
    +            return true;
    +        }
    +        Set<String> allNodes = new HashSet<>();
    +        for(Map.Entry<String, Assignment> assignmentEntry: stormClusterState.assignmentsInfo().entrySet()) {
    +            allNodes.addAll(assignmentEntry.getValue().get_node_host().keySet());
    +        }
    +        boolean isReady = heartbeatsRecoveryStrategy.isReady(allNodes);
    +        if(isReady) {
    +            heartbeatsReadyFlag.getAndSet(true);
    +        }
    +        return isReady;
    +    }
         
         private Set<List<Integer>> aliveExecutors(TopologyDetails td, Set<List<Integer>> allExecutors, Assignment assignment) {
             String topoId = td.getId();
             Map<List<Integer>, Map<String, Object>> hbCache = heartbeatsCache.get().get(topoId);
    +        //in case that no workers report any heartbeats yet.
    +        if(null == hbCache) {
    --- End diff --
    
    space between f and (


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    I just put up a pull request to make it work with worker tokens.
    
    https://github.com/danny0405/storm/pull/1
    
    I want to spend some more time looking at the code though before merging it in.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @HeartSaVioR 
    Ok, i will start my work as soon as possible, this PR is delayed because i was in holiday of the Chinese Spring Festival, and now i'm back to work.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r158669343
  
    --- Diff: storm-server/src/main/java/org/apache/storm/daemon/supervisor/ReadClusterState.java ---
    @@ -69,10 +65,8 @@
         public ReadClusterState(Supervisor supervisor) throws Exception {
             this.superConf = supervisor.getConf();
             this.stormClusterState = supervisor.getStormClusterState();
    -        this.syncSupEventManager = supervisor.getEventManger();
    -        this.assignmentVersions = new AtomicReference<>(new HashMap<>());
             this.assignmentId = supervisor.getAssignmentId();
    -        this.iSuper = supervisor.getiSupervisor();
    +        this.iSupervisor = supervisor.getiSupervisor();
    --- End diff --
    
    Please rebase the change with latest master and fix new code with `iSuper`. The compilation breaks with latest master + current patch.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r153055629
  
    --- Diff: storm-client/src/jvm/org/apache/storm/Config.java ---
    @@ -1293,6 +1318,44 @@
         @isPositiveNumber
         public static final String SUPERVISOR_CPU_CAPACITY = "supervisor.cpu.capacity";
     
    +    @isInteger
    --- End diff --
    
    Makes sense. Thanks for clarifying.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @danny0405 
    There's a tip regarding supported features on Github: you can get a patch file which contains commits or a full diff file. Just add ".patch" for former, or add ".diff" for latter, and Github will serve the file.
    
    Since we just want to have a squashed commit, you can follow below steps to push effectively squashed commit: 
    1. create a new branch based on latest master branch
    2. download diff file via `wget https://github.com/apache/storm/pull/2433.diff` or curl or whatever
    3. apply diff via `git apply 2433.diff` (after 3 you can see modified files as well as untracked files)
    4. `mvn clean install -Prat,all-tests` to run tests (mandatory)
    5. (optional, but recommended) build binary dist (`mvn clean package` in storm-dist/binary) and do manual tests with binary dist as well
    6. if everything is OK, add files to the stage and commit the change with proper commit log message
    
    Please let me know if the tip doesn't work.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r158737618
  
    --- Diff: storm-client/src/jvm/org/apache/storm/assignments/LocalAssignmentsBackendFactory.java ---
    @@ -0,0 +1,48 @@
    +/**
    + * 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.assignments;
    +
    +import com.google.common.base.Preconditions;
    +import org.apache.storm.Config;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.ReflectionUtils;
    +
    +import java.util.Map;
    +
    +/**
    + * Factory class for creating local assignments.
    + */
    +public class LocalAssignmentsBackendFactory {
    +
    +    public static ILocalAssignmentsBackend getBackend(Map<String, Object> conf) {
    +        if (conf.get(Config.NIMBUS_LOCAL_ASSIGNMENTS_BACKEND_CLASS) != null) {
    +            Object targetObj = ReflectionUtils.newInstance((String) conf.get(Config.NIMBUS_LOCAL_ASSIGNMENTS_BACKEND_CLASS));
    +            Preconditions.checkState(targetObj instanceof ILocalAssignmentsBackend, "{} must implements ILocalAssignmentsBackend", Config.NIMBUS_LOCAL_ASSIGNMENTS_BACKEND_CLASS);
    +            ((ILocalAssignmentsBackend)targetObj).prepare(conf);
    +            return (ILocalAssignmentsBackend) targetObj;
    +        }
    +
    +        return getDefault();
    --- End diff --
    
    It is invoked by ClusterUtils and conf is passed, but it is passed by Object hence it must do `instanceof`. I think `mkStormClusterStateImpl` method is not beauty (allowing Object param and relying on instanceof) but let's leave it then for now. ClusterUtils may need to be refactored afterwards.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152705555
  
    --- Diff: storm-client/test/jvm/org/apache/storm/assignments/LocalAssignmentsBackendTest.java ---
    @@ -0,0 +1,95 @@
    +/*
    + * 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.assignments;
    +
    +import org.apache.commons.collections.map.HashedMap;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.NodeInfo;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.junit.Test;
    +
    +import java.util.*;
    --- End diff --
    
    avoid wildcard import


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152705847
  
    --- Diff: storm-core/test/clj/org/apache/storm/nimbus_test.clj ---
    @@ -520,7 +522,7 @@
           (is (= 10 (count (task-info "4"))))
           (is (= 7 (storm-num-workers state "test")))
         )))
    -
    +;
    --- End diff --
    
    unnecessary change


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152717747
  
    --- Diff: storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java ---
    @@ -231,7 +183,15 @@
     
         @VisibleForTesting
         public static final List<ACL> ZK_ACLS = Arrays.asList(ZooDefs.Ids.CREATOR_ALL_ACL.get(0),
    -            new ACL(ZooDefs.Perms.READ | ZooDefs.Perms.CREATE, ZooDefs.Ids.ANYONE_ID_UNSAFE));
    +            new ACL(ZooDefs.Perms.READ ^ ZooDefs.Perms.CREATE, ZooDefs.Ids.ANYONE_ID_UNSAFE));
    --- End diff --
    
    i copy it from old nimbus.clj, i didn't get the point why 2.0 change it to or


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r173202043
  
    --- Diff: storm-client/src/jvm/org/apache/storm/assignments/ILocalAssignmentsBackend.java ---
    @@ -0,0 +1,113 @@
    +/**
    + * 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.assignments;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.storm.generated.Assignment;
    +
    +/**
    + * Interface for storing local assignments.
    + */
    +public interface ILocalAssignmentsBackend {
    +    /**
    +     * Decide if the assignments is synchronized from remote state-store.
    +     */
    +    boolean isSynchronized();
    +
    +    /**
    +     * Mark this backend as synchronized when sync work is done.
    +     */
    +    void setSynchronized();
    +
    +    /**
    +     * Initial function for creating backend.
    +     * @param conf config
    +     */
    +    void prepare(Map conf);
    +
    +    /**
    +     * Keep a storm assignment to local state or update old assignment.
    +     * @param stormID storm runtime id
    +     * @param assignment assignment as thrift
    +     */
    +    void keepOrUpdateAssignment(String stormID, Assignment assignment);
    +
    +    /**
    +     * Get assignment as {@link Assignment} for a storm.
    +     * @param stormId storm runtime id
    +     * @return assignment
    +     */
    +    Assignment getAssignment(String stormId);
    +
    +    void removeAssignment(String stormId);
    +
    +    /**
    +     * List all the storm runtime ids of local assignments.
    +     * @return a list of storm ids
    +     */
    +    List<String> assignments();
    +
    +    /**
    +     * Get all the local assignments of local state.
    +     * @return mapping of storm-id -> assignment
    +     */
    +    Map<String, Assignment> assignmentsInfo();
    +
    +    /**
    +     * Sync remote assignments to local, if remote is null, we will sync it from zk.
    +     * @param remote specific remote assignments, if it is null, it will sync from zookeeper[only used for nimbus]
    +     */
    +    void syncRemoteAssignments(Map<String, byte[]> remote);
    +
    +    /**
    +     * Keep a mapping storm-name -> storm-id to local state.
    +     * @param stormName storm name
    +     * @param stormId storm runtime id
    +     */
    +    void keepStormId(String stormName, String stormId);
    +
    +    /**
    +     * Get storm runtime id from local.
    +     * @param stormName name of a storm
    +     * @return runtime storm id
    +     */
    +    String getStormId(String stormName);
    +
    +    /**
    +     * Sync remote storm ids to local, will just used for nimbus.
    +     * @param remote remote ids from state store
    +     */
    +    void syncRemoteIds(Map<String, String> remote);
    +
    +    void deleteStormId(String stormName);
    --- End diff --
    
    nit: add some javadocs for this.  Also is it really the name of the topology and not the id?  It just would be good to have the javadocs so there is no confusion on it.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r173615818
  
    --- Diff: storm-server/src/main/java/org/apache/storm/daemon/supervisor/Supervisor.java ---
    @@ -234,6 +295,60 @@ public void launchDaemon() {
             }
         }
     
    +    private void launchSupervisorThriftServer(Map conf) throws IOException {
    +        // validate port
    +        int port = getThriftServerPort();
    +        try {
    +            ServerSocket socket = new ServerSocket(port);
    +            socket.close();
    +        } catch (BindException e) {
    +            LOG.error("{} is not available. Check if another process is already listening on {}", port, port);
    +            throw new RuntimeException(e);
    +        }
    +
    +        TProcessor processor = new org.apache.storm.generated.Supervisor.Processor(
    +                new org.apache.storm.generated.Supervisor.Iface() {
    +                    @Override
    +                    public void sendSupervisorAssignments(SupervisorAssignments assignments)
    +                            throws AuthorizationException, TException {
    +                        LOG.info("Got an assignments from master, will start to sync with assignments: {}", assignments);
    +                        SynchronizeAssignments syn = new SynchronizeAssignments(getSupervisor(), assignments, getReadClusterState());
    +                        getEventManger().add(syn);
    +                    }
    +
    +                    @Override
    +                    public Assignment getLocalAssignmentForStorm(String id)
    +                            throws NotAliveException, AuthorizationException, TException {
    +                        Assignment assignment = getStormClusterState().assignmentInfo(id, null);
    +                        if (null == assignment) {
    +                            throw new NotAliveException("No local assignment assigned for storm: " + id + " for node: " + getHostName());
    +                        }
    +                        return assignment;
    +                    }
    +
    +                    @Override
    +                    public void sendSupervisorWorkerHeartbeat(SupervisorWorkerHeartbeat heartbeat)
    +                            throws AuthorizationException, TException {
    +                        // do nothing now
    +                    }
    --- End diff --
    
    This is where the authorization should take place, but i'm not familia with the auth thing.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152694460
  
    --- Diff: storm-client/src/jvm/org/apache/storm/assignments/InMemoryAssignmentBackend.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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.assignments;
    +
    +import org.apache.storm.cluster.ClusterUtils;
    +import org.apache.storm.generated.Assignment;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +/**
    + * An assignment backend which will keep all assignments and id-info in memory. Only used if no backend is specified internal.
    + */
    +public class InMemoryAssignmentBackend implements ILocalAssignmentsBackend {
    +    private static final Logger LOG = LoggerFactory.getLogger(InMemoryAssignmentBackend.class);
    +
    +    protected Map<String, Assignment> idToAssignment;
    +    protected Map<String, String> idToName;
    +    protected Map<String, String> nameToId;
    +    /**
    +     * Used for assignments set/get, assignments set/get should be kept thread safe
    +     */
    +    private final Object assignmentsLock = new Object();
    +
    +    public InMemoryAssignmentBackend() {}
    +
    +    @Override
    +    public void prepare(Map conf) {
    +        // do nothing for conf now
    +        this.idToAssignment = new HashMap<>();
    +        this.idToName = new HashMap<>();
    +        this.nameToId = new HashMap<>();
    +    }
    +
    +    @Override
    +    public void keepOrUpdateAssignment(String stormID, Assignment assignment) {
    +        synchronized (assignmentsLock) {
    +            this.idToAssignment.put(stormID, assignment);
    +        }
    +    }
    +
    +    @Override
    +    public Assignment getAssignment(String stormID) {
    +        synchronized (assignmentsLock) {
    +            return this.idToAssignment.get(stormID);
    +        }
    +    }
    +
    +    @Override
    +    public void removeAssignment(String stormID) {
    +        synchronized (assignmentsLock) {
    +            this.idToAssignment.remove(stormID);
    +        }
    +    }
    +
    +    @Override
    +    public List<String> assignments() {
    +        if(idToAssignment == null) {
    +            return new ArrayList<>();
    +        }
    +        List<String> ret = new ArrayList<>();
    +        synchronized (assignmentsLock) {
    +            ret.addAll(this.idToAssignment.keySet());
    +            return ret;
    +        }
    +    }
    +
    +    @Override
    +    public Map<String, Assignment> assignmentsInfo() {
    +        Map<String, Assignment> ret = new HashMap<>();
    +        synchronized (assignmentsLock) {
    +            ret.putAll(this.idToAssignment);
    +        }
    +
    +        return ret;
    +    }
    +
    +    @Override
    +    public void syncRemoteAssignments(Map<String, byte[]> remote) {
    +        Map<String, Assignment> tmp = new HashMap<>();
    --- End diff --
    
    We could use Stream to simplify lines here.
    
    this.idToAssignment = remote.entrySet().stream()
    .collect(toMap(e -> e.getKey(), 
    e -> ClusterUtils.maybeDeserialize(e.getValue(), Assignment.class));
    
    But current logic is also clear to understand, so either is fine.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152696040
  
    --- Diff: storm-client/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java ---
    @@ -43,13 +44,15 @@
         private static Logger LOG = LoggerFactory.getLogger(StormClusterStateImpl.class);
     
         private IStateStorage stateStorage;
    +    private ILocalAssignmentsBackend backend;
    --- End diff --
    
    Let's make it clearer: `localAssignments` (or `assignments`) feels me more important, so maybe better to have it in field name.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @revans2 
    Thanks for double checking. I think you've also suggested removing the behavior leveraging disk to communicate between worker and supervisor (so that the way of communication is consistent), but you don't think it's not mandatory. Got it.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152900918
  
    --- Diff: storm-server/src/main/java/org/apache/storm/nimbus/LeaderListenerCallback.java ---
    @@ -0,0 +1,213 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.nimbus;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.collect.Sets;
    +import org.apache.commons.io.IOUtils;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.curator.framework.recipes.leader.LeaderLatch;
    +import org.apache.storm.Config;
    +import org.apache.storm.blobstore.BlobStore;
    +import org.apache.storm.blobstore.InputStreamWithMeta;
    +import org.apache.storm.cluster.ClusterUtils;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.daemon.nimbus.TopoCache;
    +import org.apache.storm.generated.AuthorizationException;
    +import org.apache.storm.generated.KeyNotFoundException;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.security.auth.ReqContext;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.Utils;
    +import org.apache.storm.zookeeper.ClientZookeeper;
    +import org.apache.zookeeper.CreateMode;
    +import org.apache.zookeeper.data.ACL;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.io.IOException;
    +import java.util.*;
    +
    +/**
    + * A callback function when nimbus gains leadership
    + */
    +public class LeaderListenerCallback {
    +    private static final Logger LOG = LoggerFactory.getLogger(LeaderListenerCallback.class);
    +
    +    private final BlobStore blobStore;
    +    private final TopoCache tc;
    +    private final IStormClusterState clusterState;
    +
    +    private final CuratorFramework zk;
    +    private final LeaderLatch leaderLatch;
    +
    +    private final Map conf;
    +    private final List<ACL> acls;
    +
    +    private static final String STORM_JAR_SUFFIX = "-stormjar.jar";
    +    private static final String STORM_CODE_SUFFIX = "-stormcode.ser";
    +    private static final String STORM_CONF_SUFFIX = "-stormconf.ser";
    +
    +    public LeaderListenerCallback(Map conf, CuratorFramework zk, LeaderLatch leaderLatch, BlobStore blobStore, TopoCache tc, IStormClusterState clusterState, List<ACL> acls) {
    +        this.blobStore = blobStore;
    +        this.tc = tc;
    +        this.clusterState = clusterState;
    +        this.zk = zk;
    +        this.leaderLatch = leaderLatch;
    +        this.conf = conf;
    +        this.acls = acls;
    +    }
    +
    +    /**
    +     * Invoke when gains leadership.
    +     */
    +    public void leaderCallBack() {
    +        //in local mode, only one leader exist
    +        if (ConfigUtils.isLocalMode(conf)) {
    +            return;
    +        }
    +        //set up nimbus-info to zk
    +        setUpNimbusInfo(acls);
    +        //sync zk assignments/id-info to local
    +        LOG.info("Sync remote assignments and id-info to local");
    +        clusterState.syncRemoteAssignments(null);
    +        clusterState.syncRemoteIds(null);
    +
    +        Set<String> activeTopologyIds = new TreeSet<>(ClientZookeeper.getChildren(zk,
    +                conf.get(Config.STORM_ZOOKEEPER_ROOT) + ClusterUtils.STORMS_SUBTREE, false));
    +
    +        Set<String> activeTopologyBlobKeys = populateTopologyBlobKeys(activeTopologyIds);
    +        Set<String> activeTopologyCodeKeys = filterTopologyCodeKeys(activeTopologyBlobKeys);
    +        Set<String> allLocalBlobKeys = Sets.newHashSet(blobStore.listKeys());
    +        Set<String> allLocalTopologyBlobKeys = filterTopologyBlobKeys(allLocalBlobKeys);
    +
    +        // this finds all active topologies blob keys from all local topology blob keys
    +        Sets.SetView<String> diffTopology = Sets.difference(activeTopologyBlobKeys, allLocalTopologyBlobKeys);
    +        LOG.info("active-topology-blobs [{}] local-topology-blobs [{}] diff-topology-blobs [{}]",
    +                generateJoinedString(activeTopologyIds), generateJoinedString(allLocalTopologyBlobKeys),
    +                generateJoinedString(diffTopology));
    +
    +        if (diffTopology.isEmpty()) {
    +            Set<String> activeTopologyDependencies = getTopologyDependencyKeys(activeTopologyCodeKeys);
    +
    +            // this finds all dependency blob keys from active topologies from all local blob keys
    +            Sets.SetView<String> diffDependencies = Sets.difference(activeTopologyDependencies, allLocalBlobKeys);
    +            LOG.info("active-topology-dependencies [{}] local-blobs [{}] diff-topology-dependencies [{}]",
    +                    generateJoinedString(activeTopologyDependencies), generateJoinedString(allLocalBlobKeys),
    +                    generateJoinedString(diffDependencies));
    +
    +            if (diffDependencies.isEmpty()) {
    +                LOG.info("Accepting leadership, all active topologies and corresponding dependencies found locally.");
    +                tc.clear();
    +            } else {
    +                LOG.info("Code for all active topologies is available locally, but some dependencies are not found locally, "
    +                        + "giving up leadership.");
    +                closeLatch();
    +            }
    +        } else {
    +            LOG.info("code for all active topologies not available locally, giving up leadership.");
    +            closeLatch();
    +        }
    +    }
    +
    +    /**
    +     * Invoke when lost leadership.
    +     */
    +    public void notLeaderCallback() {
    --- End diff --
    
    Please add `LOG.info("{} lost leadership.", hostName);` given that it helps on investigating the issue on Nimbus.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152897261
  
    --- Diff: storm-server/src/main/java/org/apache/storm/daemon/supervisor/timer/ReportWorkerHeartbeats.java ---
    @@ -0,0 +1,119 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.daemon.supervisor.timer;
    +
    +import org.apache.storm.daemon.supervisor.Supervisor;
    +import org.apache.storm.daemon.supervisor.SupervisorUtils;
    +import org.apache.storm.generated.LSWorkerHeartbeat;
    +import org.apache.storm.generated.SupervisorWorkerHeartbeat;
    +import org.apache.storm.generated.SupervisorWorkerHeartbeats;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.NimbusClient;
    +import org.apache.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Map;
    +
    +/**
    + * Runnable reporting local worker reported heartbeats to master, supervisor should take care the of the heartbeats
    + * integrity for the master heartbeats recovery, a non-null node id means that the heartbeats are full,
    + * and master can go on to check and wait others nodes when doing a heartbeats recovery.
    + */
    +public class ReportWorkerHeartbeats implements Runnable {
    +    private static final Logger LOG = LoggerFactory.getLogger(ReportWorkerHeartbeats.class);
    +
    +    private Supervisor supervisor;
    +    private Map<String, Object> conf;
    +
    +    public ReportWorkerHeartbeats(Map<String, Object> conf, Supervisor supervisor) {
    +        this.conf = conf;
    +        this.supervisor = supervisor;
    +    }
    +
    +    @Override
    +    public void run() {
    +        SupervisorWorkerHeartbeats supervisorWorkerHeartbeats = getAndResetWorkerHeartbeats();
    +        reportWorkerHeartbeats(supervisorWorkerHeartbeats);
    +    }
    +
    +    private SupervisorWorkerHeartbeats getAndResetWorkerHeartbeats() {
    +        Map<String, LSWorkerHeartbeat> localHeartbeats;
    +        try {
    +            localHeartbeats = SupervisorUtils.readWorkerHeartbeats(this.conf);
    +            return getSupervisorWorkerHeartbeatsFromLocal(localHeartbeats);
    +        } catch (Exception e) {
    +            LOG.error("Read local worker heartbeats error, skipping heartbeats for this round, msg:{}", e.getMessage());
    +            return null;
    +        }
    +    }
    +
    +    private SupervisorWorkerHeartbeats getSupervisorWorkerHeartbeatsFromLocal(Map<String, LSWorkerHeartbeat> localHeartbeats) {
    +        SupervisorWorkerHeartbeats supervisorWorkerHeartbeats = new SupervisorWorkerHeartbeats();
    +
    +        List<SupervisorWorkerHeartbeat> heartbeatList = new ArrayList<>();
    +
    +        for (Map.Entry<String, LSWorkerHeartbeat> lsWorkerHeartbeatEntry : localHeartbeats.entrySet()) {
    +            LSWorkerHeartbeat lsWorkerHeartbeat = lsWorkerHeartbeatEntry.getValue();
    +            // local worker heartbeat can be null cause some error/exception
    +            if (null == lsWorkerHeartbeat) {
    +                continue;
    +            }
    +
    +            SupervisorWorkerHeartbeat supervisorWorkerHeartbeat = new SupervisorWorkerHeartbeat();
    +            supervisorWorkerHeartbeat.set_storm_id(lsWorkerHeartbeat.get_topology_id());
    +            supervisorWorkerHeartbeat.set_executors(lsWorkerHeartbeat.get_executors());
    +            supervisorWorkerHeartbeat.set_time_secs(lsWorkerHeartbeat.get_time_secs());
    +
    +            heartbeatList.add(supervisorWorkerHeartbeat);
    +        }
    +        supervisorWorkerHeartbeats.set_supervisor_id(this.supervisor.getId());
    +        supervisorWorkerHeartbeats.set_worker_heartbeats(heartbeatList);
    +        return supervisorWorkerHeartbeats;
    +    }
    +
    +    private void reportWorkerHeartbeats(SupervisorWorkerHeartbeats supervisorWorkerHeartbeats) {
    +        if (supervisorWorkerHeartbeats == null) {
    +            // error/exception thrown, just skip
    +            return;
    +        }
    +        // if is local mode, just get the local nimbus instance and set the heartbeats
    --- End diff --
    
    nit: if it is local mode


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152901980
  
    --- Diff: storm-server/src/main/java/org/apache/storm/nimbus/TimeOutWorkerHeartbeatsRecoveryStrategy.java ---
    @@ -0,0 +1,86 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.nimbus;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.utils.ObjectReader;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.HashSet;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * Wait for a node to report worker heartbeats until a configured timeout. For cases below we have strategies:
    + * <p>
    + * 1: When nimbus gains leader ship, it will decide if the heartbeats are ready based on the reported node ids,
    + * supervisors/nodes will take care of the worker heartbeats recovery, a reported node id means all the workers heartbeats on the node are reported.
    + * <p>
    + * 2: If several supervisor also crush and will never recover[or all crush for some unknown reason],
    + * workers will report their heartbeats directly to master, so it has not any effect.
    + */
    +public class TimeOutWorkerHeartbeatsRecoveryStrategy implements IWorkerHeartbeatsRecoveryStrategy {
    +    private static final Logger LOG = LoggerFactory.getLogger(TimeOutWorkerHeartbeatsRecoveryStrategy.class);
    +
    +    private static int NODE_MAX_TIMEOUT_SECS = 600;
    +
    +    private long startTimeSecs;
    +
    +    private Set<String> reportedIDs;
    +
    +    @Override
    +    public void prepare(Map conf) {
    +        NODE_MAX_TIMEOUT_SECS = ObjectReader.getInt(conf.get(Config.SUPERVISOR_WORKER_HEARTBEATS_MAX_TIMEOUT_SECS), 600);
    +        this.startTimeSecs = System.currentTimeMillis() / 1000L;
    --- End diff --
    
    Let's use `Times.currentTimeMillis()` for being compatible with simulating time.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @erikdw 
    
    OK, sorry @JessicaLHartog for misunderstanding, and thanks for noticing us to not making us unintentionally breaking the thing again.
    
    > Is it possible to specify this setting (supervisor.thrift.port) as a CLI parameter to the supervisor as it is launched?
    
    Unless there's no bug on storm binary script, `-c supervisor.thrift.port=<port>` should work as same as it is described in storm.yaml.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @HeartSaVioR
    
    I didn't really want to make pacemaker the default because not everyone needs it, standing up yet another server is a pain for customers, and I was hoping it would be temporary.  Once we have the metrics out of the heartbeat, which yes I hope to do in conditionally in 2.x and completely remove in 3.x, we can leave simple heartbeats in ZK, or move to push the heartbeats directly to nimbus, whichever is fine with me.  Then there would be no need for pacemaker and it could be deprecated.
    
    Hopefully I answered most of your other questions in my response to @danny0405 [above](https://github.com/apache/storm/pull/2433#issuecomment-356965437)


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r173219240
  
    --- Diff: storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java ---
    @@ -319,14 +344,17 @@ private static StormBase make(TopologyStatus status) {
         private static final TopologyStateTransition REMOVE_TRANSITION = (args, nimbus, topoId, base) -> {
             LOG.info("Killing topology: {}", topoId);
             IStormClusterState state = nimbus.getStormClusterState();
    +        Assignment oldAss = state.assignmentInfo(topoId, null);
    --- End diff --
    
    Can we rename this variable.  It might be offensive to some people.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @danny0405 Kindly reminder.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152702351
  
    --- Diff: storm-client/src/jvm/org/apache/storm/daemon/worker/Worker.java ---
    @@ -50,25 +51,14 @@
     import org.apache.storm.executor.ExecutorShutdown;
     import org.apache.storm.executor.IRunningExecutor;
     import org.apache.storm.executor.LocalExecutor;
    -import org.apache.storm.generated.Credentials;
    -import org.apache.storm.generated.ExecutorInfo;
    -import org.apache.storm.generated.ExecutorStats;
    -import org.apache.storm.generated.LSWorkerHeartbeat;
    -import org.apache.storm.generated.LogConfig;
    +import org.apache.storm.generated.*;
     import org.apache.storm.messaging.IConnection;
     import org.apache.storm.messaging.IContext;
     import org.apache.storm.messaging.TaskMessage;
     import org.apache.storm.security.auth.AuthUtils;
     import org.apache.storm.security.auth.IAutoCredentials;
     import org.apache.storm.stats.StatsUtil;
    -import org.apache.storm.utils.ConfigUtils;
    -import org.apache.storm.utils.Utils;
    -import org.apache.storm.utils.DisruptorBackpressureCallback;
    -import org.apache.storm.utils.LocalState;
    -import org.apache.storm.utils.ObjectReader;
    -import org.apache.storm.utils.Time;
    -import org.apache.storm.utils.WorkerBackpressureCallback;
    -import org.apache.storm.utils.WorkerBackpressureThread;
    +import org.apache.storm.utils.*;
    --- End diff --
    
    Same here.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r172979050
  
    --- Diff: storm-client/src/jvm/org/apache/storm/Config.java ---
    @@ -991,6 +991,31 @@
         @isPositiveNumber
         public static final String NIMBUS_QUEUE_SIZE = "nimbus.queue.size";
     
    +    /**
    --- End diff --
    
    We cannot do this without a lot of refactoring.  Currently the backend is a part of the StormClusterState, when to me it feels like storm cluster state should actually be one of the backends instead.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152717373
  
    --- Diff: storm-client/src/jvm/org/apache/storm/stats/StatsUtil.java ---
    @@ -1589,18 +1570,16 @@ public static ComponentPageInfo aggCompExecsStats(
                 if (lastReportedTime != null) {
                     reportedTime = lastReportedTime;
                 } else {
    -                reportedTime = 0;
    +                reportedTime = lastReportedTime = 0;
                 }
             }
     
             if (lastNimbusTime == null || !reportedTime.equals(lastReportedTime)) {
                 lastNimbusTime = Time.currentTimeSecs();
             }
     
    -        ret.put("is-timed-out", Time.deltaSecs(lastNimbusTime) >= timeout);
    --- End diff --
    
    yes, it's safe we will refresh the is-timed-out when we make assignments


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @danny0405 Thanks for your thoughtful review. Updated the branch with commit amended.
    https://github.com/HeartSaVioR/storm/commit/9cc8a1540985d350ede4738a38173df47ae99a32


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152893646
  
    --- Diff: storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java ---
    @@ -1672,7 +1703,7 @@ private int fragmentedCpu() {
             long beforeSchedule = System.currentTimeMillis();
             scheduler.schedule(topologies, cluster);
             long scheduleTimeElapsedMs = System.currentTimeMillis() - beforeSchedule;
    -        LOG.info("Scheduling took {} ms for {} topologies", scheduleTimeElapsedMs, topologies.getTopologies().size());
    +        LOG.debug("Scheduling took {} ms for {} topologies", scheduleTimeElapsedMs, topologies.getTopologies().size());
    --- End diff --
    
    Makes sense.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @danny0405 
    Btw, the build in local passed, but it incurs considerable addition in build time.
    
    ```
    [INFO] ------------------------------------------------------------------------
    [INFO] Reactor Summary:
    [INFO]
    [INFO] Storm .............................................. SUCCESS [  3.699 s]
    [INFO] Apache Storm - Checkstyle .......................... SUCCESS [  0.708 s]
    [INFO] multilang-javascript ............................... SUCCESS [  0.126 s]
    [INFO] multilang-python ................................... SUCCESS [  0.265 s]
    [INFO] multilang-ruby ..................................... SUCCESS [  0.093 s]
    [INFO] maven-shade-clojure-transformer .................... SUCCESS [  2.413 s]
    [INFO] storm-maven-plugins ................................ SUCCESS [  2.638 s]
    [INFO] Storm Client ....................................... SUCCESS [ 44.607 s]
    [INFO] storm-server ....................................... SUCCESS [02:27 min]
    [INFO] storm-clojure ...................................... SUCCESS [  5.257 s]
    [INFO] Storm Core ......................................... SUCCESS [23:34 min]
    [INFO] Storm Webapp ....................................... SUCCESS [ 11.411 s]
    [INFO] storm-clojure-test ................................. SUCCESS [  2.185 s]
    [INFO] storm-submit-tools ................................. SUCCESS [  8.076 s]
    [INFO] flux ............................................... SUCCESS [  0.093 s]
    [INFO] flux-wrappers ...................................... SUCCESS [  0.354 s]
    [INFO] storm-kafka ........................................ SUCCESS [01:48 min]
    [INFO] storm-autocreds .................................... SUCCESS [  3.273 s]
    [INFO] storm-hdfs ......................................... SUCCESS [ 39.636 s]
    [INFO] storm-hbase ........................................ SUCCESS [  4.100 s]
    [INFO] flux-core .......................................... SUCCESS [  4.035 s]
    [INFO] flux-examples ...................................... SUCCESS [ 12.051 s]
    [INFO] storm-sql-runtime .................................. SUCCESS [  2.391 s]
    [INFO] storm-sql-core ..................................... SUCCESS [01:47 min]
    [INFO] storm-sql-kafka .................................... SUCCESS [  4.736 s]
    [INFO] storm-redis ........................................ SUCCESS [  5.799 s]
    [INFO] storm-sql-redis .................................... SUCCESS [  8.231 s]
    [INFO] storm-mongodb ...................................... SUCCESS [  0.826 s]
    [INFO] storm-sql-mongodb .................................. SUCCESS [  3.936 s]
    [INFO] storm-sql-hdfs ..................................... SUCCESS [ 12.313 s]
    [INFO] sql ................................................ SUCCESS [  0.150 s]
    [INFO] storm-hive ......................................... SUCCESS [ 18.669 s]
    [INFO] storm-jdbc ......................................... SUCCESS [  1.777 s]
    [INFO] storm-eventhubs .................................... SUCCESS [  3.725 s]
    [INFO] storm-elasticsearch ................................ SUCCESS [ 20.401 s]
    [INFO] storm-solr ......................................... SUCCESS [  1.515 s]
    [INFO] storm-metrics ...................................... SUCCESS [  0.590 s]
    [INFO] storm-cassandra .................................... SUCCESS [01:01 min]
    [INFO] storm-mqtt ......................................... SUCCESS [ 22.396 s]
    [INFO] storm-kafka-client ................................. SUCCESS [02:06 min]
    [INFO] storm-opentsdb ..................................... SUCCESS [  0.664 s]
    [INFO] storm-kafka-monitor ................................ SUCCESS [  2.430 s]
    [INFO] storm-kinesis ...................................... SUCCESS [  1.281 s]
    [INFO] storm-druid ........................................ SUCCESS [  1.958 s]
    [INFO] storm-jms .......................................... SUCCESS [ 12.099 s]
    [INFO] storm-pmml ......................................... SUCCESS [  0.267 s]
    [INFO] storm-rocketmq ..................................... SUCCESS [ 11.298 s]
    [INFO] blobstore-migrator ................................. SUCCESS [ 12.028 s]
    [INFO] Storm Integration Test ............................. SUCCESS [  0.696 s]
    [INFO] storm-starter ...................................... SUCCESS [ 19.945 s]
    [INFO] storm-loadgen ...................................... SUCCESS [  2.457 s]
    [INFO] storm-mongodb-examples ............................. SUCCESS [  0.771 s]
    [INFO] storm-redis-examples ............................... SUCCESS [  1.096 s]
    [INFO] storm-opentsdb-examples ............................ SUCCESS [  1.489 s]
    [INFO] storm-solr-examples ................................ SUCCESS [  1.312 s]
    [INFO] storm-kafka-examples ............................... SUCCESS [  2.364 s]
    [INFO] storm-kafka-client-examples ........................ SUCCESS [  2.005 s]
    [INFO] storm-jdbc-examples ................................ SUCCESS [  0.654 s]
    [INFO] storm-hdfs-examples ................................ SUCCESS [  9.583 s]
    [INFO] storm-hbase-examples ............................... SUCCESS [ 10.147 s]
    [INFO] storm-hive-examples ................................ SUCCESS [ 10.006 s]
    [INFO] storm-elasticsearch-examples ....................... SUCCESS [  4.126 s]
    [INFO] storm-mqtt-examples ................................ SUCCESS [  3.036 s]
    [INFO] storm-pmml-examples ................................ SUCCESS [  1.100 s]
    [INFO] storm-jms-examples ................................. SUCCESS [  1.318 s]
    [INFO] storm-rocketmq-examples ............................ SUCCESS [  0.828 s]
    [INFO] Storm Perf ......................................... SUCCESS [  7.193 s]
    [INFO] ------------------------------------------------------------------------
    [INFO] BUILD SUCCESS
    [INFO] ------------------------------------------------------------------------
    [INFO] Total time: 39:03 min
    [INFO] Finished at: 2017-12-20T21:34:12+09:00
    [INFO] Final Memory: 595M/1519M
    [INFO] ------------------------------------------------------------------------
    ```
    
    Do you have any pointer for the build slowness? I haven't encounter it taking more than 15 min with any branches/commits.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @HeartSaVioR I just posted a high level design for a delegation tokens [here](https://issues.apache.org/jira/browse/STORM-2898?focusedCommentId=16323160&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-16323160)
    
    Please take a look.  If it looks good I will start throwing together a patch based on it.  The we can get into some of the specifics associated with it.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @danny0405 
    Yeah I understand it is not ideal. Let's wait for more inputs. If the new behavior (for old workers, when supervisor killed) is acceptable we can just go on. If it isn't, the patch may not be accepted without mitigation whether it is a part of the patch or follow-up patch, so even I should go ahead and implement my suggestion based on your patch.
    
    If you are planning for exploring more based on your patch and have a design doc, it would be really nice to share it to Storm dev mailing list. It may help persuade your opinion to Storm community.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152897458
  
    --- Diff: storm-server/src/main/java/org/apache/storm/daemon/supervisor/timer/ReportWorkerHeartbeats.java ---
    @@ -0,0 +1,119 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.daemon.supervisor.timer;
    +
    +import org.apache.storm.daemon.supervisor.Supervisor;
    +import org.apache.storm.daemon.supervisor.SupervisorUtils;
    +import org.apache.storm.generated.LSWorkerHeartbeat;
    +import org.apache.storm.generated.SupervisorWorkerHeartbeat;
    +import org.apache.storm.generated.SupervisorWorkerHeartbeats;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.NimbusClient;
    +import org.apache.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Map;
    +
    +/**
    + * Runnable reporting local worker reported heartbeats to master, supervisor should take care the of the heartbeats
    + * integrity for the master heartbeats recovery, a non-null node id means that the heartbeats are full,
    + * and master can go on to check and wait others nodes when doing a heartbeats recovery.
    + */
    +public class ReportWorkerHeartbeats implements Runnable {
    +    private static final Logger LOG = LoggerFactory.getLogger(ReportWorkerHeartbeats.class);
    +
    +    private Supervisor supervisor;
    +    private Map<String, Object> conf;
    +
    +    public ReportWorkerHeartbeats(Map<String, Object> conf, Supervisor supervisor) {
    +        this.conf = conf;
    +        this.supervisor = supervisor;
    +    }
    +
    +    @Override
    +    public void run() {
    +        SupervisorWorkerHeartbeats supervisorWorkerHeartbeats = getAndResetWorkerHeartbeats();
    +        reportWorkerHeartbeats(supervisorWorkerHeartbeats);
    +    }
    +
    +    private SupervisorWorkerHeartbeats getAndResetWorkerHeartbeats() {
    +        Map<String, LSWorkerHeartbeat> localHeartbeats;
    +        try {
    +            localHeartbeats = SupervisorUtils.readWorkerHeartbeats(this.conf);
    +            return getSupervisorWorkerHeartbeatsFromLocal(localHeartbeats);
    +        } catch (Exception e) {
    +            LOG.error("Read local worker heartbeats error, skipping heartbeats for this round, msg:{}", e.getMessage());
    +            return null;
    +        }
    +    }
    +
    +    private SupervisorWorkerHeartbeats getSupervisorWorkerHeartbeatsFromLocal(Map<String, LSWorkerHeartbeat> localHeartbeats) {
    +        SupervisorWorkerHeartbeats supervisorWorkerHeartbeats = new SupervisorWorkerHeartbeats();
    +
    +        List<SupervisorWorkerHeartbeat> heartbeatList = new ArrayList<>();
    +
    +        for (Map.Entry<String, LSWorkerHeartbeat> lsWorkerHeartbeatEntry : localHeartbeats.entrySet()) {
    +            LSWorkerHeartbeat lsWorkerHeartbeat = lsWorkerHeartbeatEntry.getValue();
    +            // local worker heartbeat can be null cause some error/exception
    +            if (null == lsWorkerHeartbeat) {
    +                continue;
    +            }
    +
    +            SupervisorWorkerHeartbeat supervisorWorkerHeartbeat = new SupervisorWorkerHeartbeat();
    +            supervisorWorkerHeartbeat.set_storm_id(lsWorkerHeartbeat.get_topology_id());
    +            supervisorWorkerHeartbeat.set_executors(lsWorkerHeartbeat.get_executors());
    +            supervisorWorkerHeartbeat.set_time_secs(lsWorkerHeartbeat.get_time_secs());
    +
    +            heartbeatList.add(supervisorWorkerHeartbeat);
    +        }
    +        supervisorWorkerHeartbeats.set_supervisor_id(this.supervisor.getId());
    +        supervisorWorkerHeartbeats.set_worker_heartbeats(heartbeatList);
    +        return supervisorWorkerHeartbeats;
    +    }
    +
    +    private void reportWorkerHeartbeats(SupervisorWorkerHeartbeats supervisorWorkerHeartbeats) {
    +        if (supervisorWorkerHeartbeats == null) {
    +            // error/exception thrown, just skip
    +            return;
    +        }
    +        // if is local mode, just get the local nimbus instance and set the heartbeats
    +        if(ConfigUtils.isLocalMode(conf)){
    +            try {
    +                this.supervisor.getLocalNimbus().sendSupervisorWorkerHeartbeats(supervisorWorkerHeartbeats);
    +            } catch (TException tex) {
    +                LOG.error("Send local supervisor heartbeats error", tex);
    +            }
    +        } else {
    +            NimbusClient master;
    --- End diff --
    
    Let's use try-with-resource like 
    https://github.com/apache/storm/blob/cef450064fa20e2194ef3f51a21c8e6693a285e3/storm-client/src/jvm/org/apache/storm/utils/NimbusClient.java#L77-L79
    
    Local variable `master` will be leaked when L108 throws Exception.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @HeartSaVioR  @revans2
    1. Yeah, we did used pacemaker for a while for a cluster about 200 topologies, but the workers restart frequently just because the Pacemaker heartbeats packets discard.
    2. Also the pacemaker is a single point for the cluster, there is even no HA for it, when pacemaker restart, it will take a long time to recover heartbeats for it[even it has a HA], then most of the workers will time out and be reassigned by master. I raise doubts about keeping heartbeats into just one single point, and it is hard to scale laterally. @revans2 said that your cluster has 900 supervisors but only 120 topologies which push heatbeats to pacemaker. So do we have a index/metics between pacemaker and workers/topologies while not supervisors ?
    3. This patch can really support large cluster, and it is very stable for out production, we have about 8000 topologies, and the new patch can support at least 2000 topologies at least for now. Also the patch has no single point problem compared to Pacemaker, and the heartbeat is very lightweight.
    4. If we just have security problem, i can fix it.



---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r158705496
  
    --- Diff: storm-core/test/clj/org/apache/storm/nimbus_test.clj ---
    @@ -792,15 +795,18 @@
           (bind _ (log-message "ass1, t76, pre beat: " (pr-str ass1)))
           (bind _ (log-message "ass2, t76, pre beat: " (pr-str ass2)))
           (is (= ass1 (executor-assignment cluster storm-id executor-id1)))
    +      ;TODO: fix local test of worker reports
    --- End diff --
    
    Is it addressed? If then please remove it. If not, let's have TODO in body of PR.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @revans2 @HeartSaVioR 
    I have merged master in and fixed the conflicts.
    
    I tried to rebase based on master but there are too much conflicts and it's even harder to pick out which code segment is what i want because the patch review goes to long time, can we just keep what it is ?


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    I am +1, but a little nervous that the tests are failing consistently on travis in exactly the same way, but never on my laptop, but I think we can work that out later if it is a real issue.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @HeartSaVioR i have fixed the checkstyle issue already, yeah the storm-core building is slow, i will check the reason.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @HeartSaVioR I can do this
    But what is the motivation to let 2.0.0 daemons interact with Storm 1.x and 0.10.x workers?
    Another issue: is there already an option i can reuse or i need to fire a new one?


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @danny0405 
    I assume @revans2 is on vacation, so you may want to take your time to handle other tasks for now, and back to this issue when @revans2 is back. Looks like he would like to understand the code more deeper, since this touches across many parts of core thing. 
    
    Thanks for your great efforts and patience!


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r173042840
  
    --- Diff: conf/defaults.yaml ---
    @@ -182,7 +195,7 @@ worker.log.level.reset.poll.secs: 30
     # control how many worker receiver threads we need per worker
     topology.worker.receiver.thread.count: 1
     
    -task.heartbeat.frequency.secs: 3
    +task.heartbeat.frequency.secs: 60
    --- End diff --
    
    Actually task.heartbeat.frequency.secs is now only used as a metrics reporting interval, so it can totally be longer, RPC heartbeat reporting interval is configured by Config.WORKER_HEARTBEAT_FREQUENCY_SECS.
    
    Because now we still use ZK as a storage of our metrics, one nimute interval is ok in production which is the internal metrics consumtion/aggregation time interval.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    OK I'll try to put up another pull request to cover basic auth for the supervisor.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @revans2 
    I'm also not familiar with newly added worker token feature, and you said we need to apply token auth to supervisor as well.
    Could you apply worker token feature to this patch so that we could test the final patch and merge this in? Thanks in advance!


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152903034
  
    --- Diff: storm-server/src/main/java/org/apache/storm/nimbus/AssignmentDistributionService.java ---
    @@ -0,0 +1,280 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.nimbus;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.daemon.supervisor.Supervisor;
    +import org.apache.storm.generated.SupervisorAssignments;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.ObjectReader;
    +import org.apache.storm.utils.SupervisorClient;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.Closeable;
    +import java.io.IOException;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.Random;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.TimeUnit;
    +
    +/**
    + * <p>A service for distributing master assignments to supervisors, this service makes the assignments notification asynchronous.
    + * <p>We support multiple working threads to distribute assignment, every thread has a queue buffer.
    + * <p>Master will shuffle its node request to the queues, if the target queue is full, we just discard the request, let the supervisors sync instead.
    + * <p/>
    + * <pre>{@code
    + * Working mode
    + *                      +--------+         +-----------------+
    + *                      | queue1 |   ==>   | Working thread1 |
    + * +--------+ shuffle   +--------+         +-----------------+
    + * | Master |   ==>
    + * +--------+           +--------+         +-----------------+
    + *                      | queue2 |   ==>   | Working thread2 |
    + *                      +--------+         +-----------------+
    + * }
    + * </pre>
    + */
    +public class AssignmentDistributionService implements Closeable {
    +    private static final Logger LOG = LoggerFactory.getLogger(AssignmentDistributionService.class);
    +    private ExecutorService service;
    +
    +    /**
    +     * Flag to indicate if the service is active
    +     */
    +    private volatile boolean active = false;
    +
    +    private Random random;
    +    /**
    +     * Working threads num.
    +     */
    +    private int threadsNum = 0;
    +    /**
    +     * Working thread queue size.
    +     */
    +    private int queueSize = 0;
    +
    +    /**
    +     * Assignments request queue.
    +     */
    +    private volatile Map<Integer, LinkedBlockingQueue<NodeAssignments>> assignmentsQueue;
    +
    +    /**
    +     * local supervisors for local cluster assignments distribution
    +     */
    +    private Map<String, Supervisor> localSupervisors;
    +
    +    private Map conf;
    +
    +    private boolean isLocalMode = false; // boolean cache for local mode decision
    +
    +    /**
    +     * Function for initialization.
    +     *
    +     * @param conf
    +     */
    +    public void prepare(Map conf) {
    +        this.conf = conf;
    +        this.random = new Random(47);
    +
    +        this.threadsNum = ObjectReader.getInt(conf.get(Config.NIMBUS_ASSIGNMENTS_SERVICE_THREADS), 10);
    +        this.queueSize = ObjectReader.getInt(conf.get(Config.NIMBUS_ASSIGNMENTS_SERVICE_THREAD_QUEUE_SIZE), 100);
    +
    +        this.assignmentsQueue = new HashMap<>();
    +        for (int i = 0; i < threadsNum; i++) {
    +            this.assignmentsQueue.put(i, new LinkedBlockingQueue<NodeAssignments>(queueSize));
    +        }
    +        //start the thread pool
    +        this.service = Executors.newFixedThreadPool(threadsNum);
    +        this.active = true;
    +        //start the threads
    +        for (int i = 0; i < threadsNum; i++) {
    +            this.service.submit(new DistributeTask(this, i));
    +        }
    +        // for local cluster
    +        localSupervisors = new HashMap<>();
    +        if (ConfigUtils.isLocalMode(conf)) {
    +            isLocalMode = true;
    +        }
    +    }
    +
    +    @Override
    +    public void close() throws IOException {
    +        this.active = false;
    +        this.service.shutdownNow();
    +        try {
    +            this.service.awaitTermination(10l, TimeUnit.SECONDS);
    +        } catch (InterruptedException e) {
    +            LOG.error("Failed to close assignments distribute service");
    +        }
    +        this.assignmentsQueue = null;
    +    }
    +
    +    public void addAssignmentsForNode(String node, String host, SupervisorAssignments assignments) {
    +        try {
    +            boolean success = nextQueue().offer(NodeAssignments.getInstance(node, host, assignments), 5l, TimeUnit.SECONDS);
    +            if (!success) {
    +                LOG.warn("Discard an assignment distribution for node {} because the target sub queue is full", node);
    +            }
    +
    +        } catch (InterruptedException e) {
    +            LOG.error("Add node assignments interrupted: {}", e.getMessage());
    +            throw new RuntimeException(e);
    +        }
    +    }
    +
    +    static class NodeAssignments {
    +        private String node;
    +        private SupervisorAssignments assignments;
    +        private String host;
    +
    +        private NodeAssignments(String node, String host, SupervisorAssignments assignments) {
    +            this.node = node;
    +            this.host = host;
    +            this.assignments = assignments;
    +        }
    +
    +        public static NodeAssignments getInstance(String node, String host, SupervisorAssignments assignments) {
    +            return new NodeAssignments(node, host, assignments);
    +        }
    +
    +        //supervisor assignment id/supervisor id
    +        public String getNode() {
    +            return this.node;
    +        }
    +
    +        public String getHost() {
    +            return host;
    +        }
    +
    +        public SupervisorAssignments getAssignments() {
    +            return this.assignments;
    +        }
    +
    +    }
    +
    +    /**
    +     * Task to distribute assignments.
    +     */
    +    static class DistributeTask implements Runnable {
    +        private AssignmentDistributionService service;
    +        private Integer queueIndex;
    +
    +        DistributeTask(AssignmentDistributionService service, Integer index) {
    +            this.service = service;
    +            this.queueIndex = index;
    +        }
    +
    +        @Override
    +        public void run() {
    +            while (true) {
    +                try {
    +                    NodeAssignments nodeAssignments = this.service.nextAssignments(queueIndex);
    +                    sendAssignmentsToNode(nodeAssignments);
    +                } catch (InterruptedException e) {
    +                    if (service.isActive()) {
    +                        LOG.error("Get an unexpected interrupt when distributing assignments to node, {}", e.getCause());
    +                    } else {
    +                        // service is off now just interrupt it.
    +                        Thread.currentThread().interrupt();
    +                    }
    +                }
    +            }
    +        }
    +
    +        private void sendAssignmentsToNode(NodeAssignments assignments) {
    +            if (this.service.isLocalMode) {
    +                //local node
    +                Supervisor supervisor = this.service.localSupervisors.get(assignments.getNode());
    +                if (supervisor != null) {
    +                    supervisor.sendSupervisorAssignments(assignments.getAssignments());
    +                } else {
    +                    LOG.error("Can not find node {} for assignments distribution");
    +                    throw new RuntimeException("null for node {} supervisor instance");
    +                }
    +            } else {
    +                // distributed mode
    +                try {
    +                    SupervisorClient client = SupervisorClient.getConfiguredClient(service.getConf(), assignments.getHost());
    +                    try {
    +                        client.getClient().sendSupervisorAssignments(assignments.getAssignments());
    +                    } catch (Exception e) {
    +                        //just ignore the exception.
    +                        LOG.error("Exception when trying to send assignments to node {}: {}", assignments.getNode(), e.getMessage());
    +                    } finally {
    +                        try {
    +                            if (client != null) {
    +                                client.close();
    +                            }
    +                        } catch (Exception e) {
    +                            LOG.error("Exception closing client for node: {}", assignments.getNode());
    +                        }
    +                    }
    +                } catch (Throwable e) {
    +                    //just ignore any error/exception.
    +                    LOG.error("Exception to create supervisor client for node{}: {}", assignments.getNode(), e.getMessage());
    +                }
    +
    +            }
    +        }
    +    }
    +
    +    public void addLocalSupervisor(Supervisor supervisor) {
    +        this.localSupervisors.put(supervisor.getId(), supervisor);
    +    }
    +
    +    private Integer nextQueueID() {
    +        return this.random.nextInt(threadsNum);
    +    }
    +
    +    private LinkedBlockingQueue<NodeAssignments> nextQueue() {
    +        return this.assignmentsQueue.get(nextQueueID());
    +    }
    +
    +    private LinkedBlockingQueue<NodeAssignments> getQueueByID(Integer queueIndex) {
    +        return this.assignmentsQueue.get(queueIndex);
    +    }
    +
    +    public NodeAssignments nextAssignments(Integer queueIndex) throws InterruptedException {
    +        NodeAssignments target = null;
    +        while (true) {
    +            target = getQueueByID(queueIndex).poll();
    +            if (target != null) {
    +                return target;
    +            }
    +            Thread.sleep(100L);
    --- End diff --
    
    Let's use `Time.sleep()`.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    I decided against refactoring here.  The changes I want to make go a lot deeper than just this patch.  and this patch is following the conventions in the code already.  I'll spend some time reviewing this and hopefully we can get this patch in soon.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r173217461
  
    --- Diff: storm-client/src/jvm/org/apache/storm/utils/SupervisorClient.java ---
    @@ -0,0 +1,80 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.utils;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.generated.Supervisor;
    +import org.apache.storm.security.auth.ThriftClient;
    +import org.apache.storm.security.auth.ThriftConnectionType;
    +import org.apache.thrift.transport.TTransportException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.Map;
    +
    +public class SupervisorClient extends ThriftClient {
    --- End diff --
    
    nit: some javadocs would be great here.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    I have done manual tests with multiple nodes (5 nodes) with Nimbus H/A (2 nodes): switching nimbus leader, killing supervisor, killing workers.
    
    I'm +1 once my remaining review comments are addressed afterwards.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r162322607
  
    --- Diff: storm-client/src/storm.thrift ---
    @@ -836,3 +848,33 @@ exception HBAuthorizationException {
     exception HBExecutionException {
       1: required string msg;
     }
    +
    +struct SupervisorAssignments {
    +    1: optional map<string, Assignment> storm_assignment = {}
    +}
    +
    +service Supervisor {
    +  /**
    +   * Send node specific assignments to supervisor
    +   */
    +  void sendSupervisorAssignments(1: SupervisorAssignments assignments) throws (1: AuthorizationException aze);
    +  /**
    +   * Get local assignment for a storm
    +   */
    +  Assignment getLocalAssignmentForStorm(1: string id) throws (1: NotAliveException e, 2: AuthorizationException aze);
    +  /**
    +   * Send worker heartbeat to local supervisor
    +   */
    +  void sendSupervisorWorkerHeartbeat(1: SupervisorWorkerHeartbeat heartbeat) throws (1: AuthorizationException aze);
    +}
    +
    +struct SupervisorWorkerHeartbeat {
    --- End diff --
    
    You could check importing ttypes.py to see it works fine. If current patch doesn't break ttypes.py please ignore my comment.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152693807
  
    --- Diff: storm-client/src/jvm/org/apache/storm/assignments/ILocalAssignmentsBackend.java ---
    @@ -0,0 +1,103 @@
    +/**
    + * 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.assignments;
    +
    +import org.apache.storm.generated.Assignment;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +/**
    + * Interface for storing local assignments.
    + */
    +public interface ILocalAssignmentsBackend {
    +
    +    /**
    +     * initial function for creating backend.
    +     * @param conf
    +     */
    +    void prepare(Map conf);
    +
    +    /**
    +     * Keep a storm assignment to local state or update old assignment.
    +     * @param stormID storm runtime id
    +     * @param assignment assignment as thrift
    +     */
    +    void keepOrUpdateAssignment(String stormID, Assignment assignment);
    +
    +    /**
    +     * Get assignment as byte[] for a storm
    +     * @param stormID storm runtime id
    +     * @return
    +     */
    +    Assignment getAssignment(String stormID);
    +
    +    void removeAssignment(String stormID);
    +
    +    /**
    +     * List all the storm runtime ids of local assignments
    --- End diff --
    
    Please add `.` on the end. I would rather not point out all the places. You could fix them all based on checkstyle warning.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152893622
  
    --- Diff: storm-server/pom.xml ---
    @@ -130,7 +130,7 @@
                     <artifactId>maven-checkstyle-plugin</artifactId>
                     <!--Note - the version would be inherited-->
                     <configuration>
    -                    <maxAllowedViolations>2655</maxAllowedViolations>
    +                    <maxAllowedViolations>2800</maxAllowedViolations>
    --- End diff --
    
    No, that means newly added patch should not violate any checkstyle issue. Even better if we could reduce the number.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r173215685
  
    --- Diff: storm-client/src/jvm/org/apache/storm/daemon/worker/Worker.java ---
    @@ -392,6 +401,30 @@ public void establishLogSettingCallback() {
             workerState.stormClusterState.topologyLogConfig(topologyId, this::checkLogConfigChanged);
         }
     
    +    /**
    +     * Send a heartbeat to local supervisor first to check if supervisor is ok for heartbeating.
    +     */
    +    private void heartbeatToMasterIfLocalbeatFail(LSWorkerHeartbeat lsWorkerHeartbeat) {
    +        if (ConfigUtils.isLocalMode(this.conf)) {
    +            return;
    +        }
    +        //in distributed mode, send heartbeat directly to master if local supervisor goes down
    +        SupervisorWorkerHeartbeat workerHeartbeat = new SupervisorWorkerHeartbeat(lsWorkerHeartbeat.get_topology_id(),
    +                lsWorkerHeartbeat.get_executors(), lsWorkerHeartbeat.get_time_secs());
    +        try (SupervisorClient client = SupervisorClient.getConfiguredClient(conf, Utils.hostname(), supervisorPort)){
    +            client.getClient().sendSupervisorWorkerHeartbeat(workerHeartbeat);
    +        } catch (Throwable tr1) {
    --- End diff --
    
    I don't like catching Throwable.  Could we just move it to Exception, because if we get an Error often those are not recoverable and I would prefer to to let it restart the worker in those cases.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152895920
  
    --- Diff: storm-server/src/main/java/org/apache/storm/daemon/supervisor/Supervisor.java ---
    @@ -30,29 +32,36 @@
     import java.util.concurrent.atomic.AtomicReference;
     
     import org.apache.commons.io.FileUtils;
    +import org.apache.storm.Config;
     import org.apache.storm.DaemonConfig;
     import org.apache.storm.StormTimer;
     import org.apache.storm.cluster.ClusterStateContext;
     import org.apache.storm.cluster.ClusterUtils;
     import org.apache.storm.cluster.DaemonType;
     import org.apache.storm.cluster.IStormClusterState;
     import org.apache.storm.daemon.DaemonCommon;
    +import org.apache.storm.daemon.supervisor.timer.ReportWorkerHeartbeats;
     import org.apache.storm.daemon.supervisor.timer.SupervisorHealthCheck;
     import org.apache.storm.daemon.supervisor.timer.SupervisorHeartbeat;
    +import org.apache.storm.daemon.supervisor.timer.SynchronizeAssignments;
     import org.apache.storm.event.EventManager;
     import org.apache.storm.event.EventManagerImp;
    -import org.apache.storm.generated.LocalAssignment;
    +import org.apache.storm.generated.*;
    --- End diff --
    
    avoid wildcard import


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152703836
  
    --- Diff: storm-client/src/jvm/org/apache/storm/grouping/LoadAwareShuffleGrouping.java ---
    @@ -291,7 +291,10 @@ private Scope calculateScope(Map<Integer, NodeInfo> taskToNodePort, Map<String,
         private Map<String, String> getHostToRackMapping(Map<Integer, NodeInfo> taskToNodePort) {
             Set<String> hosts = new HashSet();
             for (int task: targetTasks) {
    -            hosts.add(taskToNodePort.get(task).get_node());
    +            //if is a kill, taskToNodePort will be an empty map which is refreshed by WorkerState
    --- End diff --
    
    Let's elaborate the part of comment : `if is a kill`. I don't get it.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r172945323
  
    --- Diff: storm-client/src/storm.thrift ---
    @@ -477,12 +477,13 @@ struct SupervisorInfo {
         1: required i64 time_secs;
         2: required string hostname;
         3: optional string assignment_id;
    -    4: optional list<i64> used_ports;
    -    5: optional list<i64> meta;
    -    6: optional map<string, string> scheduler_meta;
    -    7: optional i64 uptime_secs;
    -    8: optional string version;
    -    9: optional map<string, double> resources_map;
    +    4: optional i32 server_port;
    --- End diff --
    
    -1 we cannot change the tags for a thrift object like this.  It totally breaks backwards compatibility.  Granted this is for an internal class and probably does not matter much, but there is no reason to do it.  Please make the server_port 10: and set all of the others back to where they were before.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by JessicaLHartog <gi...@git.apache.org>.
Github user JessicaLHartog commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @danny0405 @revans2 Thanks for keeping the Storm on Mesos project in mind. We (@erikdw, @srishtyagrawal, and myself) appreciate it!
    
    Right now, Storm on Mesos doesn't work for Storm versions 1.1.x and above because we rely on the slot-centric scheduling instead of the now supervisor-centric scheduling (for more details see [STORM-2126](https://issues.apache.org/jira/browse/STORM-2126)). We talked to @revans2 about this in our issue[#222](https://github.com/mesos/storm/issues/222#issuecomment-352514556)... so that's a bit of context that may be necessary to understand the problem a little better.
    
    With respect to the ask to not further break Storm on Mesos we think it would be best for us to be able to specify (at runtime) to the supervisor which port it needs to listen on for these heartbeat messages. The way we see it, while the Supervisor being able to specify a range for the heartbeat port sounds like it may work, the Supervisor should _also_ be able to accept an assigned heartbeat port. Namely, once we adapt to the changes suggested [here](https://github.com/mesos/storm/issues/222#issuecomment-352530608), we can start each Supervisor using one of the ports Mesos offers to us.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r158768477
  
    --- Diff: storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java ---
    @@ -1733,13 +1825,24 @@ private TopologyResources getResourcesForTopology(String topoId, StormBase base)
             return ret;
         }
     
    +    private boolean isReadyForMKAssignments() throws Exception {
    +        if(isLeader()) {
    +            if(isHeartbeatsRecovered()) {
    +                return true;
    +            }
    +            LOG.info("waiting for worker heartbeats recovery, skipping assignments");
    --- End diff --
    
    done


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @HeartSaVioR
    But with this patch nimbus can collect both the 2.0./1.x workers heartbeats, this is no differences here.
    
    We do not collect heartbeats based on workers now.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @HeartSaVioR i have fix the building slow issue, it is because during the nimbus-test clojure testing, i start all the workers every time after submitting topology, in the old mode, when the cluster attribute SUPERVISOR-ENABLE is turned false, no workers will be started in local.
    
    This is also the reason why there are 2 TODOs in nimbus-test with new mode, i have fixed it together.
    
    I turned of master forward assignments distribution and just let supervisor sync when time simulation is turned on for local mode.
    
    So this is fixed.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r158758661
  
    --- Diff: storm-client/src/jvm/org/apache/storm/assignments/LocalAssignmentsBackendFactory.java ---
    @@ -0,0 +1,48 @@
    +/**
    + * 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.assignments;
    +
    +import com.google.common.base.Preconditions;
    +import org.apache.storm.Config;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.ReflectionUtils;
    +
    +import java.util.Map;
    +
    +/**
    + * Factory class for creating local assignments.
    + */
    +public class LocalAssignmentsBackendFactory {
    +
    +    public static ILocalAssignmentsBackend getBackend(Map<String, Object> conf) {
    +        if (conf.get(Config.NIMBUS_LOCAL_ASSIGNMENTS_BACKEND_CLASS) != null) {
    +            Object targetObj = ReflectionUtils.newInstance((String) conf.get(Config.NIMBUS_LOCAL_ASSIGNMENTS_BACKEND_CLASS));
    +            Preconditions.checkState(targetObj instanceof ILocalAssignmentsBackend, "{} must implements ILocalAssignmentsBackend", Config.NIMBUS_LOCAL_ASSIGNMENTS_BACKEND_CLASS);
    +            ((ILocalAssignmentsBackend)targetObj).prepare(conf);
    +            return (ILocalAssignmentsBackend) targetObj;
    +        }
    +
    +        return getDefault();
    --- End diff --
    
    Okey


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @HeartSaVioR @revans2 
    I have add support for multiple supervisors for one node.
    Now for supervisor change:
    
    1. The supervisor should be started with a specified port.
    2. All the workers will be started with the port of its parent supervisor as a start arg.
    3. Supervisor will report port -info together with SupervisorInfo to ZK.
    4. Nimbus will get the supervisor port-info from reported SupervisorInfos.
    
    Really thx for your nice review work!


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @HeartSaVioR
    Really thx for your nice work, i will apply your patch and do some tests.
    
    Also i'm doing add Supervisor heartbeat port to `Worker.java` as start args and `SupervisorHeartbeat` to support multi-supervisors for one node.



---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r158705200
  
    --- Diff: storm-client/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java ---
    @@ -147,21 +153,52 @@ protected void issueMapCallback(ConcurrentHashMap<String, Runnable> callbackConc
     
         @Override
         public List<String> assignments(Runnable callback) {
    +        //deprecated
             if (callback != null) {
                 assignmentsCallback.set(callback);
             }
    -        return stateStorage.get_children(ClusterUtils.ASSIGNMENTS_SUBTREE, callback != null);
    +        return this.backend.assignments();
         }
     
         @Override
         public Assignment assignmentInfo(String stormId, Runnable callback) {
    +        //deprecated
    +        if (callback != null) {
    +            assignmentInfoCallback.put(stormId, callback);
    +        }
    +        return this.backend.getAssignment(stormId);
    +    }
    +
    +    @Override
    +    public Assignment remoteAssignmentInfo(String stormId, Runnable callback) {
    +        //deprecated
    --- End diff --
    
    This comment is missed to addressed.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152897084
  
    --- Diff: storm-server/src/main/java/org/apache/storm/daemon/supervisor/timer/ReportWorkerHeartbeats.java ---
    @@ -0,0 +1,119 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.daemon.supervisor.timer;
    +
    +import org.apache.storm.daemon.supervisor.Supervisor;
    +import org.apache.storm.daemon.supervisor.SupervisorUtils;
    +import org.apache.storm.generated.LSWorkerHeartbeat;
    +import org.apache.storm.generated.SupervisorWorkerHeartbeat;
    +import org.apache.storm.generated.SupervisorWorkerHeartbeats;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.NimbusClient;
    +import org.apache.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Map;
    +
    +/**
    + * Runnable reporting local worker reported heartbeats to master, supervisor should take care the of the heartbeats
    + * integrity for the master heartbeats recovery, a non-null node id means that the heartbeats are full,
    + * and master can go on to check and wait others nodes when doing a heartbeats recovery.
    + */
    +public class ReportWorkerHeartbeats implements Runnable {
    +    private static final Logger LOG = LoggerFactory.getLogger(ReportWorkerHeartbeats.class);
    +
    +    private Supervisor supervisor;
    +    private Map<String, Object> conf;
    +
    +    public ReportWorkerHeartbeats(Map<String, Object> conf, Supervisor supervisor) {
    +        this.conf = conf;
    +        this.supervisor = supervisor;
    +    }
    +
    +    @Override
    +    public void run() {
    +        SupervisorWorkerHeartbeats supervisorWorkerHeartbeats = getAndResetWorkerHeartbeats();
    +        reportWorkerHeartbeats(supervisorWorkerHeartbeats);
    +    }
    +
    +    private SupervisorWorkerHeartbeats getAndResetWorkerHeartbeats() {
    +        Map<String, LSWorkerHeartbeat> localHeartbeats;
    +        try {
    +            localHeartbeats = SupervisorUtils.readWorkerHeartbeats(this.conf);
    +            return getSupervisorWorkerHeartbeatsFromLocal(localHeartbeats);
    +        } catch (Exception e) {
    +            LOG.error("Read local worker heartbeats error, skipping heartbeats for this round, msg:{}", e.getMessage());
    +            return null;
    +        }
    +    }
    +
    +    private SupervisorWorkerHeartbeats getSupervisorWorkerHeartbeatsFromLocal(Map<String, LSWorkerHeartbeat> localHeartbeats) {
    +        SupervisorWorkerHeartbeats supervisorWorkerHeartbeats = new SupervisorWorkerHeartbeats();
    +
    +        List<SupervisorWorkerHeartbeat> heartbeatList = new ArrayList<>();
    +
    +        for (Map.Entry<String, LSWorkerHeartbeat> lsWorkerHeartbeatEntry : localHeartbeats.entrySet()) {
    --- End diff --
    
    We could simplify logic using stream:
    
    ```
    List<SupervisorWorkerHeartbeat> heartbeatList = localHeartbeats.values().stream()
    .filter(Objects::nonNull).map(hb -> {
        SupervisorWorkerHeartbeat supervisorWorkerHeartbeat = new SupervisorWorkerHeartbeat();
        supervisorWorkerHeartbeat.set_storm_id(lsWorkerHeartbeat.get_topology_id());
        supervisorWorkerHeartbeat.set_executors(lsWorkerHeartbeat.get_executors());
        supervisorWorkerHeartbeat.set_time_secs(lsWorkerHeartbeat.get_time_secs());
        return supervisorWorkerHeartbeat;
    }).collect(toList());
    ```
    
    or at least use values() instead of entrySet() given that we don't use keys at all.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r158768637
  
    --- Diff: storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java ---
    @@ -4121,7 +4236,43 @@ public boolean isTopologyNameAllowed(String name) throws AuthorizationException,
                 throw new RuntimeException(e);
             }
         }
    -    
    +
    +    @Override
    +    public SupervisorAssignments getSupervisorAssignments(String node) throws AuthorizationException, TException {
    +        try {
    +            if(isLeader()) {
    +                SupervisorAssignments supervisorAssignments = new SupervisorAssignments();
    +                supervisorAssignments.set_storm_assignment(assignmentsForNode(stormClusterState.assignmentsInfo(), node));
    +                return supervisorAssignments;
    +            }
    +        } catch (Exception e) {
    +            //when not leader just return nil which will cause client to get an unknown error
    --- End diff --
    
    I have added log.debug message.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @revans2
    I have fixed the nits problems and merge your patch in.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r173207181
  
    --- Diff: storm-client/src/jvm/org/apache/storm/assignments/InMemoryAssignmentBackend.java ---
    @@ -0,0 +1,152 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.assignments;
    +
    +import java.util.ArrayList;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.storm.cluster.ClusterUtils;
    +import org.apache.storm.generated.Assignment;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * An assignment backend which will keep all assignments and id-info in memory. Only used if no backend is specified internal.
    + */
    +public class InMemoryAssignmentBackend implements ILocalAssignmentsBackend {
    +    private static final Logger LOG = LoggerFactory.getLogger(InMemoryAssignmentBackend.class);
    +
    +    protected Map<String, Assignment> idToAssignment;
    +    protected Map<String, String> idToName;
    +    protected Map<String, String> nameToId;
    +    private volatile boolean isSynchronized = false;
    +
    +    public InMemoryAssignmentBackend() {}
    --- End diff --
    
    nit: you could just delete this line and get the same result.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152716491
  
    --- Diff: storm-client/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java ---
    @@ -43,13 +44,15 @@
         private static Logger LOG = LoggerFactory.getLogger(StormClusterStateImpl.class);
     
         private IStateStorage stateStorage;
    +    private ILocalAssignmentsBackend backend;
    --- End diff --
    
    okey


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @HeartSaVioR @revans2 
    I have merged in master branch and fix the conflicts, but i'm not very familiar with the authorization thing. So could you please help to make some contribution ?
    
    Thx very much in advance.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @revans2 
    First of all, I missed Storm on containers and around security. Thanks for the pointer. Much helped even in my side.
    
    Looks like there are some things to sort out.
    
    1.
    Please take a look at https://issues.apache.org/jira/browse/STORM-2693 and comments. I've already proposed @danny0405 to try out Pacemaker, and he said it didn't help in some cases. Looks like operational experience regarding Pacemaker between you and @danny0405 are somewhat different. 
    https://issues.apache.org/jira/browse/STORM-2693?focusedCommentId=16146530&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-16146530
    
    Maybe @danny0405 is using old version of Storm, or there might be some internal patches not contributed to Apache side. One thing to sort out, but if comment from @danny0405 is valid, that means Pacemaker is not working as expected, and that would be a critical issue for Pacemaker. I don't have huge scale cluster so I haven't manage cluster with Pacemaker.
    
    2.
    We know we utilize ZK in some bad way (write goes heavier whenever workers/topologies are being added), and we introduce Pacemaker as a nice mitigation.
    If we think Pacemaker is now really stable and have all of essential functionalities, we may need to consider making it as default, or publicizing Pacemaker more via guiding when to consider using Pacemaker.
    Without making it as a default or any guide to consider Pacemaker while sizing, users would normally struggle ZK issue first (got bad experience already), and try out Pacemaker as an alternative.
    
    3.
    The bigger concern for me between metrics and assignment is metrics. I don't know how much assignment via ZK affects the performance (if it hurts much it should be also considered), but it is really clear that worker metrics in ZK has been problematic for us.
    
    I believe we will (and should) eventually drop current heartbeat structure which includes metrics, and the sooner the better.
    
    What I have been not clear is how and when. From that point I have been expecting that Metrics V2 will take up the issue, and unfortunately, based on the current patch of Metrics V2, we would probably still use Metrics V1 for built-in metrics in Storm 2.0.0 unless we have separate patch for Metrics V2.
    
    We should have a plan to migrate built-in metrics from Metrics V1 to Metrics V2, because there would be some more TODOs to make it done, and it can't be done partially (especially 2 and 3).
    
    1. Implement worker metrics reporter which reports to Nimbus.
    2. Change Nimbus to get metrics from metrics store instead of heartbeat, which makes UI leverage the metrics.
    3. Migrate built-in metrics from Metrics V1 to Metrics V2: after the patch built-in metrics will not be presented to metric consumer (Metrics V1).
    
    The above work would be backward incompatible, but we have no plan for Storm 2.0.0, and I'd rather not thinking about 3.0.0 even we don't have Storm 2.0.0. It is ideal to be done before Storm 2.0.0, and if it's not possible, I wouldn't mind introducing it in 2.x with disregarding backward compatibility.
    
    I think it is related to the change opened partially and now internally testing in Oath. Could you share a plan for this so that community can determine whether community should wait on news or there're some tasks Storm community should work on?
    
    4. 
    We may want to sort out which should be secure and which are not. Does metric need to be secured? If then same security issue will arise when implementing worker metrics reporter.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152893327
  
    --- Diff: storm-client/src/jvm/org/apache/storm/grouping/LoadAwareShuffleGrouping.java ---
    @@ -291,7 +291,10 @@ private Scope calculateScope(Map<Integer, NodeInfo> taskToNodePort, Map<String,
         private Map<String, String> getHostToRackMapping(Map<Integer, NodeInfo> taskToNodePort) {
             Set<String> hosts = new HashSet();
             for (int task: targetTasks) {
    -            hosts.add(taskToNodePort.get(task).get_node());
    +            //if is a kill, taskToNodePort will be an empty map which is refreshed by WorkerState
    --- End diff --
    
    OK please make a sentence perfect: it doesn't have `subject` which is the point I didn't get it.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152705026
  
    --- Diff: storm-client/src/jvm/org/apache/storm/stats/StatsUtil.java ---
    @@ -1589,18 +1570,16 @@ public static ComponentPageInfo aggCompExecsStats(
                 if (lastReportedTime != null) {
                     reportedTime = lastReportedTime;
                 } else {
    -                reportedTime = 0;
    +                reportedTime = lastReportedTime = 0;
                 }
             }
     
             if (lastNimbusTime == null || !reportedTime.equals(lastReportedTime)) {
                 lastNimbusTime = Time.currentTimeSecs();
             }
     
    -        ret.put("is-timed-out", Time.deltaSecs(lastNimbusTime) >= timeout);
    --- End diff --
    
    Is it safe to remove two key-values? Just to check again.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r158742516
  
    --- Diff: conf/defaults.yaml ---
    @@ -162,6 +168,13 @@ supervisor.memory.capacity.mb: 4096.0
     # for single threaded bolts
     supervisor.cpu.capacity: 400.0
     
    +#Supervisor thrift config
    +supervisor.thrift.port: 6627
    --- End diff --
    
    We often setup test cluster in a node, hence two ports (nimbus/supervisor) should be different.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152902841
  
    --- Diff: storm-server/src/main/java/org/apache/storm/nimbus/WorkerHeartbeatsRecoveryStrategyFactory.java ---
    @@ -0,0 +1,44 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.nimbus;
    +
    +import com.google.common.base.Preconditions;
    +import org.apache.storm.Config;
    +import org.apache.storm.utils.ReflectionUtils;
    +
    +import java.util.Map;
    +
    +/**
    + * Factory class for recovery strategy.
    + */
    +public class WorkerHeartbeatsRecoveryStrategyFactory {
    +
    +    public static IWorkerHeartbeatsRecoveryStrategy getStrategy(Map conf) {
    +        if (conf.get(Config.NIMBUS_WORKER_HEARTBEATS_RECOVERY_STRATEGY_CLASS) != null) {
    +            Object targetObj = ReflectionUtils.newInstance((String) conf.get(Config.NIMBUS_WORKER_HEARTBEATS_RECOVERY_STRATEGY_CLASS));
    +            Preconditions.checkState(targetObj instanceof IWorkerHeartbeatsRecoveryStrategy, "{} must implements IWorkerHeartbeatsRecoveryStrategy", Config.NIMBUS_WORKER_HEARTBEATS_RECOVERY_STRATEGY_CLASS);
    +            ((IWorkerHeartbeatsRecoveryStrategy) targetObj).prepare(conf);
    --- End diff --
    
    Two lines looks duplicated. We could remove duplicated two lines via following:
    
    ```
    IWorkerHeartbeatsRecoveryStrategy strategy;
    if (...) {
       ...
       strategy = (IWorkerHeartbeatsRecoveryStrategy) targetObj;
    } else {
       strategy = new TimeOutWorkerHeartbeatsRecoveryStrategy();
    }
    strategy.prepare(conf);
    return strategy;
    ```


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r158704904
  
    --- Diff: storm-client/src/jvm/org/apache/storm/assignments/InMemoryAssignmentBackend.java ---
    @@ -0,0 +1,152 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.assignments;
    +
    +import java.util.ArrayList;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.storm.cluster.ClusterUtils;
    +import org.apache.storm.generated.Assignment;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * An assignment backend which will keep all assignments and id-info in memory. Only used if no backend is specified internal.
    + */
    +public class InMemoryAssignmentBackend implements ILocalAssignmentsBackend {
    +    private static final Logger LOG = LoggerFactory.getLogger(InMemoryAssignmentBackend.class);
    +
    +    protected Map<String, Assignment> idToAssignment;
    +    protected Map<String, String> idToName;
    +    protected Map<String, String> nameToId;
    +    private volatile boolean isSynchronized = false;
    +
    +    public InMemoryAssignmentBackend() {}
    +
    +    @Override
    +    public boolean isSynchronized() {
    +        return this.isSynchronized;
    +    }
    +
    +    @Override
    +    public void setSynchronized() {
    +        this.isSynchronized = true;
    +    }
    +
    +    @Override
    +    public void prepare(Map conf) {
    +        // do nothing for conf now
    +        this.idToAssignment = new ConcurrentHashMap<>();
    +        this.idToName = new ConcurrentHashMap<>();
    +        this.nameToId = new ConcurrentHashMap<>();
    +    }
    +
    +    @Override
    +    public void keepOrUpdateAssignment(String stormId, Assignment assignment) {
    +        this.idToAssignment.put(stormId, assignment);
    +    }
    +
    +    @Override
    +    public Assignment getAssignment(String stormId) {
    +        return this.idToAssignment.get(stormId);
    +    }
    +
    +    @Override
    +    public void removeAssignment(String stormId) {
    +        this.idToAssignment.remove(stormId);
    +    }
    +
    +    @Override
    +    public List<String> assignments() {
    +        if(idToAssignment == null) {
    +            return new ArrayList<>();
    +        }
    +        List<String> ret = new ArrayList<>();
    +        ret.addAll(this.idToAssignment.keySet());
    +        return ret;
    +    }
    +
    +    @Override
    +    public Map<String, Assignment> assignmentsInfo() {
    +        Map<String, Assignment> ret = new HashMap<>();
    +        ret.putAll(this.idToAssignment);
    +
    +        return ret;
    +    }
    +
    +    @Override
    +    public void syncRemoteAssignments(Map<String, byte[]> remote) {
    +        Map<String, Assignment> tmp = new ConcurrentHashMap<>();
    --- End diff --
    
    It was just a HashMap while I reviewed, and it's changed to ConcurrentHashMap. Is there any finding on thread-safety?


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by erikdw <gi...@git.apache.org>.
Github user erikdw commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @HeartSaVioR : really @revans2 noticed the change's implications for storm-on-mesos, so he should get the credit. :-)  He's rightly suggested that we create some tests to codify everything that might break us -- that is understandably difficult, but I'm collecting a list of things we'll need to test for.
    
    And awesome, `-c supervisor.thrift.port=<port>` will work great for us, thanks for confirming!  (I was able to look through the `bin/storm.py` code and see that the `-c/--config` handling ends up putting elements into `-Dstorm.options=` for any executed storm java class.)


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r153774053
  
    --- Diff: storm-server/src/main/java/org/apache/storm/daemon/supervisor/Supervisor.java ---
    @@ -228,6 +257,51 @@ private void launchDaemon() {
             }
         }
     
    +    private void launchSupervisorThriftServer(Map conf) throws IOException {
    +        // validate port
    +        try {
    +            ServerSocket socket = new ServerSocket(ObjectReader.getInt(conf.get(Config.SUPERVISOR_THRIFT_PORT)));
    --- End diff --
    
    yeah


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r158768248
  
    --- Diff: storm-server/src/main/java/org/apache/storm/nimbus/TimeOutWorkerHeartbeatsRecoveryStrategy.java ---
    @@ -0,0 +1,86 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.nimbus;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.utils.ObjectReader;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.HashSet;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * Wait for a node to report worker heartbeats until a configured timeout. For cases below we have strategies:
    + * <p>
    + * 1: When nimbus gains leader ship, it will decide if the heartbeats are ready based on the reported node ids,
    + * supervisors/nodes will take care of the worker heartbeats recovery, a reported node id means all the workers heartbeats on the node are reported.
    + * <p>
    + * 2: If several supervisor also crush and will never recover[or all crush for some unknown reason],
    + * workers will report their heartbeats directly to master, so it has not any effect.
    + */
    +public class TimeOutWorkerHeartbeatsRecoveryStrategy implements IWorkerHeartbeatsRecoveryStrategy {
    +    private static final Logger LOG = LoggerFactory.getLogger(TimeOutWorkerHeartbeatsRecoveryStrategy.class);
    +
    +    private static int NODE_MAX_TIMEOUT_SECS = 600;
    +
    +    private long startTimeSecs;
    +
    +    private Set<String> reportedIDs;
    --- End diff --
    
    done


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152902020
  
    --- Diff: storm-server/src/main/java/org/apache/storm/nimbus/TimeOutWorkerHeartbeatsRecoveryStrategy.java ---
    @@ -0,0 +1,86 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.nimbus;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.utils.ObjectReader;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.HashSet;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * Wait for a node to report worker heartbeats until a configured timeout. For cases below we have strategies:
    + * <p>
    + * 1: When nimbus gains leader ship, it will decide if the heartbeats are ready based on the reported node ids,
    + * supervisors/nodes will take care of the worker heartbeats recovery, a reported node id means all the workers heartbeats on the node are reported.
    + * <p>
    + * 2: If several supervisor also crush and will never recover[or all crush for some unknown reason],
    + * workers will report their heartbeats directly to master, so it has not any effect.
    + */
    +public class TimeOutWorkerHeartbeatsRecoveryStrategy implements IWorkerHeartbeatsRecoveryStrategy {
    +    private static final Logger LOG = LoggerFactory.getLogger(TimeOutWorkerHeartbeatsRecoveryStrategy.class);
    +
    +    private static int NODE_MAX_TIMEOUT_SECS = 600;
    +
    +    private long startTimeSecs;
    +
    +    private Set<String> reportedIDs;
    --- End diff --
    
    `reportedIDs` -> `reportedIds`


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @HeartSaVioR 
    Yes, this is the case, but i don't suggest to mix two kinds of heartbeats aware logics in Nimbus, it's too heavy for scheduling[with old workers] and this is not the intention of this patch.
    
    I don't think it is worthy of making the mix for the little difference of workers.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by erikdw <gi...@git.apache.org>.
Github user erikdw commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @HeartSaVioR : 
    
    > By the way, I was not aware of the discussion in storm-mesos, so don't know which works should be done in Storm side, and how these are coupled with "this issue". Maybe better to only pointing out relevant things, and discuss in dev. mailing list if necessary for things out of topic.
    
    I think you're misinterpreting [Jessica's comments](https://github.com/apache/storm/pull/2433#issuecomment-357140893).  She wasn't trying to bog down this discussion with an attempt to resolve the other breakages of storm-on-mesos, I believe she was just providing some context to Bobby's early statement](https://github.com/apache/storm/pull/2433#issuecomment-356661822) about storm-on-mesos being broken a few times by changes like this in storm-core.  Furthermore, I believe she was also clarifying that storm-on-mesos is currently *totally* broken (as explained in the context she provided) in the 1.1+ branches, so this proposed change technically is not *breaking* storm-on-mesos, since it's already broken.  Our goal is simply to prevent even more breaking changes.
    
    > As far as I understand in your comment, only concern with this issue is specifying Supervisor's thrift port, which shouldn't be random in range but just using specified port. If I understand correctly, the patch already does that (via configuration), and storm-mesos could launch Supervisor instance with overriding supervisor thrift port. Makes sense?
    
    That has potential to work -- can you please clarify something though?  Is it possible to specify this setting (`supervisor.thrift.port`) as a CLI parameter to the supervisor as it is launched?  If that works then awesome, that means the requirement Jessica outlined is already satisfied since we can simply specify that setting when we launch each supervisor.  However, if the option instead must be in the storm configuration yaml file, then it is insufficient.  That is because storm-on-mesos *must* be able to have different ports for every supervisor, but every supervisor shares the same config file in storm-on-mesos.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152706310
  
    --- Diff: storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java ---
    @@ -71,69 +75,17 @@
     import org.apache.storm.daemon.DaemonCommon;
     import org.apache.storm.daemon.Shutdownable;
     import org.apache.storm.daemon.StormCommon;
    -import org.apache.storm.generated.AlreadyAliveException;
    -import org.apache.storm.generated.Assignment;
    -import org.apache.storm.generated.AuthorizationException;
    -import org.apache.storm.generated.BeginDownloadResult;
    -import org.apache.storm.generated.ClusterSummary;
    -import org.apache.storm.generated.CommonAggregateStats;
    -import org.apache.storm.generated.ComponentAggregateStats;
    -import org.apache.storm.generated.ComponentPageInfo;
    -import org.apache.storm.generated.ComponentType;
    -import org.apache.storm.generated.Credentials;
    -import org.apache.storm.generated.DebugOptions;
    -import org.apache.storm.generated.ErrorInfo;
    -import org.apache.storm.generated.ExecutorInfo;
    -import org.apache.storm.generated.ExecutorStats;
    -import org.apache.storm.generated.ExecutorSummary;
    -import org.apache.storm.generated.GetInfoOptions;
    -import org.apache.storm.generated.InvalidTopologyException;
    -import org.apache.storm.generated.KeyAlreadyExistsException;
    -import org.apache.storm.generated.KeyNotFoundException;
    -import org.apache.storm.generated.KillOptions;
    -import org.apache.storm.generated.LSTopoHistory;
    -import org.apache.storm.generated.ListBlobsResult;
    -import org.apache.storm.generated.LogConfig;
    -import org.apache.storm.generated.LogLevel;
    -import org.apache.storm.generated.LogLevelAction;
    +import org.apache.storm.daemon.supervisor.*;
    --- End diff --
    
    avoid wildcard import


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @danny0405 Please let me know when you are finished and the PR is ready to review again. Thanks a lot for the patience!


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152703971
  
    --- Diff: storm-client/src/jvm/org/apache/storm/stats/StatsUtil.java ---
    @@ -1255,11 +1227,11 @@ public static ComponentPageInfo aggCompExecsStats(
         /**
          * aggregate statistics per worker for a topology. Optionally filtering on specific supervisors
          *
    -     * @param topologyId       topology id
    -     * @param topology         storm topology
    -     * @param task2component   a Map of {task id -> component}
    +     * @param stormId       topology id
    --- End diff --
    
    Nice catch!


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @danny0405 
    TODO 2: That's same as my understanding. This patch becomes depending on https://issues.apache.org/jira/browse/STORM-2898. If you are familiar with security please take a look at the issue and comment as well.
    
    TODO 3: Yes I'm willing to do it, but sadly I have some higher priority and even urgent works outside of Storm project, so I can't promise I will start it in this month. 
    (There's a holiday week in China and Korea in next month btw.) 
    
    Btw, I'm not sure but I guess STORM-2898 would take even couple of months in worst case, so I guess it is OK.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @JessicaLHartog 
    Thanks for noticing! By the way, I was not aware of the discussion in storm-mesos, so don't know which works should be done in Storm side, and how these are coupled with this issue. 
    
    As far as I understand in your comment, only concern with this issue is specifying Supervisor's thrift port, which shouldn't be random in range but just using specified port. If I understand correctly, the patch already does that (via configuration), and storm-mesos could launch Supervisor instance with overriding supervisor thrift port. Makes sense?
    
    cc. @danny0405 FYI.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @danny0405 
    You may also want to make sure the squashed commit contains full diff via comparing contents of commit with PR diff. Once line count goes similar and UTs / manual tests work it will be safe.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152702114
  
    --- Diff: storm-client/src/jvm/org/apache/storm/assignments/InMemoryAssignmentBackend.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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.assignments;
    +
    +import org.apache.storm.cluster.ClusterUtils;
    +import org.apache.storm.generated.Assignment;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +/**
    + * An assignment backend which will keep all assignments and id-info in memory. Only used if no backend is specified internal.
    + */
    +public class InMemoryAssignmentBackend implements ILocalAssignmentsBackend {
    +    private static final Logger LOG = LoggerFactory.getLogger(InMemoryAssignmentBackend.class);
    +
    +    protected Map<String, Assignment> idToAssignment;
    +    protected Map<String, String> idToName;
    +    protected Map<String, String> nameToId;
    +    /**
    +     * Used for assignments set/get, assignments set/get should be kept thread safe
    +     */
    +    private final Object assignmentsLock = new Object();
    +
    +    public InMemoryAssignmentBackend() {}
    +
    +    @Override
    +    public void prepare(Map conf) {
    +        // do nothing for conf now
    +        this.idToAssignment = new HashMap<>();
    +        this.idToName = new HashMap<>();
    +        this.nameToId = new HashMap<>();
    +    }
    +
    +    @Override
    +    public void keepOrUpdateAssignment(String stormID, Assignment assignment) {
    +        synchronized (assignmentsLock) {
    +            this.idToAssignment.put(stormID, assignment);
    +        }
    +    }
    +
    +    @Override
    +    public Assignment getAssignment(String stormID) {
    +        synchronized (assignmentsLock) {
    +            return this.idToAssignment.get(stormID);
    +        }
    +    }
    +
    +    @Override
    +    public void removeAssignment(String stormID) {
    +        synchronized (assignmentsLock) {
    +            this.idToAssignment.remove(stormID);
    +        }
    +    }
    +
    +    @Override
    +    public List<String> assignments() {
    +        if(idToAssignment == null) {
    +            return new ArrayList<>();
    +        }
    +        List<String> ret = new ArrayList<>();
    +        synchronized (assignmentsLock) {
    +            ret.addAll(this.idToAssignment.keySet());
    +            return ret;
    +        }
    +    }
    +
    +    @Override
    +    public Map<String, Assignment> assignmentsInfo() {
    +        Map<String, Assignment> ret = new HashMap<>();
    +        synchronized (assignmentsLock) {
    +            ret.putAll(this.idToAssignment);
    +        }
    +
    +        return ret;
    +    }
    +
    +    @Override
    +    public void syncRemoteAssignments(Map<String, byte[]> remote) {
    +        Map<String, Assignment> tmp = new HashMap<>();
    +        for(Map.Entry<String, byte[]> entry: remote.entrySet()) {
    +            tmp.put(entry.getKey(), ClusterUtils.maybeDeserialize(entry.getValue(), Assignment.class));
    +        }
    +        this.idToAssignment = tmp;
    +    }
    +
    +    @Override
    +    public void keepStormId(String stormName, String stormID) {
    +        this.nameToId.put(stormName, stormID);
    +        this.idToName.put(stormID, stormName);
    +    }
    +
    +    @Override
    +    public String getStormId(String stormName) {
    +        return this.nameToId.get(stormName);
    +    }
    +
    +    @Override
    +    public void syncRemoteIDS(Map<String, String> remote) {
    +        Map<String, String> tmpNameToID = new HashMap<>();
    --- End diff --
    
    ID -> Id


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152703902
  
    --- Diff: storm-client/src/jvm/org/apache/storm/stats/StatsUtil.java ---
    @@ -21,43 +21,15 @@
     
     import org.apache.storm.cluster.ExecutorBeat;
     import org.apache.storm.cluster.IStormClusterState;
    -import org.apache.storm.generated.Bolt;
    -import org.apache.storm.generated.BoltAggregateStats;
    -import org.apache.storm.generated.BoltStats;
    -import org.apache.storm.generated.ClusterWorkerHeartbeat;
    -import org.apache.storm.generated.CommonAggregateStats;
    -import org.apache.storm.generated.ComponentAggregateStats;
    -import org.apache.storm.generated.ComponentPageInfo;
    -import org.apache.storm.generated.ComponentType;
    -import org.apache.storm.generated.ErrorInfo;
    -import org.apache.storm.generated.ExecutorAggregateStats;
    -import org.apache.storm.generated.ExecutorInfo;
    -import org.apache.storm.generated.ExecutorSpecificStats;
    -import org.apache.storm.generated.ExecutorStats;
    -import org.apache.storm.generated.ExecutorSummary;
    -import org.apache.storm.generated.GlobalStreamId;
    -import org.apache.storm.generated.SpecificAggregateStats;
    -import org.apache.storm.generated.SpoutAggregateStats;
    -import org.apache.storm.generated.SpoutStats;
    -import org.apache.storm.generated.StormTopology;
    -import org.apache.storm.generated.TopologyPageInfo;
    -import org.apache.storm.generated.TopologyStats;
    -import org.apache.storm.generated.WorkerResources;
    -import org.apache.storm.generated.WorkerSummary;
    +import org.apache.storm.generated.*;
     import org.apache.storm.scheduler.WorkerSlot;
     import org.apache.storm.utils.Utils;
     import org.apache.storm.utils.Time;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
    -import java.util.ArrayList;
    -import java.util.HashMap;
    -import java.util.HashSet;
    -import java.util.Iterator;
    -import java.util.List;
    -import java.util.Map;
    +import java.util.*;
    --- End diff --
    
    Same here.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r158714261
  
    --- Diff: storm-client/src/jvm/org/apache/storm/assignments/LocalAssignmentsBackendFactory.java ---
    @@ -0,0 +1,48 @@
    +/**
    + * 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.assignments;
    +
    +import com.google.common.base.Preconditions;
    +import org.apache.storm.Config;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.ReflectionUtils;
    +
    +import java.util.Map;
    +
    +/**
    + * Factory class for creating local assignments.
    + */
    +public class LocalAssignmentsBackendFactory {
    +
    +    public static ILocalAssignmentsBackend getBackend(Map<String, Object> conf) {
    +        if (conf.get(Config.NIMBUS_LOCAL_ASSIGNMENTS_BACKEND_CLASS) != null) {
    +            Object targetObj = ReflectionUtils.newInstance((String) conf.get(Config.NIMBUS_LOCAL_ASSIGNMENTS_BACKEND_CLASS));
    +            Preconditions.checkState(targetObj instanceof ILocalAssignmentsBackend, "{} must implements ILocalAssignmentsBackend", Config.NIMBUS_LOCAL_ASSIGNMENTS_BACKEND_CLASS);
    +            ((ILocalAssignmentsBackend)targetObj).prepare(conf);
    +            return (ILocalAssignmentsBackend) targetObj;
    +        }
    +
    +        return getDefault();
    --- End diff --
    
    i wanna reuse conf too, but it is invoked by ConfigUtils, and no conf passed in by outer method.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152902125
  
    --- Diff: storm-server/src/main/java/org/apache/storm/nimbus/TimeOutWorkerHeartbeatsRecoveryStrategy.java ---
    @@ -0,0 +1,86 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.nimbus;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.utils.ObjectReader;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.HashSet;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * Wait for a node to report worker heartbeats until a configured timeout. For cases below we have strategies:
    + * <p>
    + * 1: When nimbus gains leader ship, it will decide if the heartbeats are ready based on the reported node ids,
    + * supervisors/nodes will take care of the worker heartbeats recovery, a reported node id means all the workers heartbeats on the node are reported.
    + * <p>
    + * 2: If several supervisor also crush and will never recover[or all crush for some unknown reason],
    + * workers will report their heartbeats directly to master, so it has not any effect.
    + */
    +public class TimeOutWorkerHeartbeatsRecoveryStrategy implements IWorkerHeartbeatsRecoveryStrategy {
    +    private static final Logger LOG = LoggerFactory.getLogger(TimeOutWorkerHeartbeatsRecoveryStrategy.class);
    +
    +    private static int NODE_MAX_TIMEOUT_SECS = 600;
    +
    +    private long startTimeSecs;
    +
    +    private Set<String> reportedIDs;
    +
    +    @Override
    +    public void prepare(Map conf) {
    +        NODE_MAX_TIMEOUT_SECS = ObjectReader.getInt(conf.get(Config.SUPERVISOR_WORKER_HEARTBEATS_MAX_TIMEOUT_SECS), 600);
    +        this.startTimeSecs = System.currentTimeMillis() / 1000L;
    +        this.reportedIDs = new HashSet<>();
    +    }
    +
    +    @Override
    +    public boolean isReady(Set<String> nodeIds) {
    +        if (isMaxTimeOut()) {
    +            HashSet<String> tmp = new HashSet<>();
    +            for(String nodeID : nodeIds) {
    +                if (!this.reportedIDs.contains(nodeID))
    --- End diff --
    
    indentation messed up: we encourage always use {} even it is a single line.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r173215759
  
    --- Diff: storm-client/src/jvm/org/apache/storm/daemon/worker/Worker.java ---
    @@ -392,6 +401,30 @@ public void establishLogSettingCallback() {
             workerState.stormClusterState.topologyLogConfig(topologyId, this::checkLogConfigChanged);
         }
     
    +    /**
    +     * Send a heartbeat to local supervisor first to check if supervisor is ok for heartbeating.
    +     */
    +    private void heartbeatToMasterIfLocalbeatFail(LSWorkerHeartbeat lsWorkerHeartbeat) {
    +        if (ConfigUtils.isLocalMode(this.conf)) {
    +            return;
    +        }
    +        //in distributed mode, send heartbeat directly to master if local supervisor goes down
    +        SupervisorWorkerHeartbeat workerHeartbeat = new SupervisorWorkerHeartbeat(lsWorkerHeartbeat.get_topology_id(),
    +                lsWorkerHeartbeat.get_executors(), lsWorkerHeartbeat.get_time_secs());
    +        try (SupervisorClient client = SupervisorClient.getConfiguredClient(conf, Utils.hostname(), supervisorPort)){
    +            client.getClient().sendSupervisorWorkerHeartbeat(workerHeartbeat);
    +        } catch (Throwable tr1) {
    +            //if any error/exception thrown, report directly to nimbus.
    +            LOG.warn("Exception when send heartbeat to local supervisor", tr1.getMessage());
    +            try (NimbusClient nimbusClient = NimbusClient.getConfiguredClient(conf)){
    +                nimbusClient.getClient().sendSupervisorWorkerHeartbeat(workerHeartbeat);
    +            } catch (Throwable tr2) {
    --- End diff --
    
    Here too on Exception vs Throwable.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r163215412
  
    --- Diff: storm-client/src/storm.thrift ---
    @@ -836,3 +848,33 @@ exception HBAuthorizationException {
     exception HBExecutionException {
       1: required string msg;
     }
    +
    +struct SupervisorAssignments {
    +    1: optional map<string, Assignment> storm_assignment = {}
    +}
    +
    +service Supervisor {
    +  /**
    +   * Send node specific assignments to supervisor
    +   */
    +  void sendSupervisorAssignments(1: SupervisorAssignments assignments) throws (1: AuthorizationException aze);
    +  /**
    +   * Get local assignment for a storm
    +   */
    +  Assignment getLocalAssignmentForStorm(1: string id) throws (1: NotAliveException e, 2: AuthorizationException aze);
    +  /**
    +   * Send worker heartbeat to local supervisor
    +   */
    +  void sendSupervisorWorkerHeartbeat(1: SupervisorWorkerHeartbeat heartbeat) throws (1: AuthorizationException aze);
    +}
    +
    +struct SupervisorWorkerHeartbeat {
    --- End diff --
    
    @HeartSaVioR 
    I have tweaked the position of structs in storm.thrift.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r158705083
  
    --- Diff: storm-client/src/jvm/org/apache/storm/assignments/LocalAssignmentsBackendFactory.java ---
    @@ -0,0 +1,48 @@
    +/**
    + * 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.assignments;
    +
    +import com.google.common.base.Preconditions;
    +import org.apache.storm.Config;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.ReflectionUtils;
    +
    +import java.util.Map;
    +
    +/**
    + * Factory class for creating local assignments.
    + */
    +public class LocalAssignmentsBackendFactory {
    +
    +    public static ILocalAssignmentsBackend getBackend(Map<String, Object> conf) {
    +        if (conf.get(Config.NIMBUS_LOCAL_ASSIGNMENTS_BACKEND_CLASS) != null) {
    +            Object targetObj = ReflectionUtils.newInstance((String) conf.get(Config.NIMBUS_LOCAL_ASSIGNMENTS_BACKEND_CLASS));
    +            Preconditions.checkState(targetObj instanceof ILocalAssignmentsBackend, "{} must implements ILocalAssignmentsBackend", Config.NIMBUS_LOCAL_ASSIGNMENTS_BACKEND_CLASS);
    +            ((ILocalAssignmentsBackend)targetObj).prepare(conf);
    +            return (ILocalAssignmentsBackend) targetObj;
    +        }
    +
    +        return getDefault();
    --- End diff --
    
    As I've left comment earlier, we could just pass `conf` to getDefault() to avoid re-read storm conf.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152901995
  
    --- Diff: storm-server/src/main/java/org/apache/storm/nimbus/TimeOutWorkerHeartbeatsRecoveryStrategy.java ---
    @@ -0,0 +1,86 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.nimbus;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.utils.ObjectReader;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.HashSet;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * Wait for a node to report worker heartbeats until a configured timeout. For cases below we have strategies:
    + * <p>
    + * 1: When nimbus gains leader ship, it will decide if the heartbeats are ready based on the reported node ids,
    + * supervisors/nodes will take care of the worker heartbeats recovery, a reported node id means all the workers heartbeats on the node are reported.
    + * <p>
    + * 2: If several supervisor also crush and will never recover[or all crush for some unknown reason],
    + * workers will report their heartbeats directly to master, so it has not any effect.
    + */
    +public class TimeOutWorkerHeartbeatsRecoveryStrategy implements IWorkerHeartbeatsRecoveryStrategy {
    +    private static final Logger LOG = LoggerFactory.getLogger(TimeOutWorkerHeartbeatsRecoveryStrategy.class);
    +
    +    private static int NODE_MAX_TIMEOUT_SECS = 600;
    +
    +    private long startTimeSecs;
    +
    +    private Set<String> reportedIDs;
    +
    +    @Override
    +    public void prepare(Map conf) {
    +        NODE_MAX_TIMEOUT_SECS = ObjectReader.getInt(conf.get(Config.SUPERVISOR_WORKER_HEARTBEATS_MAX_TIMEOUT_SECS), 600);
    +        this.startTimeSecs = System.currentTimeMillis() / 1000L;
    +        this.reportedIDs = new HashSet<>();
    +    }
    +
    +    @Override
    +    public boolean isReady(Set<String> nodeIds) {
    +        if (isMaxTimeOut()) {
    +            HashSet<String> tmp = new HashSet<>();
    +            for(String nodeID : nodeIds) {
    +                if (!this.reportedIDs.contains(nodeID))
    +                tmp.add(nodeID);
    +            }
    +            LOG.warn("Failed to recover heartbeats for nodes: {} with timeout {}s", tmp, NODE_MAX_TIMEOUT_SECS);
    +            return true;
    +        }
    +        for (String nodeID : nodeIds) {
    +            if (this.reportedIDs.contains(nodeID)) {
    +                continue;
    +            } else {
    +                return false;
    +            }
    +
    +        }
    +
    +        return true;
    +    }
    +
    +    @Override
    +    public void reportNodeId(String nodeId) {
    +        this.reportedIDs.add(nodeId);
    +    }
    +
    +    private boolean isMaxTimeOut() {
    +        return (System.currentTimeMillis() / 1000L - this.startTimeSecs) > NODE_MAX_TIMEOUT_SECS;
    --- End diff --
    
    Same here: Let's use Times.currentTimeMillis() for being compatible with simulating time.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152698363
  
    --- Diff: storm-client/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java ---
    @@ -147,21 +153,52 @@ protected void issueMapCallback(ConcurrentHashMap<String, Runnable> callbackConc
     
         @Override
         public List<String> assignments(Runnable callback) {
    +        //deprecated
    --- End diff --
    
    What we would want to deprecate? If we want to deprecate kind of public API, we should announce it to users (marking deprecated annotation, release note, etc) or if it is Storm internal, we could just remove it.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    Please fix the checkstyle issue on `storm-client` or "temporary" change max violation count so that Travis CI can build the code properly. Please also note that max violation count for all of modules should not increase before merging this in.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r153778405
  
    --- Diff: storm-server/src/main/java/org/apache/storm/nimbus/TimeOutWorkerHeartbeatsRecoveryStrategy.java ---
    @@ -0,0 +1,86 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.nimbus;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.utils.ObjectReader;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.HashSet;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * Wait for a node to report worker heartbeats until a configured timeout. For cases below we have strategies:
    + * <p>
    + * 1: When nimbus gains leader ship, it will decide if the heartbeats are ready based on the reported node ids,
    + * supervisors/nodes will take care of the worker heartbeats recovery, a reported node id means all the workers heartbeats on the node are reported.
    + * <p>
    + * 2: If several supervisor also crush and will never recover[or all crush for some unknown reason],
    + * workers will report their heartbeats directly to master, so it has not any effect.
    + */
    +public class TimeOutWorkerHeartbeatsRecoveryStrategy implements IWorkerHeartbeatsRecoveryStrategy {
    +    private static final Logger LOG = LoggerFactory.getLogger(TimeOutWorkerHeartbeatsRecoveryStrategy.class);
    +
    +    private static int NODE_MAX_TIMEOUT_SECS = 600;
    +
    +    private long startTimeSecs;
    +
    +    private Set<String> reportedIDs;
    +
    +    @Override
    +    public void prepare(Map conf) {
    +        NODE_MAX_TIMEOUT_SECS = ObjectReader.getInt(conf.get(Config.SUPERVISOR_WORKER_HEARTBEATS_MAX_TIMEOUT_SECS), 600);
    +        this.startTimeSecs = System.currentTimeMillis() / 1000L;
    --- End diff --
    
    ok


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r173214066
  
    --- Diff: storm-client/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java ---
    @@ -745,6 +828,7 @@ public void disconnect() {
             stateStorage.unregister(stateId);
             if (solo) {
                 stateStorage.close();
    +        this.assignmentsBackend.dispose();
    --- End diff --
    
    nit: spacing.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152896601
  
    --- Diff: storm-server/src/main/java/org/apache/storm/daemon/supervisor/Supervisor.java ---
    @@ -228,6 +257,51 @@ private void launchDaemon() {
             }
         }
     
    +    private void launchSupervisorThriftServer(Map conf) throws IOException {
    +        // validate port
    +        try {
    +            ServerSocket socket = new ServerSocket(ObjectReader.getInt(conf.get(Config.SUPERVISOR_THRIFT_PORT)));
    --- End diff --
    
    Maybe better to extract a variable for `conf.get(Config.SUPERVISOR_THRIFT_PORT)` given that it's used for 3 times in several lines.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r174188374
  
    --- Diff: storm-client/src/jvm/org/apache/storm/cluster/IStormClusterState.java ---
    @@ -233,23 +249,11 @@
          * @return the id of the topology or null if it is not alive.
          */
         default Optional<String> getTopoId(final String topologyName) {
    --- End diff --
    
    It's a wrapper func which change the result to be of an `Option` type.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @danny0405 
    OK. I have been switching the context from multiple projects/issues recent days so may miss the detail on this patch, and even confusion with current implementation (sad).
    
    Could you verify if I understand your statements correctly?
    
    1. Older Worker is already leaving heartbeats to local state and newer Supervisor can leverage them to report to newer Nimbus hence no need to do additional work on that.
    2. Older worker can't report its heartbeat to newer Nimbus directly, hence newer Nimbus can't get older workers' heartbeat if newer supervisor is down. <= This would be a major difference between older worker and newer worker for this patch.
    
    If my understanding is right, looking into ZK for fail-back mechanism (for workers which relevant supervisor is down) might still make sense for old workers, which work would be not easy. 
    If it is a hard requirement, let's not be smart for old workers. If we can identify topology version is under 2.0.0, just ignore heartbeats supervisor is reporting and read heartbeats from ZK. This will get rid of headache between aggregation between supervisor RPC and ZK.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152701486
  
    --- Diff: storm-client/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java ---
    @@ -243,6 +280,25 @@ public StormBase stormBase(String stormId, Runnable callback) {
             return ClusterUtils.maybeDeserialize(stateStorage.get_data(ClusterUtils.stormPath(stormId), callback != null), StormBase.class);
         }
     
    +    @Override
    +    public String stormId(String stormName) {
    +        return this.backend.getStormId(stormName);
    +    }
    +
    +    @Override
    +    public void syncRemoteIds(Map<String, String> remote) {
    +        if (null != remote) {
    +            this.backend.syncRemoteIDS(remote);
    +        }else {
    +            Map<String, String> tmp = new HashMap<>();
    +            List<String> activeStorms = activeStorms();
    +            for (String stormID: activeStorms) {
    --- End diff --
    
    Same here: `stormID` -> `stormId`


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    I also noticed that the unit tests are taking much much longer to run with this patch.  33 mins vs 18 without it.  I am going to spend some time investigating that too.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152902065
  
    --- Diff: storm-server/src/main/java/org/apache/storm/nimbus/TimeOutWorkerHeartbeatsRecoveryStrategy.java ---
    @@ -0,0 +1,86 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.nimbus;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.utils.ObjectReader;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.HashSet;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * Wait for a node to report worker heartbeats until a configured timeout. For cases below we have strategies:
    + * <p>
    + * 1: When nimbus gains leader ship, it will decide if the heartbeats are ready based on the reported node ids,
    + * supervisors/nodes will take care of the worker heartbeats recovery, a reported node id means all the workers heartbeats on the node are reported.
    + * <p>
    + * 2: If several supervisor also crush and will never recover[or all crush for some unknown reason],
    + * workers will report their heartbeats directly to master, so it has not any effect.
    + */
    +public class TimeOutWorkerHeartbeatsRecoveryStrategy implements IWorkerHeartbeatsRecoveryStrategy {
    +    private static final Logger LOG = LoggerFactory.getLogger(TimeOutWorkerHeartbeatsRecoveryStrategy.class);
    +
    +    private static int NODE_MAX_TIMEOUT_SECS = 600;
    +
    +    private long startTimeSecs;
    +
    +    private Set<String> reportedIDs;
    +
    +    @Override
    +    public void prepare(Map conf) {
    +        NODE_MAX_TIMEOUT_SECS = ObjectReader.getInt(conf.get(Config.SUPERVISOR_WORKER_HEARTBEATS_MAX_TIMEOUT_SECS), 600);
    +        this.startTimeSecs = System.currentTimeMillis() / 1000L;
    +        this.reportedIDs = new HashSet<>();
    +    }
    +
    +    @Override
    +    public boolean isReady(Set<String> nodeIds) {
    +        if (isMaxTimeOut()) {
    --- End diff --
    
    `exceedsMaxTimeout()`?


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @revans2 
    I'm going through workarounds here. Could you review this as well, and provide review comments/suggestions? It would be really appreciated if you could test the patch against old workers.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r173211013
  
    --- Diff: storm-client/src/jvm/org/apache/storm/assignments/ILocalAssignmentsBackend.java ---
    @@ -0,0 +1,113 @@
    +/**
    + * 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.assignments;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.storm.generated.Assignment;
    +
    +/**
    + * Interface for storing local assignments.
    + */
    +public interface ILocalAssignmentsBackend {
    +    /**
    +     * Decide if the assignments is synchronized from remote state-store.
    +     */
    +    boolean isSynchronized();
    +
    +    /**
    +     * Mark this backend as synchronized when sync work is done.
    +     */
    +    void setSynchronized();
    +
    +    /**
    +     * Initial function for creating backend.
    +     * @param conf config
    +     */
    +    void prepare(Map conf);
    +
    +    /**
    +     * Keep a storm assignment to local state or update old assignment.
    +     * @param stormID storm runtime id
    +     * @param assignment assignment as thrift
    +     */
    +    void keepOrUpdateAssignment(String stormID, Assignment assignment);
    +
    +    /**
    +     * Get assignment as {@link Assignment} for a storm.
    +     * @param stormId storm runtime id
    +     * @return assignment
    +     */
    +    Assignment getAssignment(String stormId);
    +
    +    void removeAssignment(String stormId);
    +
    +    /**
    +     * List all the storm runtime ids of local assignments.
    +     * @return a list of storm ids
    +     */
    +    List<String> assignments();
    +
    +    /**
    +     * Get all the local assignments of local state.
    +     * @return mapping of storm-id -> assignment
    +     */
    +    Map<String, Assignment> assignmentsInfo();
    +
    +    /**
    +     * Sync remote assignments to local, if remote is null, we will sync it from zk.
    +     * @param remote specific remote assignments, if it is null, it will sync from zookeeper[only used for nimbus]
    +     */
    +    void syncRemoteAssignments(Map<String, byte[]> remote);
    +
    +    /**
    +     * Keep a mapping storm-name -> storm-id to local state.
    +     * @param stormName storm name
    +     * @param stormId storm runtime id
    +     */
    +    void keepStormId(String stormName, String stormId);
    +
    +    /**
    +     * Get storm runtime id from local.
    +     * @param stormName name of a storm
    +     * @return runtime storm id
    +     */
    +    String getStormId(String stormName);
    +
    +    /**
    +     * Sync remote storm ids to local, will just used for nimbus.
    +     * @param remote remote ids from state store
    +     */
    +    void syncRemoteIds(Map<String, String> remote);
    +
    +    void deleteStormId(String stormName);
    +
    +    /**
    +     * Clear all the state for a storm.
    +     * @param stormId storm id
    +     */
    +    void clearStateForStorm(String stormId);
    +
    +    /**
    +     * Function to release resource.
    +     */
    +    void dispose();
    --- End diff --
    
    nit:  Can we just make this extend AutoClosable instead?


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r177711557
  
    --- Diff: storm-client/src/jvm/org/apache/storm/cluster/IStormClusterState.java ---
    @@ -42,16 +42,44 @@
     public interface IStormClusterState {
         List<String> assignments(Runnable callback);
     
    +    /**
    +     * Get the assignment based on storm id from local backend.
    +     * @param stormId topology id
    +     * @param callback callback function
    +     * @return {@link Assignment}
    +     */
         Assignment assignmentInfo(String stormId, Runnable callback);
     
    +    /**
    +     * Get the assignment based on storm id from remote state store, eg: ZK.
    +     * @param stormId topology id
    +     * @param callback callback function
    +     * @return
    --- End diff --
    
    nit: `@return {@link Assignment}`


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @revans2 
    I'm feeling that the feature may effectively disable compatibility feature - Storm 2.0.0 daemons to work with Storm 1.x/0.10.x workers - since heartbeat mechanism is changed. I hope it is OK to all of us, and personally I don't want to hesitate introducing backward-incompatible changes in major version to not miss out valuable efforts.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    Manual tests with a node works nicely. (killing nimbus, killing supervisor, killing workers)
    Will work on manual tests with small cluster. (3 or 5 nodes)
    
    I still would want to wait for other reviewers to review the code before giving +1.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152716651
  
    --- Diff: storm-client/src/jvm/org/apache/storm/daemon/worker/Worker.java ---
    @@ -382,6 +374,33 @@ public void establishLogSettingCallback() {
             workerState.stormClusterState.topologyLogConfig(topologyId, this::checkLogConfigChanged);
         }
     
    +    /**
    +     * Send a heartbeat to local supervisor first to check if supervisor is ok for heartbeating.
    +     */
    +    private void heartbeatToMasterIfLocalbeatFail(LSWorkerHeartbeat lsWorkerHeartbeat) {
    --- End diff --
    
    Normally we use local-state and supervisor reporting heartbeats, only if when supervisor goes down, worker will reports heartbeats directly.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152897808
  
    --- Diff: storm-server/src/main/java/org/apache/storm/daemon/supervisor/timer/SynchronizeAssignments.java ---
    @@ -0,0 +1,133 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.daemon.supervisor.timer;
    +
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.daemon.supervisor.ReadClusterState;
    +import org.apache.storm.daemon.supervisor.Supervisor;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.SupervisorAssignments;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.NimbusClient;
    +import org.apache.storm.utils.Utils;
    +import org.apache.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.HashMap;
    +import java.util.Map;
    +
    +/**
    + * A runnable which will synchronize assignments to node local and then worker processes.
    + */
    +public class SynchronizeAssignments implements Runnable {
    +    private static final Logger LOG = LoggerFactory.getLogger(SynchronizeAssignments.class);
    +
    +    private Supervisor supervisor;
    +    private SupervisorAssignments assignments;
    +    private ReadClusterState readClusterState;
    +
    +    public SynchronizeAssignments(Supervisor supervisor, SupervisorAssignments assignments, ReadClusterState readClusterState) {
    +        this.supervisor = supervisor;
    +        this.assignments = assignments;
    +        this.readClusterState = readClusterState;
    +    }
    +
    +    @Override
    +    public void run() {
    +        // first sync assignments to local, then sync processes.
    +        if (null == assignments) {
    +            getAssignmentsFromMaster(this.supervisor.getConf(), this.supervisor.getStormClusterState(), this.supervisor.getAssignmentId());
    +        } else {
    +            assignedAssignmentsToLocal(this.supervisor.getStormClusterState(), assignments);
    +        }
    +        this.readClusterState.run();
    +    }
    +
    +    /**
    +     * Used by {@link Supervisor} to fetch assignments when start up.
    +     * @param supervisor
    +     */
    +    public void getAssignmentsFromMasterUntilSuccess(Supervisor supervisor) {
    +        boolean success = false;
    +        NimbusClient master;
    +        while (!success) {
    +            try {
    +                master = NimbusClient.getConfiguredClient(supervisor.getConf());
    +                SupervisorAssignments assignments = master.getClient().getSupervisorAssignments(supervisor.getAssignmentId());
    +                assignedAssignmentsToLocal(supervisor.getStormClusterState(), assignments);
    +                success = true;
    +                try {
    +                    master.close();
    +                } catch (Throwable t) {
    +                    LOG.warn("Close master client exception", t);
    +                }
    +            } catch (Exception t) {
    +                // just ignore the exception
    +            }
    +            if (!success) {
    +                LOG.info("Waiting for a success sync of assignments from master...");
    +                try {
    +                    Thread.sleep(5000l);
    +                } catch (InterruptedException e) {
    +                    throw new RuntimeException(e);
    +                }
    +            }
    +
    +        }
    +
    +    }
    +
    +    public void getAssignmentsFromMaster(Map conf, IStormClusterState clusterState, String node) {
    +        if(ConfigUtils.isLocalMode(conf)) {
    --- End diff --
    
    space between f and (


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152706237
  
    --- Diff: storm-server/src/main/java/org/apache/storm/LocalCluster.java ---
    @@ -48,35 +48,9 @@
     import org.apache.storm.daemon.supervisor.StandaloneSupervisor;
     import org.apache.storm.daemon.supervisor.Supervisor;
     import org.apache.storm.executor.LocalExecutor;
    -import org.apache.storm.generated.AlreadyAliveException;
    -import org.apache.storm.generated.AuthorizationException;
    -import org.apache.storm.generated.BeginDownloadResult;
    -import org.apache.storm.generated.ClusterSummary;
    -import org.apache.storm.generated.ComponentPageInfo;
    -import org.apache.storm.generated.Credentials;
    -import org.apache.storm.generated.GetInfoOptions;
    -import org.apache.storm.generated.InvalidTopologyException;
    -import org.apache.storm.generated.KeyAlreadyExistsException;
    -import org.apache.storm.generated.KeyNotFoundException;
    -import org.apache.storm.generated.KillOptions;
    -import org.apache.storm.generated.ListBlobsResult;
    -import org.apache.storm.generated.LogConfig;
    +import org.apache.storm.generated.*;
    --- End diff --
    
    avoid wildcard import


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152902522
  
    --- Diff: storm-server/src/main/java/org/apache/storm/nimbus/TimeOutWorkerHeartbeatsRecoveryStrategy.java ---
    @@ -0,0 +1,86 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.nimbus;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.utils.ObjectReader;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.HashSet;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * Wait for a node to report worker heartbeats until a configured timeout. For cases below we have strategies:
    + * <p>
    + * 1: When nimbus gains leader ship, it will decide if the heartbeats are ready based on the reported node ids,
    + * supervisors/nodes will take care of the worker heartbeats recovery, a reported node id means all the workers heartbeats on the node are reported.
    + * <p>
    + * 2: If several supervisor also crush and will never recover[or all crush for some unknown reason],
    + * workers will report their heartbeats directly to master, so it has not any effect.
    + */
    +public class TimeOutWorkerHeartbeatsRecoveryStrategy implements IWorkerHeartbeatsRecoveryStrategy {
    +    private static final Logger LOG = LoggerFactory.getLogger(TimeOutWorkerHeartbeatsRecoveryStrategy.class);
    +
    +    private static int NODE_MAX_TIMEOUT_SECS = 600;
    +
    +    private long startTimeSecs;
    +
    +    private Set<String> reportedIDs;
    +
    +    @Override
    +    public void prepare(Map conf) {
    +        NODE_MAX_TIMEOUT_SECS = ObjectReader.getInt(conf.get(Config.SUPERVISOR_WORKER_HEARTBEATS_MAX_TIMEOUT_SECS), 600);
    +        this.startTimeSecs = System.currentTimeMillis() / 1000L;
    +        this.reportedIDs = new HashSet<>();
    +    }
    +
    +    @Override
    +    public boolean isReady(Set<String> nodeIds) {
    +        if (isMaxTimeOut()) {
    +            HashSet<String> tmp = new HashSet<>();
    +            for(String nodeID : nodeIds) {
    +                if (!this.reportedIDs.contains(nodeID))
    +                tmp.add(nodeID);
    +            }
    +            LOG.warn("Failed to recover heartbeats for nodes: {} with timeout {}s", tmp, NODE_MAX_TIMEOUT_SECS);
    +            return true;
    +        }
    +        for (String nodeID : nodeIds) {
    --- End diff --
    
    We could simplify the logic:
    
    ```
    return nodeIds.stream().allMatch(id -> this.reportedIds.contains(id));
    ```


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152897992
  
    --- Diff: storm-server/src/main/java/org/apache/storm/daemon/supervisor/timer/SynchronizeAssignments.java ---
    @@ -0,0 +1,133 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.daemon.supervisor.timer;
    +
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.daemon.supervisor.ReadClusterState;
    +import org.apache.storm.daemon.supervisor.Supervisor;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.SupervisorAssignments;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.NimbusClient;
    +import org.apache.storm.utils.Utils;
    +import org.apache.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.HashMap;
    +import java.util.Map;
    +
    +/**
    + * A runnable which will synchronize assignments to node local and then worker processes.
    + */
    +public class SynchronizeAssignments implements Runnable {
    +    private static final Logger LOG = LoggerFactory.getLogger(SynchronizeAssignments.class);
    +
    +    private Supervisor supervisor;
    +    private SupervisorAssignments assignments;
    +    private ReadClusterState readClusterState;
    +
    +    public SynchronizeAssignments(Supervisor supervisor, SupervisorAssignments assignments, ReadClusterState readClusterState) {
    +        this.supervisor = supervisor;
    +        this.assignments = assignments;
    +        this.readClusterState = readClusterState;
    +    }
    +
    +    @Override
    +    public void run() {
    +        // first sync assignments to local, then sync processes.
    +        if (null == assignments) {
    +            getAssignmentsFromMaster(this.supervisor.getConf(), this.supervisor.getStormClusterState(), this.supervisor.getAssignmentId());
    +        } else {
    +            assignedAssignmentsToLocal(this.supervisor.getStormClusterState(), assignments);
    +        }
    +        this.readClusterState.run();
    +    }
    +
    +    /**
    +     * Used by {@link Supervisor} to fetch assignments when start up.
    +     * @param supervisor
    +     */
    +    public void getAssignmentsFromMasterUntilSuccess(Supervisor supervisor) {
    +        boolean success = false;
    +        NimbusClient master;
    +        while (!success) {
    +            try {
    +                master = NimbusClient.getConfiguredClient(supervisor.getConf());
    +                SupervisorAssignments assignments = master.getClient().getSupervisorAssignments(supervisor.getAssignmentId());
    +                assignedAssignmentsToLocal(supervisor.getStormClusterState(), assignments);
    +                success = true;
    +                try {
    +                    master.close();
    +                } catch (Throwable t) {
    +                    LOG.warn("Close master client exception", t);
    +                }
    +            } catch (Exception t) {
    +                // just ignore the exception
    +            }
    +            if (!success) {
    +                LOG.info("Waiting for a success sync of assignments from master...");
    +                try {
    +                    Thread.sleep(5000l);
    +                } catch (InterruptedException e) {
    +                    throw new RuntimeException(e);
    +                }
    +            }
    +
    +        }
    +
    +    }
    +
    +    public void getAssignmentsFromMaster(Map conf, IStormClusterState clusterState, String node) {
    +        if(ConfigUtils.isLocalMode(conf)) {
    +            try {
    +                SupervisorAssignments assignments = this.supervisor.getLocalNimbus().getSupervisorAssignments(node);
    +                assignedAssignmentsToLocal(clusterState, assignments);
    +            } catch (TException e) {
    +                LOG.error("Get assignments from local master exception", e);
    +            }
    +        } else {
    +            NimbusClient master;
    --- End diff --
    
    Same here: use try-with-resource.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by erikdw <gi...@git.apache.org>.
Github user erikdw commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @revans2 : thanks, please just loop us in and we'll carve out time to provide feedback.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @HeartSaVioR @danny0405 
    
    Sorry about my delay on this.  I got pulled off onto some other things for the past week or so.  I will start working on a patch to integrate this with the worker token code that we already merged in, but it might take a few days to get it done.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r173179815
  
    --- Diff: conf/defaults.yaml ---
    @@ -182,7 +195,7 @@ worker.log.level.reset.poll.secs: 30
     # control how many worker receiver threads we need per worker
     topology.worker.receiver.thread.count: 1
     
    -task.heartbeat.frequency.secs: 3
    +task.heartbeat.frequency.secs: 60
    --- End diff --
    
    Okey, i will  put a metric reporting interval conf item in.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    Here's patch for supporting old ZK mechanism.
    
    https://github.com/HeartSaVioR/storm/commit/79aa12b78591aa04c1ef8855032859318a83185d
    
    Didn't test manually, just for early review.
    
    @danny0405 Could you take a look at and pull if it's OK? It would be nice if you have time to do some manual tests, but I'll try to spend time to get it, so don't worry if you don't have time. No need to preserve authorship when pulling.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r173218011
  
    --- Diff: storm-client/src/jvm/org/apache/storm/utils/SupervisorClient.java ---
    @@ -0,0 +1,80 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.utils;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.generated.Supervisor;
    +import org.apache.storm.security.auth.ThriftClient;
    +import org.apache.storm.security.auth.ThriftConnectionType;
    +import org.apache.thrift.transport.TTransportException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.Map;
    +
    +public class SupervisorClient extends ThriftClient {
    +    private Supervisor.Client _client;
    --- End diff --
    
    nit lets follow convention and not use the _ at the beginning.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r158705506
  
    --- Diff: storm-core/test/clj/org/apache/storm/nimbus_test.clj ---
    @@ -792,15 +795,18 @@
           (bind _ (log-message "ass1, t76, pre beat: " (pr-str ass1)))
           (bind _ (log-message "ass2, t76, pre beat: " (pr-str ass2)))
           (is (= ass1 (executor-assignment cluster storm-id executor-id1)))
    +      ;TODO: fix local test of worker reports
           (is (not= ass2 (executor-assignment cluster storm-id executor-id2)))
           (bind ass2 (executor-assignment cluster storm-id executor-id2))
           (bind _ (log-message "ass1, t76, post beat: " (pr-str ass1)))
           (bind _ (log-message "ass2, t76, post beat: " (pr-str ass2)))
           (check-consistency cluster "test")
     
           (.advanceClusterTime cluster 31)
    +      ;TODO: fix local test of worker reports
    --- End diff --
    
    Same here.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r173220578
  
    --- Diff: storm-server/src/main/java/org/apache/storm/daemon/supervisor/ReadClusterState.java ---
    @@ -293,14 +268,15 @@ public synchronized void run() {
                                 }
                                 if (hasShared) {
                                     localAssignment.set_total_node_shared(amountShared);
    -			    }
    +			                }
    --- End diff --
    
    nit: spacing.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @revans2 
    Thanks again for your detailed explanation and taking up the hard security work which makes the patch stuck to go on. 
    
    I see you are thinking about deprecating pacemaker after addressing the metrics load in ZK, then you are totally right we may want to avoid making it as default. At least I think now you are OK to go with this patch with some follow-up works, then metrics load in ZK would be removed when we finish all the necessary works and merge this in, which is OK for me.
    
    TBH I'm still unsure the migration plan from Metrics V1 to V2 (including built-in) since it also opens the possibility for metrics to be transferred in different way (maybe via metrics reporter), but that might be out of topic for this patch, and we could discuss that after merging Metrics V2 to both 1.x and master.
    
    Please double-check my understanding. If my understanding is right, you're suggesting to modify below things:
    
    1. (a little unsure whether it is also mandatory for non-container) modify supervisor to pick a free port (range should be configurable) for thrift server instead of configured static value
    2. change supervisor to include picked port information in heartbeat, and change nimbus to leverage the port information instead of configured static value for communicating via thrift RPC
    3. change worker to report heartbeat to supervisor via RPC, not via local state (disk) which might be problematic from storm-mesos or other container solutions
    4. address heartbeat fail-back mechanism for old version workers (reading from ZK)
    
    @danny0405 
    Could you check comments from @revans2 and provide inputs? Please also let me know if you would not want to deal with supporting for old version workers. I'll see I can address it on top of your patch.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r173046065
  
    --- Diff: storm-server/src/main/java/org/apache/storm/nimbus/AssignmentDistributionService.java ---
    @@ -0,0 +1,312 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.nimbus;
    +
    +import java.io.Closeable;
    +import java.io.IOException;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.Random;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.TimeUnit;
    +
    +import org.apache.storm.DaemonConfig;
    +import org.apache.storm.daemon.supervisor.Supervisor;
    +import org.apache.storm.generated.SupervisorAssignments;
    +import org.apache.storm.scheduler.SupervisorDetails;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.ObjectReader;
    +import org.apache.storm.utils.SupervisorClient;
    +import org.apache.storm.utils.Time;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * A service for distributing master assignments to supervisors, this service makes the assignments notification
    + * asynchronous.
    + *
    + * <p>We support multiple working threads to distribute assignment, every thread has a queue buffer.
    + *
    + * <p>Master will shuffle its node request to the queues, if the target queue is full, we just discard the request,
    + * let the supervisors sync instead.
    + *
    + * <p>Caution: this class is not thread safe.
    + *
    + * <pre>{@code
    + * Working mode
    + *                      +--------+         +-----------------+
    + *                      | queue1 |   ==>   | Working thread1 |
    + * +--------+ shuffle   +--------+         +-----------------+
    + * | Master |   ==>
    + * +--------+           +--------+         +-----------------+
    + *                      | queue2 |   ==>   | Working thread2 |
    + *                      +--------+         +-----------------+
    + * }
    + * </pre>
    + */
    +public class AssignmentDistributionService implements Closeable {
    +    private static final Logger LOG = LoggerFactory.getLogger(AssignmentDistributionService.class);
    +    private ExecutorService service;
    +    /**
    +     * Flag to indicate if the service is active.
    +     */
    +    private volatile boolean active = false;
    +
    +    private Random random;
    +    /**
    +     * Working threads num.
    +     */
    +    private int threadsNum = 0;
    +    /**
    +     * Working thread queue size.
    +     */
    +    private int queueSize = 0;
    +
    +    /**
    +     * Assignments request queue.
    +     */
    +    private volatile Map<Integer, LinkedBlockingQueue<NodeAssignments>> assignmentsQueue;
    +
    +    /**
    +     * local supervisors for local cluster assignments distribution.
    +     */
    +    private Map<String, Supervisor> localSupervisors;
    +
    +    private Map conf;
    +
    +    private boolean isLocalMode = false; // boolean cache for local mode decision
    +
    +    /**
    +     * Function for initialization.
    +     *
    +     * @param conf config
    +     */
    +    public void prepare(Map conf) {
    +        this.conf = conf;
    +        this.random = new Random(47);
    +
    +        this.threadsNum = ObjectReader.getInt(conf.get(DaemonConfig.NIMBUS_ASSIGNMENTS_SERVICE_THREADS), 10);
    +        this.queueSize = ObjectReader.getInt(conf.get(DaemonConfig.NIMBUS_ASSIGNMENTS_SERVICE_THREAD_QUEUE_SIZE), 100);
    +
    +        this.assignmentsQueue = new HashMap<>();
    +        for (int i = 0; i < threadsNum; i++) {
    +            this.assignmentsQueue.put(i, new LinkedBlockingQueue<NodeAssignments>(queueSize));
    +        }
    +        //start the thread pool
    +        this.service = Executors.newFixedThreadPool(threadsNum);
    +        this.active = true;
    +        //start the threads
    +        for (int i = 0; i < threadsNum; i++) {
    +            this.service.submit(new DistributeTask(this, i));
    +        }
    +        // for local cluster
    +        localSupervisors = new HashMap<>();
    +        if (ConfigUtils.isLocalMode(conf)) {
    +            isLocalMode = true;
    +        }
    +    }
    +
    +    @Override
    +    public void close() throws IOException {
    +        this.active = false;
    +        this.service.shutdownNow();
    +        try {
    +            this.service.awaitTermination(10L, TimeUnit.SECONDS);
    --- End diff --
    
    Okey, i will add interruption decision to DistributionTask to let it shut down immediately.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r173212518
  
    --- Diff: storm-client/src/jvm/org/apache/storm/cluster/IStormClusterState.java ---
    @@ -44,6 +44,16 @@
     
         Assignment assignmentInfo(String stormId, Runnable callback);
     
    +    Assignment remoteAssignmentInfo(String stormId, Runnable callback);
    --- End diff --
    
    nit: could we get javadocs for the new methods?  I know we don't have them for any of the others, but I think it would help long term to know what they APIs are for and which daemon is supposed to call them.  This is especially true for the case of stormId, which translates a topology name to a topology id, even though getTopoId already does the same thing?  It would be nice to know why that is the case.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152717439
  
    --- Diff: storm-client/test/jvm/org/apache/storm/assignments/LocalAssignmentsBackendTest.java ---
    @@ -0,0 +1,95 @@
    +/*
    + * 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.assignments;
    +
    +import org.apache.commons.collections.map.HashedMap;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.NodeInfo;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.junit.Test;
    +
    +import java.util.*;
    +
    +public class LocalAssignmentsBackendTest {
    +
    +    @Test
    +    public void testLocalAssignment(){
    +        Map<String, Assignment> stormToAssignment = new HashMap<>();
    +        String storm1 = "storm1";
    +        String storm2 = "storm2";
    +        Assignment ass1 = mockedAssignment(1);
    +        Assignment ass2 = mockedAssignment(2);
    +
    +        ILocalAssignmentsBackend backend = LocalAssignmentsBackendFactory.getBackend(ConfigUtils.readStormConfig());
    +        assert (null == backend.getAssignment(storm1));
    --- End diff --
    
    okey


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @danny0405 the changes look good, the conflicts are minimal and I think the test failures are spurious.  I am +1 for merging this in.  Please rebase/squash the commits, resolve the minor conflicts and I will be happy to merge it in.
    
    I really would like to have someone else give a +1 for the patch too as I made some of the changes myself.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152899531
  
    --- Diff: storm-server/src/main/java/org/apache/storm/nimbus/AssignmentDistributionService.java ---
    @@ -0,0 +1,280 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.nimbus;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.daemon.supervisor.Supervisor;
    +import org.apache.storm.generated.SupervisorAssignments;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.ObjectReader;
    +import org.apache.storm.utils.SupervisorClient;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.Closeable;
    +import java.io.IOException;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.Random;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.TimeUnit;
    +
    +/**
    + * <p>A service for distributing master assignments to supervisors, this service makes the assignments notification asynchronous.
    + * <p>We support multiple working threads to distribute assignment, every thread has a queue buffer.
    + * <p>Master will shuffle its node request to the queues, if the target queue is full, we just discard the request, let the supervisors sync instead.
    + * <p/>
    + * <pre>{@code
    + * Working mode
    + *                      +--------+         +-----------------+
    + *                      | queue1 |   ==>   | Working thread1 |
    + * +--------+ shuffle   +--------+         +-----------------+
    + * | Master |   ==>
    + * +--------+           +--------+         +-----------------+
    + *                      | queue2 |   ==>   | Working thread2 |
    + *                      +--------+         +-----------------+
    + * }
    + * </pre>
    + */
    +public class AssignmentDistributionService implements Closeable {
    +    private static final Logger LOG = LoggerFactory.getLogger(AssignmentDistributionService.class);
    +    private ExecutorService service;
    +
    +    /**
    +     * Flag to indicate if the service is active
    +     */
    +    private volatile boolean active = false;
    +
    +    private Random random;
    +    /**
    +     * Working threads num.
    +     */
    +    private int threadsNum = 0;
    +    /**
    +     * Working thread queue size.
    +     */
    +    private int queueSize = 0;
    +
    +    /**
    +     * Assignments request queue.
    +     */
    +    private volatile Map<Integer, LinkedBlockingQueue<NodeAssignments>> assignmentsQueue;
    +
    +    /**
    +     * local supervisors for local cluster assignments distribution
    +     */
    +    private Map<String, Supervisor> localSupervisors;
    +
    +    private Map conf;
    +
    +    private boolean isLocalMode = false; // boolean cache for local mode decision
    +
    +    /**
    +     * Function for initialization.
    +     *
    +     * @param conf
    +     */
    +    public void prepare(Map conf) {
    +        this.conf = conf;
    +        this.random = new Random(47);
    +
    +        this.threadsNum = ObjectReader.getInt(conf.get(Config.NIMBUS_ASSIGNMENTS_SERVICE_THREADS), 10);
    +        this.queueSize = ObjectReader.getInt(conf.get(Config.NIMBUS_ASSIGNMENTS_SERVICE_THREAD_QUEUE_SIZE), 100);
    +
    +        this.assignmentsQueue = new HashMap<>();
    +        for (int i = 0; i < threadsNum; i++) {
    +            this.assignmentsQueue.put(i, new LinkedBlockingQueue<NodeAssignments>(queueSize));
    +        }
    +        //start the thread pool
    +        this.service = Executors.newFixedThreadPool(threadsNum);
    +        this.active = true;
    +        //start the threads
    +        for (int i = 0; i < threadsNum; i++) {
    +            this.service.submit(new DistributeTask(this, i));
    +        }
    +        // for local cluster
    +        localSupervisors = new HashMap<>();
    +        if (ConfigUtils.isLocalMode(conf)) {
    +            isLocalMode = true;
    +        }
    +    }
    +
    +    @Override
    +    public void close() throws IOException {
    +        this.active = false;
    +        this.service.shutdownNow();
    +        try {
    +            this.service.awaitTermination(10l, TimeUnit.SECONDS);
    +        } catch (InterruptedException e) {
    +            LOG.error("Failed to close assignments distribute service");
    +        }
    +        this.assignmentsQueue = null;
    +    }
    +
    +    public void addAssignmentsForNode(String node, String host, SupervisorAssignments assignments) {
    +        try {
    +            boolean success = nextQueue().offer(NodeAssignments.getInstance(node, host, assignments), 5l, TimeUnit.SECONDS);
    +            if (!success) {
    +                LOG.warn("Discard an assignment distribution for node {} because the target sub queue is full", node);
    +            }
    +
    +        } catch (InterruptedException e) {
    +            LOG.error("Add node assignments interrupted: {}", e.getMessage());
    +            throw new RuntimeException(e);
    +        }
    +    }
    +
    +    static class NodeAssignments {
    +        private String node;
    +        private SupervisorAssignments assignments;
    +        private String host;
    +
    +        private NodeAssignments(String node, String host, SupervisorAssignments assignments) {
    +            this.node = node;
    +            this.host = host;
    +            this.assignments = assignments;
    +        }
    +
    +        public static NodeAssignments getInstance(String node, String host, SupervisorAssignments assignments) {
    +            return new NodeAssignments(node, host, assignments);
    +        }
    +
    +        //supervisor assignment id/supervisor id
    +        public String getNode() {
    +            return this.node;
    +        }
    +
    +        public String getHost() {
    +            return host;
    +        }
    +
    +        public SupervisorAssignments getAssignments() {
    +            return this.assignments;
    +        }
    +
    +    }
    +
    +    /**
    +     * Task to distribute assignments.
    +     */
    +    static class DistributeTask implements Runnable {
    +        private AssignmentDistributionService service;
    +        private Integer queueIndex;
    +
    +        DistributeTask(AssignmentDistributionService service, Integer index) {
    +            this.service = service;
    +            this.queueIndex = index;
    +        }
    +
    +        @Override
    +        public void run() {
    +            while (true) {
    +                try {
    +                    NodeAssignments nodeAssignments = this.service.nextAssignments(queueIndex);
    +                    sendAssignmentsToNode(nodeAssignments);
    +                } catch (InterruptedException e) {
    +                    if (service.isActive()) {
    +                        LOG.error("Get an unexpected interrupt when distributing assignments to node, {}", e.getCause());
    +                    } else {
    +                        // service is off now just interrupt it.
    +                        Thread.currentThread().interrupt();
    +                    }
    +                }
    +            }
    +        }
    +
    +        private void sendAssignmentsToNode(NodeAssignments assignments) {
    +            if (this.service.isLocalMode) {
    +                //local node
    +                Supervisor supervisor = this.service.localSupervisors.get(assignments.getNode());
    +                if (supervisor != null) {
    +                    supervisor.sendSupervisorAssignments(assignments.getAssignments());
    +                } else {
    +                    LOG.error("Can not find node {} for assignments distribution");
    +                    throw new RuntimeException("null for node {} supervisor instance");
    +                }
    +            } else {
    +                // distributed mode
    +                try {
    +                    SupervisorClient client = SupervisorClient.getConfiguredClient(service.getConf(), assignments.getHost());
    --- End diff --
    
    Please use try-with-resource.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r158713978
  
    --- Diff: storm-client/src/jvm/org/apache/storm/assignments/InMemoryAssignmentBackend.java ---
    @@ -0,0 +1,152 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.assignments;
    +
    +import java.util.ArrayList;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.storm.cluster.ClusterUtils;
    +import org.apache.storm.generated.Assignment;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * An assignment backend which will keep all assignments and id-info in memory. Only used if no backend is specified internal.
    + */
    +public class InMemoryAssignmentBackend implements ILocalAssignmentsBackend {
    +    private static final Logger LOG = LoggerFactory.getLogger(InMemoryAssignmentBackend.class);
    +
    +    protected Map<String, Assignment> idToAssignment;
    +    protected Map<String, String> idToName;
    +    protected Map<String, String> nameToId;
    +    private volatile boolean isSynchronized = false;
    +
    +    public InMemoryAssignmentBackend() {}
    +
    +    @Override
    +    public boolean isSynchronized() {
    +        return this.isSynchronized;
    +    }
    +
    +    @Override
    +    public void setSynchronized() {
    +        this.isSynchronized = true;
    +    }
    +
    +    @Override
    +    public void prepare(Map conf) {
    +        // do nothing for conf now
    +        this.idToAssignment = new ConcurrentHashMap<>();
    +        this.idToName = new ConcurrentHashMap<>();
    +        this.nameToId = new ConcurrentHashMap<>();
    +    }
    +
    +    @Override
    +    public void keepOrUpdateAssignment(String stormId, Assignment assignment) {
    +        this.idToAssignment.put(stormId, assignment);
    +    }
    +
    +    @Override
    +    public Assignment getAssignment(String stormId) {
    +        return this.idToAssignment.get(stormId);
    +    }
    +
    +    @Override
    +    public void removeAssignment(String stormId) {
    +        this.idToAssignment.remove(stormId);
    +    }
    +
    +    @Override
    +    public List<String> assignments() {
    +        if(idToAssignment == null) {
    +            return new ArrayList<>();
    +        }
    +        List<String> ret = new ArrayList<>();
    +        ret.addAll(this.idToAssignment.keySet());
    +        return ret;
    +    }
    +
    +    @Override
    +    public Map<String, Assignment> assignmentsInfo() {
    +        Map<String, Assignment> ret = new HashMap<>();
    +        ret.putAll(this.idToAssignment);
    +
    +        return ret;
    +    }
    +
    +    @Override
    +    public void syncRemoteAssignments(Map<String, byte[]> remote) {
    +        Map<String, Assignment> tmp = new ConcurrentHashMap<>();
    --- End diff --
    
    yeah, when supervisor sync the assignments, master may changed it at the same time.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r153036746
  
    --- Diff: storm-client/src/jvm/org/apache/storm/Config.java ---
    @@ -1293,6 +1318,44 @@
         @isPositiveNumber
         public static final String SUPERVISOR_CPU_CAPACITY = "supervisor.cpu.capacity";
     
    +    @isInteger
    --- End diff --
    
    this supervisor config is used by SupervisorClient, so they can not be moved to storm-server


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    I was also just thinking that it would be really great if in the future we could drop even more access from the workers to zookeeper.  If assignments are gone and heartbeats go away all that is left is credentials, backpressure, and possibly a few other small things.  But backpressure is going to go away too so that would just be some small odds and ends.  If we could add in a return object to the heartbeats, we could in the future add in support for fetching changed credentials, etc. and not have to worry about zookeeper except in client side code.
    
    So all I am asking is if for the heartbeat thrift calls if we could make it so the result code is an empty object and not void.  That way we have a clean way to expand it in the future.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152705852
  
    --- Diff: storm-core/test/clj/org/apache/storm/nimbus_test.clj ---
    @@ -454,7 +456,7 @@
           (is (= 2 (storm-num-workers state "mystorm"))) ;; because only 2 executors
           )))
     
    -;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
    +;;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
    --- End diff --
    
    unnecessary change


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r153055623
  
    --- Diff: storm-client/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java ---
    @@ -43,13 +44,15 @@
         private static Logger LOG = LoggerFactory.getLogger(StormClusterStateImpl.class);
     
         private IStateStorage stateStorage;
    +    private ILocalAssignmentsBackend backend;
    --- End diff --
    
    Yes if you could make a change even if you didn't change, even better.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r153036973
  
    --- Diff: storm-client/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java ---
    @@ -43,13 +44,15 @@
         private static Logger LOG = LoggerFactory.getLogger(StormClusterStateImpl.class);
     
         private IStateStorage stateStorage;
    +    private ILocalAssignmentsBackend backend;
    --- End diff --
    
    there is also import org.apache.storm.generated.* here, should i make a change?


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152900152
  
    --- Diff: storm-server/src/main/java/org/apache/storm/nimbus/LeaderListenerCallback.java ---
    @@ -0,0 +1,213 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.nimbus;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.collect.Sets;
    +import org.apache.commons.io.IOUtils;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.curator.framework.recipes.leader.LeaderLatch;
    +import org.apache.storm.Config;
    +import org.apache.storm.blobstore.BlobStore;
    +import org.apache.storm.blobstore.InputStreamWithMeta;
    +import org.apache.storm.cluster.ClusterUtils;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.daemon.nimbus.TopoCache;
    +import org.apache.storm.generated.AuthorizationException;
    +import org.apache.storm.generated.KeyNotFoundException;
    +import org.apache.storm.generated.StormTopology;
    +import org.apache.storm.security.auth.ReqContext;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.Utils;
    +import org.apache.storm.zookeeper.ClientZookeeper;
    +import org.apache.zookeeper.CreateMode;
    +import org.apache.zookeeper.data.ACL;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.io.IOException;
    +import java.util.*;
    --- End diff --
    
    avoid wildcard import


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152705675
  
    --- Diff: storm-client/test/jvm/org/apache/storm/assignments/LocalAssignmentsBackendTest.java ---
    @@ -0,0 +1,95 @@
    +/*
    + * 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.assignments;
    +
    +import org.apache.commons.collections.map.HashedMap;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.NodeInfo;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.junit.Test;
    +
    +import java.util.*;
    +
    +public class LocalAssignmentsBackendTest {
    +
    +    @Test
    +    public void testLocalAssignment(){
    +        Map<String, Assignment> stormToAssignment = new HashMap<>();
    +        String storm1 = "storm1";
    +        String storm2 = "storm2";
    +        Assignment ass1 = mockedAssignment(1);
    +        Assignment ass2 = mockedAssignment(2);
    +
    +        ILocalAssignmentsBackend backend = LocalAssignmentsBackendFactory.getBackend(ConfigUtils.readStormConfig());
    +        assert (null == backend.getAssignment(storm1));
    --- End diff --
    
    Let's use JUnit Assert class instead of using Java assert in unit test. assertEquals, assertTrue, assertFalse, etc.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152896015
  
    --- Diff: storm-server/src/main/java/org/apache/storm/daemon/supervisor/Supervisor.java ---
    @@ -219,6 +246,8 @@ private void launchDaemon() {
                     throw new IllegalArgumentException("Cannot start server in local mode!");
                 }
                 launch();
    +            //must invoke after launch cause some service must be initialized
    --- End diff --
    
    nit: service -> services


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152703257
  
    --- Diff: storm-client/src/jvm/org/apache/storm/daemon/worker/Worker.java ---
    @@ -382,6 +374,33 @@ public void establishLogSettingCallback() {
             workerState.stormClusterState.topologyLogConfig(topologyId, this::checkLogConfigChanged);
         }
     
    +    /**
    +     * Send a heartbeat to local supervisor first to check if supervisor is ok for heartbeating.
    +     */
    +    private void heartbeatToMasterIfLocalbeatFail(LSWorkerHeartbeat lsWorkerHeartbeat) {
    +        if (ConfigUtils.isLocalMode(this.conf)) {
    +            return;
    +        }
    +        //in distributed mode, send heartbeat directly to master if local supervisor goes down
    +        SupervisorWorkerHeartbeat workerHeartbeat = new SupervisorWorkerHeartbeat(lsWorkerHeartbeat.get_topology_id(),
    +                lsWorkerHeartbeat.get_executors(), lsWorkerHeartbeat.get_time_secs());
    +        try{
    +            SupervisorClient client = SupervisorClient.getConfiguredClient(conf, Utils.hostname());
    +            client.getClient().sendSupervisorWorkerHeartbeat(workerHeartbeat);
    +            client.close();
    +        } catch (Throwable tr1) {
    +            //if any error/exception thrown, report directly to nimbus.
    +            LOG.debug("Exception when send heartbeat to local supervisor", tr1.getMessage());
    --- End diff --
    
    We could use `warn`, given that supervisor unavailability is also kind of critical thing in operation perspective.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r162322012
  
    --- Diff: storm-client/src/storm.thrift ---
    @@ -836,3 +848,33 @@ exception HBAuthorizationException {
     exception HBExecutionException {
       1: required string msg;
     }
    +
    +struct SupervisorAssignments {
    +    1: optional map<string, Assignment> storm_assignment = {}
    +}
    +
    +service Supervisor {
    +  /**
    +   * Send node specific assignments to supervisor
    +   */
    +  void sendSupervisorAssignments(1: SupervisorAssignments assignments) throws (1: AuthorizationException aze);
    +  /**
    +   * Get local assignment for a storm
    +   */
    +  Assignment getLocalAssignmentForStorm(1: string id) throws (1: NotAliveException e, 2: AuthorizationException aze);
    +  /**
    +   * Send worker heartbeat to local supervisor
    +   */
    +  void sendSupervisorWorkerHeartbeat(1: SupervisorWorkerHeartbeat heartbeat) throws (1: AuthorizationException aze);
    +}
    +
    +struct SupervisorWorkerHeartbeat {
    --- End diff --
    
    Looks like SupervisorWorkerHeartbeat and SupervisorWorkerHeartbeats are used "before", forward reference. 
    
    We encountered the issue because of forward reference of thrift definition. https://issues.apache.org/jira/browse/STORM-1842
    
    Could you rearrange the order so that all the structs are defined earlier before using them?


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152902232
  
    --- Diff: storm-server/src/main/java/org/apache/storm/nimbus/TimeOutWorkerHeartbeatsRecoveryStrategy.java ---
    @@ -0,0 +1,86 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.nimbus;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.utils.ObjectReader;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.HashSet;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * Wait for a node to report worker heartbeats until a configured timeout. For cases below we have strategies:
    + * <p>
    + * 1: When nimbus gains leader ship, it will decide if the heartbeats are ready based on the reported node ids,
    + * supervisors/nodes will take care of the worker heartbeats recovery, a reported node id means all the workers heartbeats on the node are reported.
    + * <p>
    + * 2: If several supervisor also crush and will never recover[or all crush for some unknown reason],
    + * workers will report their heartbeats directly to master, so it has not any effect.
    + */
    +public class TimeOutWorkerHeartbeatsRecoveryStrategy implements IWorkerHeartbeatsRecoveryStrategy {
    +    private static final Logger LOG = LoggerFactory.getLogger(TimeOutWorkerHeartbeatsRecoveryStrategy.class);
    +
    +    private static int NODE_MAX_TIMEOUT_SECS = 600;
    +
    +    private long startTimeSecs;
    +
    +    private Set<String> reportedIDs;
    +
    +    @Override
    +    public void prepare(Map conf) {
    +        NODE_MAX_TIMEOUT_SECS = ObjectReader.getInt(conf.get(Config.SUPERVISOR_WORKER_HEARTBEATS_MAX_TIMEOUT_SECS), 600);
    +        this.startTimeSecs = System.currentTimeMillis() / 1000L;
    +        this.reportedIDs = new HashSet<>();
    +    }
    +
    +    @Override
    +    public boolean isReady(Set<String> nodeIds) {
    +        if (isMaxTimeOut()) {
    +            HashSet<String> tmp = new HashSet<>();
    --- End diff --
    
    We could simplify the logic:
    
    ```
    Set<String> notReportedIds = nodeIds.stream().filter(id -> !this.reportedIds.contains(id))
    .collect(toSet());
    ```


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152703886
  
    --- Diff: storm-client/src/jvm/org/apache/storm/stats/StatsUtil.java ---
    @@ -21,43 +21,15 @@
     
     import org.apache.storm.cluster.ExecutorBeat;
     import org.apache.storm.cluster.IStormClusterState;
    -import org.apache.storm.generated.Bolt;
    -import org.apache.storm.generated.BoltAggregateStats;
    -import org.apache.storm.generated.BoltStats;
    -import org.apache.storm.generated.ClusterWorkerHeartbeat;
    -import org.apache.storm.generated.CommonAggregateStats;
    -import org.apache.storm.generated.ComponentAggregateStats;
    -import org.apache.storm.generated.ComponentPageInfo;
    -import org.apache.storm.generated.ComponentType;
    -import org.apache.storm.generated.ErrorInfo;
    -import org.apache.storm.generated.ExecutorAggregateStats;
    -import org.apache.storm.generated.ExecutorInfo;
    -import org.apache.storm.generated.ExecutorSpecificStats;
    -import org.apache.storm.generated.ExecutorStats;
    -import org.apache.storm.generated.ExecutorSummary;
    -import org.apache.storm.generated.GlobalStreamId;
    -import org.apache.storm.generated.SpecificAggregateStats;
    -import org.apache.storm.generated.SpoutAggregateStats;
    -import org.apache.storm.generated.SpoutStats;
    -import org.apache.storm.generated.StormTopology;
    -import org.apache.storm.generated.TopologyPageInfo;
    -import org.apache.storm.generated.TopologyStats;
    -import org.apache.storm.generated.WorkerResources;
    -import org.apache.storm.generated.WorkerSummary;
    +import org.apache.storm.generated.*;
    --- End diff --
    
    avoid wildcard import


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152693895
  
    --- Diff: storm-client/src/jvm/org/apache/storm/assignments/ILocalAssignmentsBackend.java ---
    @@ -0,0 +1,103 @@
    +/**
    + * 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.assignments;
    +
    +import org.apache.storm.generated.Assignment;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +/**
    + * Interface for storing local assignments.
    + */
    +public interface ILocalAssignmentsBackend {
    +
    +    /**
    +     * initial function for creating backend.
    +     * @param conf
    +     */
    +    void prepare(Map conf);
    +
    +    /**
    +     * Keep a storm assignment to local state or update old assignment.
    +     * @param stormID storm runtime id
    +     * @param assignment assignment as thrift
    +     */
    +    void keepOrUpdateAssignment(String stormID, Assignment assignment);
    +
    +    /**
    +     * Get assignment as byte[] for a storm
    +     * @param stormID storm runtime id
    +     * @return
    +     */
    +    Assignment getAssignment(String stormID);
    +
    +    void removeAssignment(String stormID);
    +
    +    /**
    +     * List all the storm runtime ids of local assignments
    +     * @return a list of storm ids
    +     */
    +    List<String> assignments();
    +
    +    /**
    +     * Get all the local assignments of local state
    +     * @return mapping of storm-id -> assignment
    +     */
    +    Map<String, Assignment> assignmentsInfo();
    +
    +    /**
    +     * Sync remote assignments to local, if remote is null, we will sync it from zk
    +     * @param remote specific remote assignments, if is null, it will sync from zookeeper[only used for nimbus]
    --- End diff --
    
    if is null -> if it is null


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r153037892
  
    --- Diff: storm-client/src/jvm/org/apache/storm/stats/StatsUtil.java ---
    @@ -1589,18 +1570,16 @@ public static ComponentPageInfo aggCompExecsStats(
                 if (lastReportedTime != null) {
                     reportedTime = lastReportedTime;
                 } else {
    -                reportedTime = 0;
    +                reportedTime = lastReportedTime = 0;
    --- End diff --
    
    we do not decide if updateHeartbeatCache is null when reportedTime.equals(lastReportedTime)


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @revans2 
    Really thx, i will fix the nits problems as soon as possible, then i will merge this patch in.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by erikdw <gi...@git.apache.org>.
Github user erikdw commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    /quote My consideration is the workers started by parent supervisor, how do the workers know their parent supervisor's heartbeat/assignment port? If i passed it as starting argument, the port will be invalid if its parent supervisor collapse or restarts.
    
    For storm-on-mesos, the Supervisor + Workers run in a per-topology container on each host.  The Supervisor is the container's init process, so if it dies then the Workers die with it.  So the problematic scenario you outlined doesn't exist for that use-case.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r158714573
  
    --- Diff: storm-core/test/clj/org/apache/storm/nimbus_test.clj ---
    @@ -792,15 +795,18 @@
           (bind _ (log-message "ass1, t76, pre beat: " (pr-str ass1)))
           (bind _ (log-message "ass2, t76, pre beat: " (pr-str ass2)))
           (is (= ass1 (executor-assignment cluster storm-id executor-id1)))
    -      (is (not= ass2 (executor-assignment cluster storm-id executor-id2)))
    +      ;TODO: fix local test of worker reports
    +      ;(is (not= ass2 (executor-assignment cluster storm-id executor-id2)))
           (bind ass2 (executor-assignment cluster storm-id executor-id2))
           (bind _ (log-message "ass1, t76, post beat: " (pr-str ass1)))
           (bind _ (log-message "ass2, t76, post beat: " (pr-str ass2)))
           (check-consistency cluster "test")
     
           (.advanceClusterTime cluster 31)
    -      (is (not= ass1 (executor-assignment cluster storm-id executor-id1)))
    +      ;TODO: fix local test of worker reports
    --- End diff --
    
    fixed already


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @revans2 
    Hi, is threre anything i can do now for this patch?


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r173213078
  
    --- Diff: storm-client/src/jvm/org/apache/storm/cluster/IStormClusterState.java ---
    @@ -94,6 +108,8 @@
         @Deprecated
         List<String> backpressureTopologies();
     
    +    NimbusInfo getLeader(Runnable callback);
    --- End diff --
    
    I am a little weary of using this vs going directly to the leader elector.  The issue really is one of fencing.  In some cases it is possible for the leader elector to lose leadership, but the other nodes in ZK to not be updated for a while.  I am fine with this API so long as we have the proper javadocs to explain that it cannot be used for fencing and is only for informational purposes.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r173216785
  
    --- Diff: storm-client/src/jvm/org/apache/storm/grouping/LoadAwareShuffleGrouping.java ---
    @@ -295,7 +295,11 @@ private Scope calculateScope(Map<Integer, NodeInfo> taskToNodePort, Map<String,
         private Map<String, String> getHostToRackMapping(Map<Integer, NodeInfo> taskToNodePort) {
             Set<String> hosts = new HashSet();
             for (int task: targetTasks) {
    -            hosts.add(taskToNodePort.get(task).get_node());
    +            //if this task containing worker will be killed by a assignments sync,
    +            //taskToNodePort will be an empty map which is refreshed by WorkerState
    +            if (taskToNodePort.containsKey(task)) {
    +                hosts.add(taskToNodePort.get(task).get_node());
    +            }
    --- End diff --
    
    Nit: could we log an error if it does not contain the task?  I just want to be sure that we know something possibly unexpected has happened here.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @HeartSaVioR 
    Actually, this patch can work with old workers if we upgrade all the daemons: Nimbus/Supervisors, cause supervisors collect local heartbeats and report to Nimbus through RPC.
    
    The only actions diff of old/new workers is that: old workers will not report tick hb to local Supervisor while new workers will do.  When new supervisor daemon die with old workers, nimbus will sense it and fire a timed-out reassign. 


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @revans2 Sure! Thanks for taking this up.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r158758510
  
    --- Diff: storm-client/src/jvm/org/apache/storm/assignments/LocalAssignmentsBackendFactory.java ---
    @@ -0,0 +1,48 @@
    +/**
    + * 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.assignments;
    +
    +import com.google.common.base.Preconditions;
    +import org.apache.storm.Config;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.ReflectionUtils;
    +
    +import java.util.Map;
    +
    +/**
    + * Factory class for creating local assignments.
    + */
    +public class LocalAssignmentsBackendFactory {
    +
    +    public static ILocalAssignmentsBackend getBackend(Map<String, Object> conf) {
    +        if (conf.get(Config.NIMBUS_LOCAL_ASSIGNMENTS_BACKEND_CLASS) != null) {
    +            Object targetObj = ReflectionUtils.newInstance((String) conf.get(Config.NIMBUS_LOCAL_ASSIGNMENTS_BACKEND_CLASS));
    +            Preconditions.checkState(targetObj instanceof ILocalAssignmentsBackend, "{} must implements ILocalAssignmentsBackend", Config.NIMBUS_LOCAL_ASSIGNMENTS_BACKEND_CLASS);
    +            ((ILocalAssignmentsBackend)targetObj).prepare(conf);
    +            return (ILocalAssignmentsBackend) targetObj;
    +        }
    +
    +        return getDefault();
    --- End diff --
    
    Let's keep it as it is. I think methods in ClusterUtils is the thing to refactor, and we can take it afterwards, in other issue.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r173232056
  
    --- Diff: storm-server/src/main/java/org/apache/storm/daemon/supervisor/Supervisor.java ---
    @@ -234,6 +295,60 @@ public void launchDaemon() {
             }
         }
     
    +    private void launchSupervisorThriftServer(Map conf) throws IOException {
    +        // validate port
    +        int port = getThriftServerPort();
    +        try {
    +            ServerSocket socket = new ServerSocket(port);
    +            socket.close();
    +        } catch (BindException e) {
    +            LOG.error("{} is not available. Check if another process is already listening on {}", port, port);
    +            throw new RuntimeException(e);
    +        }
    +
    +        TProcessor processor = new org.apache.storm.generated.Supervisor.Processor(
    +                new org.apache.storm.generated.Supervisor.Iface() {
    +                    @Override
    +                    public void sendSupervisorAssignments(SupervisorAssignments assignments)
    +                            throws AuthorizationException, TException {
    +                        LOG.info("Got an assignments from master, will start to sync with assignments: {}", assignments);
    +                        SynchronizeAssignments syn = new SynchronizeAssignments(getSupervisor(), assignments, getReadClusterState());
    +                        getEventManger().add(syn);
    +                    }
    +
    +                    @Override
    +                    public Assignment getLocalAssignmentForStorm(String id)
    +                            throws NotAliveException, AuthorizationException, TException {
    +                        Assignment assignment = getStormClusterState().assignmentInfo(id, null);
    +                        if (null == assignment) {
    +                            throw new NotAliveException("No local assignment assigned for storm: " + id + " for node: " + getHostName());
    +                        }
    +                        return assignment;
    +                    }
    +
    +                    @Override
    +                    public void sendSupervisorWorkerHeartbeat(SupervisorWorkerHeartbeat heartbeat)
    +                            throws AuthorizationException, TException {
    +                        // do nothing now
    +                    }
    --- End diff --
    
    Sorry I forgot this before.  All of these must have some kind of authorization checks.  We have authenticated the user connecting, but right now anyone with valid Kerberos credentials or a valid WorkerToken can call these APIs.  We need something that can block users that should not be calling them, and with the ability to turn it off for a non-secure cluster.
    
    `sendSupervisorAssignments` is the biggest security problem.  It needs to be restricted to only nimbus making that call.  `getLocalAssignmentForStrom` is probably okay to be totally open, but it might be good to restrict it to just the owner of that topology + nimbus.  Similar for `sendSupervisorWorkerHeartbeat`.  It is a noop right now so not that big of a deal, but in the future I would expect us to want to restrict it.
    
    Please take a look at how nimbus is doing these checks.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r173000061
  
    --- Diff: storm-server/src/main/java/org/apache/storm/nimbus/AssignmentDistributionService.java ---
    @@ -0,0 +1,312 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.nimbus;
    +
    +import java.io.Closeable;
    +import java.io.IOException;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.Random;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.TimeUnit;
    +
    +import org.apache.storm.DaemonConfig;
    +import org.apache.storm.daemon.supervisor.Supervisor;
    +import org.apache.storm.generated.SupervisorAssignments;
    +import org.apache.storm.scheduler.SupervisorDetails;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.ObjectReader;
    +import org.apache.storm.utils.SupervisorClient;
    +import org.apache.storm.utils.Time;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * A service for distributing master assignments to supervisors, this service makes the assignments notification
    + * asynchronous.
    + *
    + * <p>We support multiple working threads to distribute assignment, every thread has a queue buffer.
    + *
    + * <p>Master will shuffle its node request to the queues, if the target queue is full, we just discard the request,
    + * let the supervisors sync instead.
    + *
    + * <p>Caution: this class is not thread safe.
    + *
    + * <pre>{@code
    + * Working mode
    + *                      +--------+         +-----------------+
    + *                      | queue1 |   ==>   | Working thread1 |
    + * +--------+ shuffle   +--------+         +-----------------+
    + * | Master |   ==>
    + * +--------+           +--------+         +-----------------+
    + *                      | queue2 |   ==>   | Working thread2 |
    + *                      +--------+         +-----------------+
    + * }
    + * </pre>
    + */
    +public class AssignmentDistributionService implements Closeable {
    +    private static final Logger LOG = LoggerFactory.getLogger(AssignmentDistributionService.class);
    +    private ExecutorService service;
    +    /**
    +     * Flag to indicate if the service is active.
    +     */
    +    private volatile boolean active = false;
    +
    +    private Random random;
    +    /**
    +     * Working threads num.
    +     */
    +    private int threadsNum = 0;
    +    /**
    +     * Working thread queue size.
    +     */
    +    private int queueSize = 0;
    +
    +    /**
    +     * Assignments request queue.
    +     */
    +    private volatile Map<Integer, LinkedBlockingQueue<NodeAssignments>> assignmentsQueue;
    +
    +    /**
    +     * local supervisors for local cluster assignments distribution.
    +     */
    +    private Map<String, Supervisor> localSupervisors;
    +
    +    private Map conf;
    +
    +    private boolean isLocalMode = false; // boolean cache for local mode decision
    +
    +    /**
    +     * Function for initialization.
    +     *
    +     * @param conf config
    +     */
    +    public void prepare(Map conf) {
    +        this.conf = conf;
    +        this.random = new Random(47);
    +
    +        this.threadsNum = ObjectReader.getInt(conf.get(DaemonConfig.NIMBUS_ASSIGNMENTS_SERVICE_THREADS), 10);
    +        this.queueSize = ObjectReader.getInt(conf.get(DaemonConfig.NIMBUS_ASSIGNMENTS_SERVICE_THREAD_QUEUE_SIZE), 100);
    +
    +        this.assignmentsQueue = new HashMap<>();
    +        for (int i = 0; i < threadsNum; i++) {
    +            this.assignmentsQueue.put(i, new LinkedBlockingQueue<NodeAssignments>(queueSize));
    +        }
    +        //start the thread pool
    +        this.service = Executors.newFixedThreadPool(threadsNum);
    +        this.active = true;
    +        //start the threads
    +        for (int i = 0; i < threadsNum; i++) {
    +            this.service.submit(new DistributeTask(this, i));
    +        }
    +        // for local cluster
    +        localSupervisors = new HashMap<>();
    +        if (ConfigUtils.isLocalMode(conf)) {
    +            isLocalMode = true;
    +        }
    +    }
    +
    +    @Override
    +    public void close() throws IOException {
    +        this.active = false;
    +        this.service.shutdownNow();
    +        try {
    +            this.service.awaitTermination(10L, TimeUnit.SECONDS);
    --- End diff --
    
    This always will wait the full 10 seconds because there is no way to signal to the DistributeTask that it should shut down.  This is at least in part why it is taking so long for the tests to run.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r173213537
  
    --- Diff: storm-client/src/jvm/org/apache/storm/cluster/IStormClusterState.java ---
    @@ -233,23 +249,11 @@
          * @return the id of the topology or null if it is not alive.
          */
         default Optional<String> getTopoId(final String topologyName) {
    --- End diff --
    
    I am a little confused why we need 2 methods that appear to do the exact same thing.  Could we try to have just one of them?


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152698386
  
    --- Diff: storm-client/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java ---
    @@ -147,21 +153,52 @@ protected void issueMapCallback(ConcurrentHashMap<String, Runnable> callbackConc
     
         @Override
         public List<String> assignments(Runnable callback) {
    +        //deprecated
             if (callback != null) {
                 assignmentsCallback.set(callback);
             }
    -        return stateStorage.get_children(ClusterUtils.ASSIGNMENTS_SUBTREE, callback != null);
    +        return this.backend.assignments();
         }
     
         @Override
         public Assignment assignmentInfo(String stormId, Runnable callback) {
    +        //deprecated
    --- End diff --
    
    Same here.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r173232562
  
    --- Diff: storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java ---
    @@ -4215,7 +4474,48 @@ public boolean isTopologyNameAllowed(String name) throws AuthorizationException,
                 throw new RuntimeException(e);
             }
         }
    -    
    +
    +    @Override
    +    public SupervisorAssignments getSupervisorAssignments(String node) throws AuthorizationException, TException {
    +        try {
    --- End diff --
    
    Would be good to have some authorization checks here.  Not super critical because the data is public, but would be good to tie it into out existing authentication system.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r177712003
  
    --- Diff: storm-client/src/jvm/org/apache/storm/grouping/LoadAwareShuffleGrouping.java ---
    @@ -299,6 +299,8 @@ private Scope calculateScope(Map<Integer, NodeInfo> taskToNodePort, Map<String,
                 //taskToNodePort will be an empty map which is refreshed by WorkerState
                 if (taskToNodePort.containsKey(task)) {
                     hosts.add(taskToNodePort.get(task).get_node());
    +            } else {
    +                LOG.error("Could not find task NodeInfo from local cache.");
    --- End diff --
    
    nit: better to show which task is missing: just adding task id.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @HeartSaVioR @revans2
    Based on your comments, i thought there are mainly 3 TODOs for this patch:
    	1. Supervisor container-blity promotion, like support multiple supervisor instances on one machine.
    	2. Assignments security should be guaranteed.
    	3. Backwards compatibility for old version storm workers.
    
    For TODO 1 I can make supervisor thrift ports picked in a range for a machine node, and nimbus aware the port-info from SupervisorHeartbeats. Also i will passed the port as an start up argument so that workers will know its parent supervisor port.
    
    For TODO 2 i understood i should wait for @revans2's token authentication right?
    
    For TODO 3 i don't know how much worker would be taken, if @HeartSaVioR can help to contribute it will very appreciate it.
    
    Also the supervisor local disk worker heartbeats can also be removed actually, and it's easy to achieve for the patch, should i also do this?


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @HeartSaVioR thanks for putting up the reference to my patch for STORM-2898.  Sorry I have not been responding for the past few days.  I locked myself in a room trying to get it done ASAP, and have been ignoring e-mail.
    
    I plan next to start working on the storm on mesos/yarn/open stack/... API changes for 2.x.  the issue that we have been running into is that the APIs defined in INimbus and ISupervisor are not really that clear on how they are used or how they should be used.  So even though we don't change these APIs we change the semantics around how these APIs are used and end up breaking things.
    
    I will file a JIRA soon hopefully with a design on how I want to address this, but I am going to need a lot of feedback from @erikdw and @JessicaLHartog to make sure the API changes I am proposing will actually be better.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @danny0405 
    
    1.  The issues we saw with stability were around when a pacemaker nodes goes down that it was causing exceptions in the clients that we were not handling properly resulting in workers restarting.  We have not seen any issues with heartbeat packets being discarded.  If this did not cover the issues you were seeing I really would love to have a JIRA so I can fix it.
    
    We are not running with 2.x, or even 1.x, in production so I cannot say if there is some oddness happening with what we have pushed back, or perhaps its interactions with HA.  We are on 0.10.2++++, we pulled back a lot from 1.x.  This is why we really want to get to 2.x so we can we aligned with the community again and hopefully not have these kinds of issues.  There may be bugs we don't realize right now.
    
    2. With pacemaker HA if you have 2+ pacemaker servers each of the clients will randomly select one of the servers to send heartbeats to.  If the one they try to write to is down, at the beginning or in the middle, the heartbeats should then start going to a different, random, server.  This should hopefully keep the load even between the pacemaker servers.  Nimbus is supposed to work all of this out by reading from all the servers, and if it finds more than one heartbeat for a worker it will pick the one that has the newest timestamp in it.  This does not scale well on the nimbus side, and can take more then 2 mins to download all of the heartbeats, so we have plans to parallelize the download.
    
    The metric don't go to the supervisor, as it does not need/use them currently.  It only cares if the worker is up and still alive, so it knows if it needs to restart it.
    
    3. I totally believe you that this can support a large cluster.  Like I said this is a much better solution long term, and I would love to go this route.  We just need to fix the security issues and find a way to support containerized supervisors for me to give it a +1. Both should be doable.
    
    4. There is no security between the workers and pacemaker.  There is security between nimbus and pacemaker.  This means that only nimbus can see the heartbeats.  The worst you can do with faking heartbeats is confuse someone with bad metrics (not ideal) or trick nimbus into thinking a worker is still alive when it is not, bad but not horrible.  It is the assignment portion that is scary to me, because it says what to run.  If we pull the assignment portion out I would be OK with that.  Although it would be best to truly fix it because we don't have a way to selectively turn off authorization in thrift so to make that work we would need a separate thrift server on nimbus, which I would rather not do.
    
    I would love to see the ability to do delegation tokens in storm for authentication.  This is no small task.  It would take a lot of work, especially with HA, which is why I haven't done it.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r158768126
  
    --- Diff: conf/defaults.yaml ---
    @@ -162,6 +168,13 @@ supervisor.memory.capacity.mb: 4096.0
     # for single threaded bolts
     supervisor.cpu.capacity: 400.0
     
    +#Supervisor thrift config
    +supervisor.thrift.port: 6627
    --- End diff --
    
    done


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @HeartSaVioR Really thx for nice review work, it duration is long and the patch is huge, thx for your patience.
    
    I have fixed the comments not addressed.
    
    Nimbus will have a debug log if assignments/heartbeats-report error.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    So I just spoke with management here and did some quick back of the envelope planning and I think to unblock this we should support delegation token like functionality in storm.  This could potentially make life a lot simpler for all kinds of things.  Here and with DRPC, etc.
    
    I am willing to commit my team to make this happen, so I will file some JIRAs and try to put together a plan/architecture that hopefully others can review.
    
    Once we have delegation tokens working the only real issue is going to be containerized supervisors.  I think we can support that by having the supervisor pick a free port in a configured range, and then include that port in it's heartbeat to nimbus.  We would also need a way to tell the workers what port to use to communicate with the supervisor.
    
    For me I really would like to be able to maintain the ability to run 0.10.x and 1.x topologies under a 2.x cluster.  I think this would only require still checking for heartbeats from zookeeper before scheduling which I don't think has been removed yet, so I am hopeful that it will work with the current patch.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @HeartSaVioR @erikdw 
    Yeah, `-c supervisor.thrift.port=<port>` will works great for starting supervisor on Mesos.
    
    My consideration is the workers started by parent supervisor, how do the workers know it's parent heartbeat port? If i passed it as starting argument, the port will be invalid if its parent supervisor collapse or restarts.
    
    So who will maintain the consistency of heartbeats port for workers' parent supervisor ?



---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152708312
  
    --- Diff: storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java ---
    @@ -1733,13 +1825,24 @@ private TopologyResources getResourcesForTopology(String topoId, StormBase base)
             return ret;
         }
     
    +    private boolean isReadyForMKAssignments() throws Exception {
    +        if(isLeader()) {
    +            if(isHeartbeatsRecovered()) {
    +                return true;
    +            }
    +            LOG.info("waiting for worker heartbeats recovery, skipping assignments");
    --- End diff --
    
    We could change this to `warn`, cause Nimbus leader should be always ready to do assignments. No assignment due to leader is normal and expected.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152702580
  
    --- Diff: storm-client/src/jvm/org/apache/storm/daemon/worker/Worker.java ---
    @@ -382,6 +374,33 @@ public void establishLogSettingCallback() {
             workerState.stormClusterState.topologyLogConfig(topologyId, this::checkLogConfigChanged);
         }
     
    +    /**
    +     * Send a heartbeat to local supervisor first to check if supervisor is ok for heartbeating.
    +     */
    +    private void heartbeatToMasterIfLocalbeatFail(LSWorkerHeartbeat lsWorkerHeartbeat) {
    +        if (ConfigUtils.isLocalMode(this.conf)) {
    +            return;
    +        }
    +        //in distributed mode, send heartbeat directly to master if local supervisor goes down
    +        SupervisorWorkerHeartbeat workerHeartbeat = new SupervisorWorkerHeartbeat(lsWorkerHeartbeat.get_topology_id(),
    +                lsWorkerHeartbeat.get_executors(), lsWorkerHeartbeat.get_time_secs());
    +        try{
    --- End diff --
    
    space between y and {


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r173043037
  
    --- Diff: storm-client/src/storm.thrift ---
    @@ -477,12 +477,13 @@ struct SupervisorInfo {
         1: required i64 time_secs;
         2: required string hostname;
         3: optional string assignment_id;
    -    4: optional list<i64> used_ports;
    -    5: optional list<i64> meta;
    -    6: optional map<string, string> scheduler_meta;
    -    7: optional i64 uptime_secs;
    -    8: optional string version;
    -    9: optional map<string, double> resources_map;
    +    4: optional i32 server_port;
    --- End diff --
    
    Okey, nice catch, the tag is changed because of merging master, i will fix it.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r173201619
  
    --- Diff: storm-client/src/jvm/org/apache/storm/assignments/ILocalAssignmentsBackend.java ---
    @@ -0,0 +1,113 @@
    +/**
    + * 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.assignments;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.storm.generated.Assignment;
    +
    +/**
    + * Interface for storing local assignments.
    + */
    +public interface ILocalAssignmentsBackend {
    +    /**
    +     * Decide if the assignments is synchronized from remote state-store.
    +     */
    +    boolean isSynchronized();
    +
    +    /**
    +     * Mark this backend as synchronized when sync work is done.
    +     */
    +    void setSynchronized();
    +
    +    /**
    +     * Initial function for creating backend.
    +     * @param conf config
    +     */
    +    void prepare(Map conf);
    +
    +    /**
    +     * Keep a storm assignment to local state or update old assignment.
    +     * @param stormID storm runtime id
    +     * @param assignment assignment as thrift
    +     */
    +    void keepOrUpdateAssignment(String stormID, Assignment assignment);
    --- End diff --
    
    nit: change `stormID` to `stormId` or `topologyId`.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    FYI: The test failure looks like unrelated to this change. 
    https://travis-ci.org/apache/storm/jobs/359434070
    This build has same failure but the build is triggered before this patch is merged in.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152702042
  
    --- Diff: storm-client/src/jvm/org/apache/storm/assignments/ILocalAssignmentsBackend.java ---
    @@ -0,0 +1,103 @@
    +/**
    + * 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.assignments;
    +
    +import org.apache.storm.generated.Assignment;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +/**
    + * Interface for storing local assignments.
    + */
    +public interface ILocalAssignmentsBackend {
    +
    +    /**
    +     * initial function for creating backend.
    +     * @param conf
    +     */
    +    void prepare(Map conf);
    +
    +    /**
    +     * Keep a storm assignment to local state or update old assignment.
    +     * @param stormID storm runtime id
    +     * @param assignment assignment as thrift
    +     */
    +    void keepOrUpdateAssignment(String stormID, Assignment assignment);
    +
    +    /**
    +     * Get assignment as byte[] for a storm
    +     * @param stormID storm runtime id
    +     * @return
    +     */
    +    Assignment getAssignment(String stormID);
    +
    +    void removeAssignment(String stormID);
    +
    +    /**
    +     * List all the storm runtime ids of local assignments
    +     * @return a list of storm ids
    +     */
    +    List<String> assignments();
    +
    +    /**
    +     * Get all the local assignments of local state
    +     * @return mapping of storm-id -> assignment
    +     */
    +    Map<String, Assignment> assignmentsInfo();
    +
    +    /**
    +     * Sync remote assignments to local, if remote is null, we will sync it from zk
    +     * @param remote specific remote assignments, if is null, it will sync from zookeeper[only used for nimbus]
    +     */
    +    void syncRemoteAssignments(Map<String, byte[]> remote);
    +
    +    /**
    +     * Keep a mapping storm-name -> storm-id to local state
    +     * @param stormName
    +     * @param stormID storm runtime id
    +     */
    +    void keepStormId(String stormName, String stormID);
    +
    +    /**
    +     * Get storm runtime id from local
    +     * @param stormName name of a storm
    +     * @return
    +     */
    +    String getStormId(String stormName);
    +
    +    /**
    +     * sync remote storm ids to local, will just used for nimbus.
    +     * @param remote
    +     */
    +    void syncRemoteIDS(Map<String, String> remote);
    --- End diff --
    
    `syncRemoteIDS` -> `syncRemoteIds` (due to checkstyle)


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    Late +1.
    
    FYI: The test looks like failing intermittently, not consistently. I've seen succeed build in my fork.
    https://travis-ci.org/HeartSaVioR/storm/builds/359567639
    
    We may want to file a new issue and investigate the issue. @revans2 What do you think?


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @danny0405 
    Updated/rebased my patch for supporting old version topology. Manually tested with 1.2.1-rc topology.
    (I needed STORM-2965 to run the topology smoothly but the patch can be applied separately.)
    
    https://github.com/HeartSaVioR/storm/tree/heartbeats-promotion-v2-support-old-workers
    https://github.com/HeartSaVioR/storm/commit/f808e53f91835bab081dd0e02e4ed4ae14f164d1


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152703031
  
    --- Diff: storm-client/src/jvm/org/apache/storm/daemon/worker/Worker.java ---
    @@ -382,6 +374,33 @@ public void establishLogSettingCallback() {
             workerState.stormClusterState.topologyLogConfig(topologyId, this::checkLogConfigChanged);
         }
     
    +    /**
    +     * Send a heartbeat to local supervisor first to check if supervisor is ok for heartbeating.
    +     */
    +    private void heartbeatToMasterIfLocalbeatFail(LSWorkerHeartbeat lsWorkerHeartbeat) {
    --- End diff --
    
    Now we're heartbeating to Supervisor directly and also fail back to Nimbus. Looks like local state and supervisor heartbeat are kind of redundant. (Not sure how it works together so please explain a bit if we need both.)


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152717599
  
    --- Diff: storm-server/pom.xml ---
    @@ -130,7 +130,7 @@
                     <artifactId>maven-checkstyle-plugin</artifactId>
                     <!--Note - the version would be inherited-->
                     <configuration>
    -                    <maxAllowedViolations>2655</maxAllowedViolations>
    +                    <maxAllowedViolations>2800</maxAllowedViolations>
    --- End diff --
    
    the Nimbus.java file is too large, and 2655 is not enough for checkstyle, you means i should move some code out?


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152706092
  
    --- Diff: storm-core/test/clj/org/apache/storm/nimbus_test.clj ---
    @@ -792,15 +795,18 @@
           (bind _ (log-message "ass1, t76, pre beat: " (pr-str ass1)))
           (bind _ (log-message "ass2, t76, pre beat: " (pr-str ass2)))
           (is (= ass1 (executor-assignment cluster storm-id executor-id1)))
    -      (is (not= ass2 (executor-assignment cluster storm-id executor-id2)))
    +      ;TODO: fix local test of worker reports
    +      ;(is (not= ass2 (executor-assignment cluster storm-id executor-id2)))
           (bind ass2 (executor-assignment cluster storm-id executor-id2))
           (bind _ (log-message "ass1, t76, post beat: " (pr-str ass1)))
           (bind _ (log-message "ass2, t76, post beat: " (pr-str ass2)))
           (check-consistency cluster "test")
     
           (.advanceClusterTime cluster 31)
    -      (is (not= ass1 (executor-assignment cluster storm-id executor-id1)))
    +      ;TODO: fix local test of worker reports
    --- End diff --
    
    Same here.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152552216
  
    --- Diff: storm-client/src/jvm/org/apache/storm/Config.java ---
    @@ -991,6 +991,31 @@
         @isPositiveNumber
         public static final String NIMBUS_QUEUE_SIZE = "nimbus.queue.size";
     
    +    /**
    --- End diff --
    
    Looks like below added configurations are related to Nimbus, then we could move them to DaemonConfig in storm-server module.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r153037176
  
    --- Diff: storm-client/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java ---
    @@ -147,21 +153,52 @@ protected void issueMapCallback(ConcurrentHashMap<String, Runnable> callbackConc
     
         @Override
         public List<String> assignments(Runnable callback) {
    +        //deprecated
    --- End diff --
    
    i will just remove it


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152894349
  
    --- Diff: storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java ---
    @@ -231,7 +183,15 @@
     
         @VisibleForTesting
         public static final List<ACL> ZK_ACLS = Arrays.asList(ZooDefs.Ids.CREATOR_ALL_ACL.get(0),
    -            new ACL(ZooDefs.Perms.READ | ZooDefs.Perms.CREATE, ZooDefs.Ids.ANYONE_ID_UNSAFE));
    +            new ACL(ZooDefs.Perms.READ ^ ZooDefs.Perms.CREATE, ZooDefs.Ids.ANYONE_ID_UNSAFE));
    --- End diff --
    
    Old nimbus.clj uses `bit-or` not `bit-xor`.
    
    https://github.com/apache/storm/blob/1.x-branch/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj#L147
    https://github.com/apache/storm/blob/1.1.x-branch/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj#L147


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r153705766
  
    --- Diff: storm-server/src/main/java/org/apache/storm/nimbus/TimeOutWorkerHeartbeatsRecoveryStrategy.java ---
    @@ -0,0 +1,86 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.nimbus;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.utils.ObjectReader;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.HashSet;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * Wait for a node to report worker heartbeats until a configured timeout. For cases below we have strategies:
    + * <p>
    + * 1: When nimbus gains leader ship, it will decide if the heartbeats are ready based on the reported node ids,
    + * supervisors/nodes will take care of the worker heartbeats recovery, a reported node id means all the workers heartbeats on the node are reported.
    + * <p>
    + * 2: If several supervisor also crush and will never recover[or all crush for some unknown reason],
    + * workers will report their heartbeats directly to master, so it has not any effect.
    + */
    +public class TimeOutWorkerHeartbeatsRecoveryStrategy implements IWorkerHeartbeatsRecoveryStrategy {
    +    private static final Logger LOG = LoggerFactory.getLogger(TimeOutWorkerHeartbeatsRecoveryStrategy.class);
    +
    +    private static int NODE_MAX_TIMEOUT_SECS = 600;
    +
    +    private long startTimeSecs;
    +
    +    private Set<String> reportedIDs;
    +
    +    @Override
    +    public void prepare(Map conf) {
    +        NODE_MAX_TIMEOUT_SECS = ObjectReader.getInt(conf.get(Config.SUPERVISOR_WORKER_HEARTBEATS_MAX_TIMEOUT_SECS), 600);
    +        this.startTimeSecs = System.currentTimeMillis() / 1000L;
    +        this.reportedIDs = new HashSet<>();
    +    }
    +
    +    @Override
    +    public boolean isReady(Set<String> nodeIds) {
    +        if (isMaxTimeOut()) {
    +            HashSet<String> tmp = new HashSet<>();
    +            for(String nodeID : nodeIds) {
    +                if (!this.reportedIDs.contains(nodeID))
    +                tmp.add(nodeID);
    +            }
    +            LOG.warn("Failed to recover heartbeats for nodes: {} with timeout {}s", tmp, NODE_MAX_TIMEOUT_SECS);
    +            return true;
    --- End diff --
    
    yes


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r158705607
  
    --- Diff: storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java ---
    @@ -4121,7 +4236,43 @@ public boolean isTopologyNameAllowed(String name) throws AuthorizationException,
                 throw new RuntimeException(e);
             }
         }
    -    
    +
    +    @Override
    +    public SupervisorAssignments getSupervisorAssignments(String node) throws AuthorizationException, TException {
    +        try {
    +            if(isLeader()) {
    +                SupervisorAssignments supervisorAssignments = new SupervisorAssignments();
    +                supervisorAssignments.set_storm_assignment(assignmentsForNode(stormClusterState.assignmentsInfo(), node));
    +                return supervisorAssignments;
    +            }
    +        } catch (Exception e) {
    +            //when not leader just return nil which will cause client to get an unknown error
    --- End diff --
    
    Friendly reminder: this comment is not addressed.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r173213324
  
    --- Diff: storm-client/src/jvm/org/apache/storm/cluster/IStormClusterState.java ---
    @@ -233,23 +249,11 @@
          * @return the id of the topology or null if it is not alive.
          */
         default Optional<String> getTopoId(final String topologyName) {
    -        String ret = null;
    -        for (String topoId: activeStorms()) {
    -            StormBase base = stormBase(topoId, null);
    -            if (base != null && topologyName.equals(base.get_name())) {
    -                ret = topoId;
    -                break;
    -            }
    -        }
    -        return Optional.ofNullable(ret);
    +        return Optional.ofNullable(stormId(topologyName));
         }
         
         default Map<String, Assignment> topologyAssignments() {
    -        Map<String, Assignment> ret = new HashMap<>();
    -        for (String topoId: assignments(null)) {
    -            ret.put(topoId, assignmentInfo(topoId, null));
    -        }
    -        return ret;
    +        return assignmentsInfo();
    --- End diff --
    
    Why do we need 2 methods that do the exact same thing, but are called different things?  Please refactor them into one.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152717836
  
    --- Diff: storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java ---
    @@ -1672,7 +1703,7 @@ private int fragmentedCpu() {
             long beforeSchedule = System.currentTimeMillis();
             scheduler.schedule(topologies, cluster);
             long scheduleTimeElapsedMs = System.currentTimeMillis() - beforeSchedule;
    -        LOG.info("Scheduling took {} ms for {} topologies", scheduleTimeElapsedMs, topologies.getTopologies().size());
    +        LOG.debug("Scheduling took {} ms for {} topologies", scheduleTimeElapsedMs, topologies.getTopologies().size());
    --- End diff --
    
    yes, it is too verbose, if we want a monitor of scheduling, we should make a metric for it but not through logging


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152706208
  
    --- Diff: storm-server/pom.xml ---
    @@ -130,7 +130,7 @@
                     <artifactId>maven-checkstyle-plugin</artifactId>
                     <!--Note - the version would be inherited-->
                     <configuration>
    -                    <maxAllowedViolations>2655</maxAllowedViolations>
    +                    <maxAllowedViolations>2800</maxAllowedViolations>
    --- End diff --
    
    This must be same or decreased. Please roll back the count and fix checkstyle issues.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @danny0405 I created https://github.com/danny0405/storm/pull/2 to add in authorization support for the supervisor and the new nimbus APIs.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r158714076
  
    --- Diff: storm-core/test/clj/org/apache/storm/nimbus_test.clj ---
    @@ -792,15 +795,18 @@
           (bind _ (log-message "ass1, t76, pre beat: " (pr-str ass1)))
           (bind _ (log-message "ass2, t76, pre beat: " (pr-str ass2)))
           (is (= ass1 (executor-assignment cluster storm-id executor-id1)))
    +      ;TODO: fix local test of worker reports
    --- End diff --
    
    I have removed it and fix the 2 TODOs already.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/storm/pull/2433


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r173615676
  
    --- Diff: storm-client/src/jvm/org/apache/storm/utils/SupervisorClient.java ---
    @@ -0,0 +1,80 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.utils;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.generated.Supervisor;
    +import org.apache.storm.security.auth.ThriftClient;
    +import org.apache.storm.security.auth.ThriftConnectionType;
    +import org.apache.thrift.transport.TTransportException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.Map;
    +
    +public class SupervisorClient extends ThriftClient {
    +    private Supervisor.Client _client;
    +    private static final Logger LOG = LoggerFactory.getLogger(SupervisorClient.class);
    +
    +    public static SupervisorClient getConfiguredClient(Map conf, String host) {
    +        int port = Integer.parseInt(conf.get(Config.SUPERVISOR_THRIFT_PORT).toString());
    +        return getConfiguredClientAs(conf, host, port, null);
    +    }
    +
    +    public static SupervisorClient getConfiguredClient(Map conf, String host, int port) {
    +        return getConfiguredClientAs(conf, host, port, null);
    +    }
    +
    +    public static SupervisorClient getConfiguredClientAs(Map conf, String host, int port, String asUser) {
    --- End diff --
    
    we need it to specify the port in order to support multiple supervisors in one node.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r173177188
  
    --- Diff: conf/defaults.yaml ---
    @@ -182,7 +195,7 @@ worker.log.level.reset.poll.secs: 30
     # control how many worker receiver threads we need per worker
     topology.worker.receiver.thread.count: 1
     
    -task.heartbeat.frequency.secs: 3
    +task.heartbeat.frequency.secs: 60
    --- End diff --
    
    If that is the case can we at a minimum update the description of the config, or better yet create a new config that is specific to metrics and deprecate this one?  My biggest concern would be for running a 1.x topology on a 2.x cluster.  The 1.x topo may get some very odd behavior.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152717096
  
    --- Diff: storm-client/src/jvm/org/apache/storm/stats/StatsUtil.java ---
    @@ -1547,26 +1533,21 @@ public static ComponentPageInfo aggCompExecsStats(
          * @param timeout       timeout
          * @return a HashMap of updated executor heart beats
          */
    -    public static Map<List<Integer>, Map<String, Object>> updateHeartbeatCache(Map<List<Integer>, Map<String, Object>> cache,
    +    public static void updateHeartbeatCache(Map<List<Integer>, Map<String, Object>> cache,
    --- End diff --
    
    yeah


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152706078
  
    --- Diff: storm-core/test/clj/org/apache/storm/nimbus_test.clj ---
    @@ -792,15 +795,18 @@
           (bind _ (log-message "ass1, t76, pre beat: " (pr-str ass1)))
           (bind _ (log-message "ass2, t76, pre beat: " (pr-str ass2)))
           (is (= ass1 (executor-assignment cluster storm-id executor-id1)))
    -      (is (not= ass2 (executor-assignment cluster storm-id executor-id2)))
    +      ;TODO: fix local test of worker reports
    --- End diff --
    
    Should this be fixed before merging patch in? If then let's document it to PR's body so that we don't miss this. If not, let's file a new JIRA issue for this.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r173210647
  
    --- Diff: storm-client/src/jvm/org/apache/storm/assignments/InMemoryAssignmentBackend.java ---
    @@ -0,0 +1,152 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.assignments;
    +
    +import java.util.ArrayList;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.storm.cluster.ClusterUtils;
    +import org.apache.storm.generated.Assignment;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * An assignment backend which will keep all assignments and id-info in memory. Only used if no backend is specified internal.
    + */
    +public class InMemoryAssignmentBackend implements ILocalAssignmentsBackend {
    +    private static final Logger LOG = LoggerFactory.getLogger(InMemoryAssignmentBackend.class);
    +
    +    protected Map<String, Assignment> idToAssignment;
    +    protected Map<String, String> idToName;
    +    protected Map<String, String> nameToId;
    +    private volatile boolean isSynchronized = false;
    +
    +    public InMemoryAssignmentBackend() {}
    +
    +    @Override
    +    public boolean isSynchronized() {
    +        return this.isSynchronized;
    +    }
    +
    +    @Override
    +    public void setSynchronized() {
    +        this.isSynchronized = true;
    +    }
    +
    +    @Override
    +    public void prepare(Map conf) {
    +        // do nothing for conf now
    +        this.idToAssignment = new ConcurrentHashMap<>();
    +        this.idToName = new ConcurrentHashMap<>();
    +        this.nameToId = new ConcurrentHashMap<>();
    +    }
    +
    +    @Override
    +    public void keepOrUpdateAssignment(String stormId, Assignment assignment) {
    +        this.idToAssignment.put(stormId, assignment);
    +    }
    +
    +    @Override
    +    public Assignment getAssignment(String stormId) {
    +        return this.idToAssignment.get(stormId);
    +    }
    +
    +    @Override
    +    public void removeAssignment(String stormId) {
    +        this.idToAssignment.remove(stormId);
    +    }
    +
    +    @Override
    +    public List<String> assignments() {
    +        if(idToAssignment == null) {
    +            return new ArrayList<>();
    +        }
    +        List<String> ret = new ArrayList<>();
    +        ret.addAll(this.idToAssignment.keySet());
    +        return ret;
    +    }
    +
    +    @Override
    +    public Map<String, Assignment> assignmentsInfo() {
    +        Map<String, Assignment> ret = new HashMap<>();
    +        ret.putAll(this.idToAssignment);
    +
    +        return ret;
    +    }
    +
    +    @Override
    +    public void syncRemoteAssignments(Map<String, byte[]> remote) {
    +        Map<String, Assignment> tmp = new ConcurrentHashMap<>();
    --- End diff --
    
    The issue I am seeing is that all of this appears to not be thread safe, even with the ConcurrentHashMap.
    
    `idToAssignment`, `idToName`, `nameToId` all are set and read in different methods that all appear to be able to be called from different threads.  As such there is the real possibility that they state will not be consistent.  If we don't care too much about that it might be okay, but if we are going with an eventual consistency like route for this we should at least document it.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r158737793
  
    --- Diff: storm-core/test/clj/org/apache/storm/nimbus_test.clj ---
    @@ -792,15 +795,18 @@
           (bind _ (log-message "ass1, t76, pre beat: " (pr-str ass1)))
           (bind _ (log-message "ass2, t76, pre beat: " (pr-str ass2)))
           (is (= ass1 (executor-assignment cluster storm-id executor-id1)))
    +      ;TODO: fix local test of worker reports
    --- End diff --
    
    I pulled the latest update of PR to local but they're still existing in nimbus_test.clj. Please check again.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152704498
  
    --- Diff: storm-client/src/jvm/org/apache/storm/stats/StatsUtil.java ---
    @@ -1589,18 +1570,16 @@ public static ComponentPageInfo aggCompExecsStats(
                 if (lastReportedTime != null) {
                     reportedTime = lastReportedTime;
                 } else {
    -                reportedTime = 0;
    +                reportedTime = lastReportedTime = 0;
    --- End diff --
    
    This change doesn't look like needed.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @danny0405 Yeah it was holiday in South Korea as well but much shorter than China. I just would wanted to check you still want to follow up with this patch.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r173215187
  
    --- Diff: storm-client/src/jvm/org/apache/storm/daemon/worker/Worker.java ---
    @@ -392,6 +401,30 @@ public void establishLogSettingCallback() {
             workerState.stormClusterState.topologyLogConfig(topologyId, this::checkLogConfigChanged);
         }
     
    +    /**
    +     * Send a heartbeat to local supervisor first to check if supervisor is ok for heartbeating.
    +     */
    +    private void heartbeatToMasterIfLocalbeatFail(LSWorkerHeartbeat lsWorkerHeartbeat) {
    +        if (ConfigUtils.isLocalMode(this.conf)) {
    --- End diff --
    
    If you want to make this work in local mode we can do it.  For nimbus we have an override in NimbusClient so it will return the local nimbus instead.  We could do the same thing for a Supervisor with a specific port number too.  That would make local mode look and act a lot more like non-local mode.  But it is something we can do in a follow on JIRA.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @danny0405 
    Now Bobby's patch for STORM-2898 is merged. Could you start working on follow-up work based on STORM-2898? Thanks in advance!


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r158706034
  
    --- Diff: storm-server/src/main/java/org/apache/storm/nimbus/TimeOutWorkerHeartbeatsRecoveryStrategy.java ---
    @@ -0,0 +1,86 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.nimbus;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.utils.ObjectReader;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.HashSet;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * Wait for a node to report worker heartbeats until a configured timeout. For cases below we have strategies:
    + * <p>
    + * 1: When nimbus gains leader ship, it will decide if the heartbeats are ready based on the reported node ids,
    + * supervisors/nodes will take care of the worker heartbeats recovery, a reported node id means all the workers heartbeats on the node are reported.
    + * <p>
    + * 2: If several supervisor also crush and will never recover[or all crush for some unknown reason],
    + * workers will report their heartbeats directly to master, so it has not any effect.
    + */
    +public class TimeOutWorkerHeartbeatsRecoveryStrategy implements IWorkerHeartbeatsRecoveryStrategy {
    +    private static final Logger LOG = LoggerFactory.getLogger(TimeOutWorkerHeartbeatsRecoveryStrategy.class);
    +
    +    private static int NODE_MAX_TIMEOUT_SECS = 600;
    +
    +    private long startTimeSecs;
    +
    +    private Set<String> reportedIDs;
    +
    +    @Override
    +    public void prepare(Map conf) {
    +        NODE_MAX_TIMEOUT_SECS = ObjectReader.getInt(conf.get(Config.SUPERVISOR_WORKER_HEARTBEATS_MAX_TIMEOUT_SECS), 600);
    +        this.startTimeSecs = System.currentTimeMillis() / 1000L;
    --- End diff --
    
    Friendly reminder: this is not addressed yet.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @revans2
    Really thx for your contributions and refactoring work, i will follow in timely if i have time.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r154558915
  
    --- Diff: storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java ---
    @@ -231,7 +183,15 @@
     
         @VisibleForTesting
         public static final List<ACL> ZK_ACLS = Arrays.asList(ZooDefs.Ids.CREATOR_ALL_ACL.get(0),
    -            new ACL(ZooDefs.Perms.READ | ZooDefs.Perms.CREATE, ZooDefs.Ids.ANYONE_ID_UNSAFE));
    +            new ACL(ZooDefs.Perms.READ ^ ZooDefs.Perms.CREATE, ZooDefs.Ids.ANYONE_ID_UNSAFE));
    --- End diff --
    
    yeah, you are right


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r158758374
  
    --- Diff: storm-client/src/jvm/org/apache/storm/assignments/LocalAssignmentsBackendFactory.java ---
    @@ -0,0 +1,48 @@
    +/**
    + * 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.assignments;
    +
    +import com.google.common.base.Preconditions;
    +import org.apache.storm.Config;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.ReflectionUtils;
    +
    +import java.util.Map;
    +
    +/**
    + * Factory class for creating local assignments.
    + */
    +public class LocalAssignmentsBackendFactory {
    +
    +    public static ILocalAssignmentsBackend getBackend(Map<String, Object> conf) {
    +        if (conf.get(Config.NIMBUS_LOCAL_ASSIGNMENTS_BACKEND_CLASS) != null) {
    +            Object targetObj = ReflectionUtils.newInstance((String) conf.get(Config.NIMBUS_LOCAL_ASSIGNMENTS_BACKEND_CLASS));
    +            Preconditions.checkState(targetObj instanceof ILocalAssignmentsBackend, "{} must implements ILocalAssignmentsBackend", Config.NIMBUS_LOCAL_ASSIGNMENTS_BACKEND_CLASS);
    +            ((ILocalAssignmentsBackend)targetObj).prepare(conf);
    +            return (ILocalAssignmentsBackend) targetObj;
    +        }
    +
    +        return getDefault();
    --- End diff --
    
    then what if the Object is instanceof IStateStorage ? should i get the conf from local yaml again?


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by danny0405 <gi...@git.apache.org>.
Github user danny0405 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152716688
  
    --- Diff: storm-client/src/jvm/org/apache/storm/daemon/worker/WorkerState.java ---
    @@ -51,12 +51,7 @@
     import org.apache.storm.task.WorkerTopologyContext;
     import org.apache.storm.tuple.AddressedTuple;
     import org.apache.storm.tuple.Fields;
    -import org.apache.storm.utils.ConfigUtils;
    -import org.apache.storm.utils.Utils;
    -import org.apache.storm.utils.DisruptorQueue;
    -import org.apache.storm.utils.ObjectReader;
    -import org.apache.storm.utils.ThriftTopologyUtils;
    -import org.apache.storm.utils.TransferDrainer;
    +import org.apache.storm.utils.*;
    --- End diff --
    
    okey, i will fix it


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152898026
  
    --- Diff: storm-server/src/main/java/org/apache/storm/daemon/supervisor/timer/SynchronizeAssignments.java ---
    @@ -0,0 +1,133 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.daemon.supervisor.timer;
    +
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.daemon.supervisor.ReadClusterState;
    +import org.apache.storm.daemon.supervisor.Supervisor;
    +import org.apache.storm.generated.Assignment;
    +import org.apache.storm.generated.SupervisorAssignments;
    +import org.apache.storm.utils.ConfigUtils;
    +import org.apache.storm.utils.NimbusClient;
    +import org.apache.storm.utils.Utils;
    +import org.apache.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.HashMap;
    +import java.util.Map;
    +
    +/**
    + * A runnable which will synchronize assignments to node local and then worker processes.
    + */
    +public class SynchronizeAssignments implements Runnable {
    +    private static final Logger LOG = LoggerFactory.getLogger(SynchronizeAssignments.class);
    +
    +    private Supervisor supervisor;
    +    private SupervisorAssignments assignments;
    +    private ReadClusterState readClusterState;
    +
    +    public SynchronizeAssignments(Supervisor supervisor, SupervisorAssignments assignments, ReadClusterState readClusterState) {
    +        this.supervisor = supervisor;
    +        this.assignments = assignments;
    +        this.readClusterState = readClusterState;
    +    }
    +
    +    @Override
    +    public void run() {
    +        // first sync assignments to local, then sync processes.
    +        if (null == assignments) {
    +            getAssignmentsFromMaster(this.supervisor.getConf(), this.supervisor.getStormClusterState(), this.supervisor.getAssignmentId());
    +        } else {
    +            assignedAssignmentsToLocal(this.supervisor.getStormClusterState(), assignments);
    +        }
    +        this.readClusterState.run();
    +    }
    +
    +    /**
    +     * Used by {@link Supervisor} to fetch assignments when start up.
    +     * @param supervisor
    +     */
    +    public void getAssignmentsFromMasterUntilSuccess(Supervisor supervisor) {
    +        boolean success = false;
    +        NimbusClient master;
    --- End diff --
    
    Same here: use try-with-resource.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152693678
  
    --- Diff: storm-client/src/jvm/org/apache/storm/Config.java ---
    @@ -1293,6 +1318,44 @@
         @isPositiveNumber
         public static final String SUPERVISOR_CPU_CAPACITY = "supervisor.cpu.capacity";
     
    +    @isInteger
    --- End diff --
    
    Same here: leave configurations only when worker needs to know. For other things we could move them to DaemonConfig as well.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @danny0405 
    Sorry I should explain the motivation before. The motivation is well explained to the description on https://issues.apache.org/jira/browse/STORM-2448.
    
    Explaining background as far as I know, some vendors which leverages Storm (via product, or cluster) are always struggling with ensuring users to upgrade their clusters (or topologies) smoothly. It should be ideal if all users are using most recent version of Storm, but it is sadly true that someone still even uses Storm 0.10.x in their production. 
    
    The team which makes one of Storm's biggest contribution is also struggling on this. They have put noticeable efforts to RAS in Storm 2.0.0 which really helps utilizing huge cluster (I guess it is  centralized), but it is even not easy for them to persuade their all of clients to upgrade their topologies. (I'm not quite sure it is similar case for them but you may understand the case if you are working on company which number of employees goes over 1000~.)
    
    STORM-2448 comes in as a workaround for addressing above issue. We should (must, I wish) drop supporting them from Storm 3.0.0, but we may need to live with this for now.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152702341
  
    --- Diff: storm-client/src/jvm/org/apache/storm/daemon/worker/Worker.java ---
    @@ -50,25 +51,14 @@
     import org.apache.storm.executor.ExecutorShutdown;
     import org.apache.storm.executor.IRunningExecutor;
     import org.apache.storm.executor.LocalExecutor;
    -import org.apache.storm.generated.Credentials;
    -import org.apache.storm.generated.ExecutorInfo;
    -import org.apache.storm.generated.ExecutorStats;
    -import org.apache.storm.generated.LSWorkerHeartbeat;
    -import org.apache.storm.generated.LogConfig;
    +import org.apache.storm.generated.*;
    --- End diff --
    
    Using wildcard(*) in import is discouraged in Storm project. If it is automated by IDE, please increase the count (I'm using 999) to convert imports to *.


---

[GitHub] storm pull request #2433: [STORM-2693] Heartbeats and assignments promotion ...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2433#discussion_r152698394
  
    --- Diff: storm-client/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java ---
    @@ -147,21 +153,52 @@ protected void issueMapCallback(ConcurrentHashMap<String, Runnable> callbackConc
     
         @Override
         public List<String> assignments(Runnable callback) {
    +        //deprecated
             if (callback != null) {
                 assignmentsCallback.set(callback);
             }
    -        return stateStorage.get_children(ClusterUtils.ASSIGNMENTS_SUBTREE, callback != null);
    +        return this.backend.assignments();
         }
     
         @Override
         public Assignment assignmentInfo(String stormId, Runnable callback) {
    +        //deprecated
    +        if (callback != null) {
    +            assignmentInfoCallback.put(stormId, callback);
    +        }
    +        return this.backend.getAssignment(stormId);
    +    }
    +
    +    @Override
    +    public Assignment remoteAssignmentInfo(String stormId, Runnable callback) {
    +        //deprecated
    --- End diff --
    
    Same here.


---

[GitHub] storm issue #2433: [STORM-2693] Heartbeats and assignments promotion for sto...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/2433
  
    @danny0405 Sorry about how long this has taken.  I am back from vacation now. I will take a look at the patch again, and if the conflicts are small hopefully we can merge it in today or tomorrow.


---