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

Stig Rohde Døssing commented on STORM-2175:
-------------------------------------------

[~revans2] 
Thanks, this is a good change, and most of our tests now pass (though not 
entirely reliably). I'm running into the timeout for the test using the Kafka 
spout fairly consistently though. I've posted a small PR against the 1.x 
version of the branch to disable the timeout in local mode. Tests may fail 
spuriously due to a slow machine, or bolts/spouts being slow to close.
 
Raising the timeout in shutdownAllWorkers isn't guaranteed to work for 
everyone, and making it configurable is fiddly in tests and irrelevant for 
distributed mode, due to the 1 second limit on worker suicide. Giving up on 
shutting down the worker properly leads to leaking resources in local mode, 
since we can't just kill the JVM. I think removing the timeout for 
shutdownAllWorkers in local mode is a decent fix. It's still very visible if a 
Slot is hanging for some reason, since the warning about a slow Slot shutdown 
is still printed, and we avoid dealing with issues like Slots being slow to 
shutdown due to tests with a high fork count, or tests on slow machines, or 
optimistic bolt/spout close functions that don't time limit themselves.

> Supervisor V2 can possibly shut down workers twice in local mode
> ----------------------------------------------------------------
>
>                 Key: STORM-2175
>                 URL: https://issues.apache.org/jira/browse/STORM-2175
>             Project: Apache Storm
>          Issue Type: Bug
>          Components: storm-core
>    Affects Versions: 2.0.0, 1.1.0
>            Reporter: Robert Joseph Evans
>            Assignee: Robert Joseph Evans
>          Time Spent: 1h
>  Remaining Estimate: 0h
>
> See https://github.com/apache/storm/pull/1697#issuecomment-256456889
> {code}
> java.lang.NullPointerException
>     at 
> org.apache.storm.utils.DisruptorQueue$FlusherPool.stop(DisruptorQueue.java:110)
>     at 
> org.apache.storm.utils.DisruptorQueue$Flusher.close(DisruptorQueue.java:293)
>     at 
> org.apache.storm.utils.DisruptorQueue.haltWithInterrupt(DisruptorQueue.java:410)
>     at 
> org.apache.storm.disruptor$halt_with_interrupt_BANG_.invoke(disruptor.clj:77)
>     at 
> org.apache.storm.daemon.executor$mk_executor$reify__4923.shutdown(executor.clj:412)
>     at sun.reflect.GeneratedMethodAccessor303.invoke(Unknown Source)
>     at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>     at java.lang.reflect.Method.invoke(Method.java:498)
>     at clojure.lang.Reflector.invokeMatchingMethod(Reflector.java:93)
>     at clojure.lang.Reflector.invokeNoArgInstanceMember(Reflector.java:313)
>     at 
> org.apache.storm.daemon.worker$fn__5550$exec_fn__1372__auto__$reify__5552$shutdown_STAR___5572.invoke(worker.clj:668)
>     at 
> org.apache.storm.daemon.worker$fn__5550$exec_fn__1372__auto__$reify$reify__5598.shutdown(worker.clj:706)
>     at org.apache.storm.ProcessSimulator.killProcess(ProcessSimulator.java:66)
>     at 
> org.apache.storm.ProcessSimulator.killAllProcesses(ProcessSimulator.java:79)
>     at 
> org.apache.storm.testing$kill_local_storm_cluster.invoke(testing.clj:207)
>     at org.apache.storm.testing4j$_withLocalCluster.invoke(testing4j.clj:93)
>     at org.apache.storm.Testing.withLocalCluster(Unknown Source)
> {code}
> and
> {code}
> java.lang.IllegalStateException: Timer is not active
>     at org.apache.storm.timer$check_active_BANG_.invoke(timer.clj:87)
>     at org.apache.storm.timer$cancel_timer.invoke(timer.clj:120)
>     at 
> org.apache.storm.daemon.worker$fn__5550$exec_fn__1372__auto__$reify__5552$shutdown_STAR___5572.invoke(worker.clj:682)
>     at 
> org.apache.storm.daemon.worker$fn__5550$exec_fn__1372__auto__$reify$reify__5598.shutdown(worker.clj:706)
>     at org.apache.storm.ProcessSimulator.killProcess(ProcessSimulator.java:66)
>     at 
> org.apache.storm.ProcessSimulator.killAllProcesses(ProcessSimulator.java:79)
>     at 
> org.apache.storm.testing$kill_local_storm_cluster.invoke(testing.clj:207)
>     at org.apache.storm.testing4j$_withLocalCluster.invoke(testing4j.clj:93)
>     at org.apache.storm.Testing.withLocalCluster(Unknown Source)
> {code}
> [~Srdo] is still working on getting a reproducible use case for us. But I 
> will try to reproduce/fix it myself in the mean time.



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

Reply via email to