You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@storm.apache.org by schonfeld <gi...@git.apache.org> on 2015/11/17 04:30:52 UTC

[GitHub] storm pull request: [STORM-126] Add Lifecycle support API for work...

GitHub user schonfeld opened a pull request:

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

    [STORM-126] Add Lifecycle support API for worker nodes

    This PR is prob the 112th attempt at this ticket... But I'd really like support for worker hooks... So, here's my crack at it. Still missing tests -- I'd love some guidance re what exactly I should be testing/where I should be testing it. Thanks!

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

    $ git pull https://github.com/schonfeld/storm worker-hooks

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

    https://github.com/apache/storm/pull/884.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 #884
    
----
commit 3269caa2616734694799e81fd0fbad3066bdc538
Author: Michael Schonfeld <mi...@schonfeld.org>
Date:   2015-11-16T19:48:45Z

    DebugOptions needs to be above TopologyInfo

commit cc166d640aca069254d7e48309330b1250f4d349
Author: Michael Schonfeld <mi...@schonfeld.org>
Date:   2015-11-16T19:49:06Z

    add worker hooks thirft def

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-126] Add Lifecycle support API for work...

Posted by schonfeld <gi...@git.apache.org>.
Github user schonfeld commented on the pull request:

    https://github.com/apache/storm/pull/884#issuecomment-157600767
  
    ping @revans2 @harshach -- sorry to bug, but, mind taking a look?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-126] Add Lifecycle support API for work...

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

    https://github.com/apache/storm/pull/884#issuecomment-157780934
  
    @schonfeld the reason for doing it per component is because you have a framework, like spring, that wants to be setup and cleaned up, but only a subset of the components use it.  I supposed having the worker hook run on workers that don't use spring probably is not that bad, but I can also see that if it adds extra overhead some people may want to turn it off.
    
    I am still +1 on this change.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-126] Add Lifecycle support API for work...

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

    https://github.com/apache/storm/pull/884#discussion_r45212112
  
    --- 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 {
    +    private static final long serialVersionUID = 2589466485198339529L;
    +
    +    @Override
    +    public void start(Map stormConf, WorkerTopologyContext context, List taskIds) {
    +
    +    }
    +
    +    @Override
    +    public void shutdown() {
    +    }
    --- End diff --
    
    Similar comment here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-126] Add Lifecycle support API for work...

Posted by revans2 <gi...@git.apache.org>.
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.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-126] Add Lifecycle support API for work...

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

    https://github.com/apache/storm/pull/884#discussion_r45212589
  
    --- Diff: storm-core/test/jvm/backtype/storm/utils/ThriftTopologyUtilsTest.java ---
    @@ -0,0 +1,94 @@
    +/**
    + * 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.utils;
    +
    +import backtype.storm.generated.*;
    +import backtype.storm.hooks.BaseWorkerHook;
    +import com.google.common.collect.ImmutableMap;
    +import com.google.common.collect.ImmutableSet;
    +import junit.framework.TestCase;
    +import org.junit.Assert;
    +import org.junit.Test;
    +
    +import java.nio.ByteBuffer;
    +import java.util.Set;
    +
    +public class ThriftTopologyUtilsTest extends TestCase {
    --- End diff --
    
    For JUNIT4 I don't think you should extend TestCase.  Is there a reason you are doing it?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-126] Add Lifecycle support API for work...

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

    https://github.com/apache/storm/pull/884#discussion_r45213086
  
    --- Diff: storm-core/src/jvm/backtype/storm/hooks/IWorkerHook.java ---
    @@ -0,0 +1,29 @@
    +/**
    + * 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 interface IWorkerHook extends Serializable {
    +    void start(Map stormConf, WorkerTopologyContext context, List taskIds);
    +    void shutdown();
    +}
    --- End diff --
    
    Especially describing the type that taskIds is.  It would be nice to put in generics for it if we could.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-126] Add Lifecycle support API for work...

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

    https://github.com/apache/storm/pull/884#discussion_r45212243
  
    --- Diff: storm-core/src/jvm/backtype/storm/hooks/IWorkerHook.java ---
    @@ -0,0 +1,29 @@
    +/**
    + * 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 interface IWorkerHook extends Serializable {
    +    void start(Map stormConf, WorkerTopologyContext context, List taskIds);
    +    void shutdown();
    +}
    --- End diff --
    
    Could you put in some javadoc comments about this interface and the methods in it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-126] Add Lifecycle support API for work...

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

    https://github.com/apache/storm/pull/884#issuecomment-157750106
  
    Overall this looks good.  I have a few minor comments, mostly around code cleanup.  My biggest concern is that we seem to be installing the IWorkerHook on all workers, but not providing a simple way for them to selectively decide if they want to run on a per-bolt/spout basis.  But that can be done later on if we need it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-126] Add Lifecycle support API for work...

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

    https://github.com/apache/storm/pull/884#discussion_r45212086
  
    --- 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 {
    +    private static final long serialVersionUID = 2589466485198339529L;
    +
    +    @Override
    +    public void start(Map stormConf, WorkerTopologyContext context, List taskIds) {
    +
    --- End diff --
    
    Could we put a comment in the body here like `//NOOP`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-126] Add Lifecycle support API for work...

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

    https://github.com/apache/storm/pull/884#issuecomment-157771089
  
    @schonfeld we can make it work easily with specific components.  The taskIds are not needed when calling the start method of IWorkerHook.  You can get them out by calling getThisWorkerTasks of the WorkerTopologyContext.  But that too is really minor.  You can also call getComponentId using the task id to get the component ID.
    
    So if we wanted to we could have something like (probably does not compile)
    ```java
    public class ComponentWorkerHook implements IWorkerHook {
        private final IWorkerHook _wrapped;
        private final Set<String> _comps;
        private boolean _active = false;
    
        public ComponentWorkerHook(IWorkerHook wrapped, Set<String> comps) {
            _wrapped = wrapped;
            _comps = new HashSet<>(comps);
        }
    
        public void addComp(String comp) {
            _comps.add(comp);
        }
    
       @Override
        public void start(Map stormConf, WorkerTopologyContext context, List<Integer> taskIds) {
             Set<String> found = new HashSet<String>();
             for (Integer id: taskids) {
                 found.put(context.getComponentId(id));
             }
             found.retainAll(_comps);
             _active = !found.isEmpty();
            if (_active) {
                _wrapped.start(stromConf, context, taskIds);
            }
        }
    
        @Override
        public void shutdown() {
            if (_active) {
               _wrapped.shutdown();
            }
        }
    }
    ```
    
    I am +1 for merging this in, but I want to hear from @Parth-Brahmbhatt if he really wants to change this to be config based instead of through TopologyBuilder.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-126] Add Lifecycle support API for work...

Posted by Parth-Brahmbhatt <gi...@git.apache.org>.
Github user Parth-Brahmbhatt commented on the pull request:

    https://github.com/apache/storm/pull/884#issuecomment-158451109
  
    I'm +1 too.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-126] Add Lifecycle support API for work...

Posted by schonfeld <gi...@git.apache.org>.
Github user schonfeld commented on the pull request:

    https://github.com/apache/storm/pull/884#issuecomment-159107291
  
    @revans2 all good! we actually caught a bug (in that last commit apache/storm@9bc9350f613a75de566cf670874c6a037ecc5bfd) when we ran this in prod today.
    
    Anyway, rebased & pushed. Just gotta wait for the tests to finish...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-126] Add Lifecycle support API for work...

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

    https://github.com/apache/storm/pull/884#discussion_r45213316
  
    --- Diff: storm-core/test/jvm/backtype/storm/utils/ThriftTopologyUtilsTest.java ---
    @@ -0,0 +1,94 @@
    +/**
    + * 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.utils;
    +
    +import backtype.storm.generated.*;
    +import backtype.storm.hooks.BaseWorkerHook;
    +import com.google.common.collect.ImmutableMap;
    +import com.google.common.collect.ImmutableSet;
    +import junit.framework.TestCase;
    +import org.junit.Assert;
    +import org.junit.Test;
    +
    +import java.nio.ByteBuffer;
    +import java.util.Set;
    +
    +public class ThriftTopologyUtilsTest extends TestCase {
    --- End diff --
    
    I did that because all other test classes in `backtype.storm.utils` (DisruptorQueueBackpressureTest, DisruptorQueueTest, StormBoundedExponentialBackoffRetryTest, etc) extend `TestCase`... Should I remove?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-126] Add Lifecycle support API for work...

Posted by schonfeld <gi...@git.apache.org>.
Github user schonfeld commented on the pull request:

    https://github.com/apache/storm/pull/884#issuecomment-157777138
  
    @revans2 yah, you're right.. I'll go ahead and remove the taskIds from @start. re ComponentWorkerHook... I can't think of a reason you'd want to run the a WorkerHook only on specific components... And if you do want that, shouldn't we just implement `start` & `shutdown` methods in `IComponent`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-126] Add Lifecycle support API for work...

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

    https://github.com/apache/storm/pull/884#discussion_r45221046
  
    --- Diff: storm-core/test/jvm/backtype/storm/utils/ThriftTopologyUtilsTest.java ---
    @@ -0,0 +1,94 @@
    +/**
    + * 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.utils;
    +
    +import backtype.storm.generated.*;
    +import backtype.storm.hooks.BaseWorkerHook;
    +import com.google.common.collect.ImmutableMap;
    +import com.google.common.collect.ImmutableSet;
    +import junit.framework.TestCase;
    +import org.junit.Assert;
    +import org.junit.Test;
    +
    +import java.nio.ByteBuffer;
    +import java.util.Set;
    +
    +public class ThriftTopologyUtilsTest extends TestCase {
    --- End diff --
    
    It is very minor I am fine with leaving it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-126] Add Lifecycle support API for work...

Posted by Parth-Brahmbhatt <gi...@git.apache.org>.
Github user Parth-Brahmbhatt commented on the pull request:

    https://github.com/apache/storm/pull/884#issuecomment-157749683
  
    Thanks for the contribution, any reason you decided to make this hook part of serialized topology. If you see some other examples, like the nimbus hook (though it is not really a great example as its on nimbus side and not worker but still close) we ask the user to just provide the Fully qualified classname as a config option, create an instance of it using reflection and invoke prepare (start in your case) or cleanup on that instance. The only advantage of putting this as part of topology is that users will be able to provide objects that are completely serialized so it can be initialized with constructor args or with any other way that relies on instance variable initialization but I don't see that as a huge upside. On the other hand a consistent way to implement all hooks will make code easy to read and reason about.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-126] Add Lifecycle support API for work...

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

    https://github.com/apache/storm/pull/884#issuecomment-159090867
  
    @schonfeld sorry this took so long.  Could you rebase again?  I am going to be out for the Thanksgiving Holiday, so @Parth-Brahmbhatt no need to wait for me to give a +1 again.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-126] Add Lifecycle support API for work...

Posted by schonfeld <gi...@git.apache.org>.
Github user schonfeld commented on the pull request:

    https://github.com/apache/storm/pull/884#issuecomment-157751007
  
    @Parth-Brahmbhatt @revans2 to be honest, the reason I implement this as a worker-specific serializable hook is cos that's what @nathanmarz suggested in the discussion on [STORM-126](https://issues.apache.org/jira/browse/STORM-126)... Seemed like a logical approach, since it is bound to a worker and not a specific component..?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-126] Add Lifecycle support API for work...

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

    https://github.com/apache/storm/pull/884#discussion_r45211932
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/worker.clj ---
    @@ -665,6 +686,8 @@
                         (close-resources worker)
     
                         ;; TODO: here need to invoke the "shutdown" method of WorkerHook
    --- End diff --
    
    Please remove this TODO, you implemented it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-126] Add Lifecycle support API for work...

Posted by schonfeld <gi...@git.apache.org>.
Github user schonfeld commented on the pull request:

    https://github.com/apache/storm/pull/884#issuecomment-157439890
  
    Added some tests around `ThriftTopologyUtils`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-126] Add Lifecycle support API for work...

Posted by schonfeld <gi...@git.apache.org>.
Github user schonfeld commented on the pull request:

    https://github.com/apache/storm/pull/884#issuecomment-158452483
  
    Fantastic! I'll rebase on master real quick...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-126] Add Lifecycle support API for work...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---