[
https://issues.apache.org/jira/browse/STORM-126?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15011179#comment-15011179
]
ASF GitHub Bot commented on STORM-126:
--------------------------------------
Github user revans2 commented on a diff in the pull request:
https://github.com/apache/storm/pull/884#discussion_r45212055
--- Diff: storm-core/src/jvm/backtype/storm/hooks/BaseWorkerHook.java ---
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package backtype.storm.hooks;
+
+import backtype.storm.task.WorkerTopologyContext;
+
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+
+public class BaseWorkerHook implements IWorkerHook, Serializable {
--- End diff --
Could you add in some javadoc comments.
> Add Lifecycle support API for worker nodes
> ------------------------------------------
>
> Key: STORM-126
> URL: https://issues.apache.org/jira/browse/STORM-126
> Project: Apache Storm
> Issue Type: New Feature
> Components: storm-core
> Reporter: James Xu
> Assignee: Michael Schonfeld
> Priority: Minor
> Fix For: 0.11.0
>
>
> https://github.com/nathanmarz/storm/issues/155
> Storm is already used in variety of environments. It is important that Storm
> provides some form of "lifecycle" API specified at Topology builder level to
> be called when worker nodes start and stop.
> It is a very crucial functional piece that is missing from Storm. Many
> project have to integrate, for example, with various container-like
> frameworks like Spring or Google Guice that need to be started at stopped in
> a controlled fashion before worker nodes begin or finish their work.
> I think something like a WorkerContextListener interface with two methods:
> onStartup(SomeContextClass) and onShutdown(SomeContextClass) could go a very
> long way for allowing to user's to plugin various third-party libraries
> easily.
> Then, the TopologyBuilder needs to be modified to accept classes that
> implement this interface.
> SomeContextClass does not need to be much more than a Map for now. But it is
> important to have it as it allows propagation ofl information between those
> lifecycle context listeners.
> Nathan, it would interesting to hear your opinion.
> Thanks!
> ----------
> nathanmarz: I agree, this should be added to Storm. The lifecycle methods
> should be parameterized with which tasks are running in this worker.
> Additionally, I think lifecycle methods should be added for bolt/spouts in
> the context of workers. Sometimes there's some code you want to run for a
> spout/bolt within a worker only one time, regardless of how many tasks for
> that bolt are within the worker. Then individual tasks should be able to
> access that "global state" within the worker for that spout/bolt.
> ----------
> kyrill007: Thank you, Nathan, I think it would be relatively simple to
> implement and would have big impact. Now we're forced to manage container
> initializations through lazy static fields. You'd love to see that code. :-)
> ----------
> nathanmarz: Yup, this should be fairly easy to implement. I encourage you to
> submit a patch for this.
> ----------
> kyrill007: Oh, well, my Clojure is unfortunately too weak for this kind of
> work. But I am working on it... Any pointers as to where in Storm code
> workers are started and stopped?
> ----------
> nathanmarz: Here's the function that's called to start a worker:
> https://github.com/nathanmarz/storm/blob/master/src/clj/backtype/storm/daemon/worker.clj#L315
> And here's the code in the same file that shuts down a worker:
> https://github.com/nathanmarz/storm/blob/master/src/clj/backtype/storm/daemon/worker.clj#L352
> I think the interface for the lifecycle stuff should look something like this:
> interface WorkerHook extends Serializable {
> void start(Map conf, TopologyContext context, List taskIds);
> void shutdown();
> }
> You'll need to add a definition for worker hooks into the topology definition
> Thrift structure:
> https://github.com/nathanmarz/storm/blob/master/src/storm.thrift#L91
> I think for the first go it's ok to make this a Java-only feature by adding
> something like "4: list<binary> worker_hooks" to the StormTopology structure
> (where the "binary" refers to a Java-serialized object).
> Then TopologyBuilder can have a simple "addWorkerHook" method that will
> serialize the object and add it to the Thrift struct.
> ----------
> danehammer: I've started working on this. I've followed Nathan's proposed
> design, but I keep hitting snags with calls to ThriftTopologyUtils, now that
> there is an optional list on StormTopology.
> I would like to add some unit tests for what I change there, would it make
> more sense for those to be in Java instead of Clojure? If so, are there any
> strong preferences on what dependencies I add and how I go about adding Java
> unit tests to storm-core?
> ----------
> nathanmarz: No... unit tests should remain in Clojure. You can run Java code
> in Clojure very easily. Here's a good example of testing Java code in
> Clojure:
> https://github.com/nathanmarz/storm/blob/master/storm-core/test/clj/backtype/storm/security/auth/AuthUtils_test.clj
> ----------
> danehammer: For this design:
> interface WorkerHook extends Serializable {
> void start(Map conf, TopologyContext context, List taskIds);
> void shutdown();
> }
> When mk-worker has the worker hooks and goes to call start on them, where
> should it get the topology context? I started with what was returned by
> (worker-context worker), which returns a WorkerTopologyContext, but that
> doesn't appear to be correct.
> ----------
> danehammer: If this helps, here's what I'm looking at right now:
> https://github.com/danehammer/storm/blob/issue-155/storm-core/src/clj/backtype/storm/daemon/worker.clj#L329
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)