Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

STORM-200 Proposal for Multilang's Metrics feature #38

Merged
merged 22 commits into from
Jul 1, 2014
Merged

STORM-200 Proposal for Multilang's Metrics feature #38

merged 22 commits into from
Jul 1, 2014

Conversation

dashengju
Copy link
Contributor

for https://issues.apache.org/jira/browse/STORM-200

Storm 0.9.2 exposes a metrics interface to report summary statistics across the full topology. We can build our own metric, and build metrics consumer to use those statistics.

But when we use Multilang(ie. Python), we can not use this feature. So we want to summit a proposal for multilang's metrics.


The specifics of the proposal:

  1. The main idea is: when user want to add a metric statistics in multilang(python) spout/bolt,
    a) first, create a metric object and register in ShellSpout/ShellBolt's sub-class,
    b) then update the metric in Python spout/bolt process through RPC call.
  2. Extends Metrics API with RPC:
    In package backtype.storm.metric.api add new package backtype.storm.metric.api.rpc, which includes:
    a) IShellMetric interface, which extends IMetirc interface, to support:
    public void updateMetricFromRPC(Object params);
    b) IShellMetric implemention: AssignableShellMetric, CombinedShellMetric, CountShellMetirc, ReducedShellMetric;
  3. Extends Multilang protocol:
    a) add a command "metrics" for shell process to make RPC call. The protocol is:
    {"command":"metrics", "name":"metric-registerd-name", "params": param-value}
    b) moidfy ShellSpout/ShellBolt to support command metrics: we define a method handleMetrics to handle RPC call from Python spout/bolt process;
    c) modify TopologyContext.java, add getRegisteredMetricByName to get registerd metric to update;
    c) storm.py add method rpcMetrics(name, params), user can update remote metric through this RPC call.
  4. Extends TopologyContext add getRegisteredMetricByName to get a component's registered metric to update through rpc;

--------usage example----------------------------------------------------------------------------------------------------------------
--------ExamplePythonBolt.java----------------------------------------------------------------------------------------------------

public class ExamplePythonBolt extends ShellBolt implements IRichBolt {

public ExamplePythonBolt() {
    super("python", "ExampleBolt.py");
}

   public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
       super.prepare(stormConf, context, collector);

       //generate a metric object
       CountShellMetric cMetric = new CountShellMetric(); 
       //register metric object to context             
       context.registerMetric("PythonBoltCount", cMetric, 120);  
   }

public void declareOutputFields(OutputFieldsDeclarer declarer) {
    declarer.declare(new Fields("LogRecord"));
}

public Map<String, Object> getComponentConfiguration() {
    return null;
}
}

--------ExampleBolt.py---------------------------------------------------------------------------------------------------

class ExampleBolt(storm.BasicBolt):

    def __init__(self, boltParams):
        #from here, you can get params from java
        pass

    def initialize(self, stormconf, context):
        pass

    def process(self, tup):
        try:         
            storm.emit([tup[0]])

             //update your PythonBoltCount metric with param 1
            storm.rpcMetrics("PythonBoltCount", 1)       
        except Exception,tx:
            //handle error

--------ExamplePythonSpout.java---------------------------------------------------------------------------------------------------

    public class ExamplePythonSpout extends ShellSpout implements IRichSpout {
    public ExamplePythonSpout() {
        super("python", "ExampleSpout.py");
    }

    @Override
    public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
        super.open(conf, context, collector);

                    //generate a metric object
        MTCountMetric cMetric = new MTCountMetric();
                    //register metric object to context   
        context.registerMetric("PythonSpoutCount", cMetric, 120);
    }

    public void declareOutputFields(OutputFieldsDeclarer declarer) {
        declarer.declare(new Fields("str"));
    }

    public Map<String, Object> getComponentConfiguration() {
        return null;
    }
}

--------ExampleSpout.py---------------------------------------------------------------------------------------------------

class ExampleSpout(storm.Spout):
    def __init__(self):
        pass

    def initialize(self, stormconf, context):
        pass

    def ack(self, id):
        pass

    def fail(self, id):
        pass

    def nextTuple(self):
        str = "this is a spout message generated by python"
        storm.emit([str])

        //update your PythonBoltCount metric with param 1
        storm.rpcMetrics("PythonSpoutCount", 1)

@@ -60,6 +60,9 @@ public Object getValueAndReset() {
m.put("unusedBytes", memUsage.getCommitted() - memUsage.getUsed());
return m;
}
public void updateMetricFromRPC(List<Object> params) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You are using tabs instead of spaces here. For consistency it would be nice to switch to spaces for all of the indentation.

@revans2
Copy link
Contributor

revans2 commented Feb 25, 2014

The code also seems to have missed ShellSpout.java.

@dashengju
Copy link
Contributor Author

@revans2 , thank you very much for your review, I will redesign the code according your advice

@dashengju
Copy link
Contributor Author

@revans2 , I have redesign the code and the usage according your advice. please review again.

import backtype.storm.metric.api.IMetric;

public interface IShellMetric extends IMetric {
public static final String SHELL_METRICS_UPDATE_METHOD_NAME = "updateMetricFromRPC";
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Very minor whitespace issue.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I have replaced all tab to whitespace, thx

@revans2
Copy link
Contributor

revans2 commented Mar 5, 2014

Looks good. All of the reflection code I would like to see replaced. Also I'd like to hear back from you about my suggestion on the single method call. Like I said I am fine with how you did it, but I would like to hear your opinion on it.

@dashengju
Copy link
Contributor Author

@revans2

I have changed the code as follows:

  1. replace all tab to whitespace;
  2. modify the reflection code to normal method call;
  3. modify registerMetrics in a single method call as your suggestion;

As 3), when I modify the TopologyContext.java,I found a bug in registerMetrics( https://issues.apache.org/jira/browse/STORM-254 ), So I commented getRegisteredMetricByName when implemention.
Can you help me to confirm is it a bug? If yes, I will pull another request to fix them like this:
IMetric tmpMetric = getRegisteredMetricByName(name);
if (tmpMetric != null) {
throw new RuntimeException("The same metric name " + name + " was registered twice." );
}

again, please review.

@@ -26,6 +26,8 @@
import backtype.storm.metric.api.ICombiner;
import backtype.storm.metric.api.ReducedMetric;
import backtype.storm.metric.api.CombinedMetric;
import backtype.storm.metric.api.rpc.IShellMetric;
import backtype.storm.spout.ShellSpout;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

These are not needed any more.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

has removed

@revans2
Copy link
Contributor

revans2 commented Mar 6, 2014

The code looks fine to me once you remove the unneeded imports. But I would like to see a test updated to at least exercise some of these data paths, but preferably some unit tests too. I should have mentioned it before, sorry. You are doing great with the changes keep up the good work!

@dashengju
Copy link
Contributor Author

I have removed the unneeded imports in TopologyContext.java.
And We have put the ShellMetrics in our production environment, test ShellSpout's metrics and ShellBolt's metrics, all works well.
Now I will do some unit tests. But I can not find any JUnit test folder in storm codebase. do we test our java code in test/clj using clojure?

@revans2
Copy link
Contributor

revans2 commented Mar 14, 2014

Great to hear that they are working well in production.

Yes currently all of the tests are written with test/clj, even the ones that test pure java. This is something that Nathan wanted. If you want to write them in junit I have no objection to it, but we probably need to have that discussion again on the mailing lists before you spend too much time on writing them one way.

@dashengju
Copy link
Contributor Author

@revans2 , I have merged your update, all test cases always run success. thank you very much.

And I submit all changed to github, please review pull request again. Thanks for your carefully review.

@revans2
Copy link
Contributor

revans2 commented Jun 20, 2014

+1 Thanks again for your efforts on this.

@dashengju
Copy link
Contributor Author

@revans2 , this pull request have anything else need I do? or just waiting for you merge into master?

if (nameObj != null && nameObj instanceof String) {
metricName = (String) nameObj;
}
shellMsg.setMetricName(metricName);
Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This will not throw a ClassCastException here like with several other keys above (i.e., "command"). We would only see a NullPointerException in ShellSpout or ShellBolt when we check that the String is not empty.

This is really minor.

@d2r
Copy link

d2r commented Jun 27, 2014

I just had a couple of minor comments. I'm +1 even without further changes.

I think we will want to reformat the clojure to match the style guidelines, but this can be done when we merge this in.

@revans2
Copy link
Contributor

revans2 commented Jun 27, 2014

@dashengju Sorry I didn't respond to your comment sooner. It takes 2 +1s from committers to have the code merged in. d2r is also a committer so when he is ready we can merge it in. Thanks for your patience.

@ptgoetz
Copy link
Member

ptgoetz commented Jun 27, 2014

+1 looks okay to me.

@d2r
Copy link

d2r commented Jun 27, 2014

Metrics test hangs for me (tried three times). OSX/1.7.0_60

Still looking...

main [SLEEPING] CPU time: 1d 14h 58m 9s
java.lang.Thread.sleep(long)
backtype.storm.utils.Time.sleepUntil(long)
backtype.storm.utils.Time.sleep(long)
backtype.storm.util$sleep_secs.invoke(Object)
backtype.storm.daemon.supervisor$shutdown_worker.invoke(Object, Object)
backtype.storm.daemon.supervisor$fn__7334$exec_fn__1404__auto__$reify__7343.shutdown_all_workers()
java.lang.reflect.Method.invoke(Object, Object[])
clojure.lang.Reflector.invokeMatchingMethod(String, List, Object, Object[])
clojure.lang.Reflector.invokeNoArgInstanceMember(Object, String)
backtype.storm.testing$kill_local_storm_cluster.invoke(Object)
backtype.storm.metrics_test$fn__2189$fn__2190.invoke()
backtype.storm.metrics_test$fn__2189.invoke()
clojure.test$test_var$fn__7145.invoke()
clojure.test$test_var.invoke(Object)
clojure.test$test_all_vars$fn__7149$fn__7156.invoke()
clojure.test$default_fixture.invoke(Object)
clojure.test$test_all_vars$fn__7149.invoke()
clojure.test$default_fixture.invoke(Object)
clojure.test$test_all_vars.invoke(Object)
clojure.test$test_ns.invoke(Object)
clojure.core$map$fn__4207.invoke()
clojure.lang.LazySeq.sval()
clojure.lang.LazySeq.seq()
clojure.lang.Cons.next()
clojure.lang.RT.boundedLength(ISeq, int)
clojure.lang.RestFn.applyTo(ISeq)
clojure.core$apply.invoke(Object, Object, Object)
clojure.test$run_tests.doInvoke(Object)
clojure.lang.RestFn.invoke(Object)
com.theoryinpractise.clojure.testrunner$eval7799$fn__7804$fn__7847.invoke()
com.theoryinpractise.clojure.testrunner$eval7799$fn__7804.invoke()
com.theoryinpractise.clojure.testrunner$eval7799.invoke()
clojure.lang.Compiler.eval(Object, boolean)
clojure.lang.Compiler.load(Reader, String, String)
clojure.lang.Compiler.loadFile(String)
clojure.main$load_script.invoke(Object)
clojure.main$script_opt.invoke(Object, Object)
clojure.main$main.doInvoke(Object)
clojure.lang.RestFn.invoke(Object)
clojure.lang.Var.invoke(Object)
clojure.lang.AFn.applyToHelper(IFn, ISeq)
clojure.lang.Var.applyTo(ISeq)
clojure.main.main(String[])

@d2r
Copy link

d2r commented Jun 27, 2014

There is an issue with the shutdown of the local cluster when using simulated time. It looks as though we need to keep advancing the cluster time until we fully shut down the cluster, or else it will sleep forever.

I will continue looking into this a when I get more time.

@dashengju
Copy link
Contributor Author

@d2r , should I solve "the issue with the shutdown of the local cluster when using simulated time" at this pull request?

@d2r
Copy link

d2r commented Jun 30, 2014

@dashengju, I looked again at the issue last night and saw the same hang when trying to shut down the workers. When the cluster shuts down after the test, it has the supervisor kill all workers. But when it waits for worker termination there is a hang since simulated cluster time is no longer advancing. Right now I do not think this is happening as a result of your changes. If I commented out the new tests, I was able to run all tests successfully. However, I need to see if this is happening with other environments. Are you able to run all of the tests successfully?

@dashengju
Copy link
Contributor Author

@d2r , yes, I can run all of the tests successfully with "mvn clean install"。I ran the tests 5 times with 4 times successfully and 1 time failure(not hang, 2 error asserts about other test).

And I changed one of timeout:


--- a/storm-core/src/clj/backtype/storm/testing.clj
+++ b/storm-core/src/clj/backtype/storm/testing.clj
@@ -183,7 +183,7 @@
;; on windows, the host process still holds lock on the logfile
(catch Exception e (log-message (.getMessage e)))) ))

-(def TEST-TIMEOUT-MS 5000)
+(def TEST-TIMEOUT-MS 30000)


@dashengju
Copy link
Contributor Author

Another info: I have run just metrics_test.clj 10 times with successfully.

@d2r
Copy link

d2r commented Jun 30, 2014

OK, @revans2 confirms this hangs for him as well when merged into master. It hangs for me in a Redhat environment and on OSX.

I will try a fix today.

@d2r
Copy link

d2r commented Jun 30, 2014

@dashengju

OK, can you try with this patch?

diff --git a/storm-core/src/clj/backtype/storm/testing.clj b/storm-core/src/clj/backtype/storm/testing.clj
index c3eb76f..70c783a 100644
--- a/storm-core/src/clj/backtype/storm/testing.clj
+++ b/storm-core/src/clj/backtype/storm/testing.clj
@@ -234,7 +234,10 @@
          (log-error t# "Error in cluster")
          (throw t#))
        (finally
-         (kill-local-storm-cluster ~cluster-sym)))))
+         (let [keep-waiting?# (atom true)]
+           (future (while @keep-waiting?# (simulate-wait ~cluster-sym)))
+           (kill-local-storm-cluster ~cluster-sym)
+           (reset! keep-waiting?# false))))))

 (defmacro with-simulated-time-local-cluster
   [& args]

The above seems to fix the issue. It could prevent spurious hangs we see with other tests as well.

-(def TEST-TIMEOUT-MS 5000)
+(def TEST-TIMEOUT-MS 30000)

This change is not needed. If it does not progress within 5 seconds, then it probably will not progress after another 25 seconds either. (It is unrelated to the hang.)

@dashengju
Copy link
Contributor Author

@d2r , I have merged your patch.
under ubuntu, run all the tests successfully with 10 times;
But under centos 6, always hang by test-builtin-metrics-2 case. It seems your patch does not work.

========== below is the stdout error info ==============================================================
114011 [main] ERROR backtype.storm.metrics-test - Error in cluster
java.lang.AssertionError: Test timed out (30000ms)
at backtype.storm.metrics_test$wait_for_atleast_N_buckets_BANG_.invoke(metrics_test.clj:74) ~[na:na]
at backtype.storm.metrics_test$fn__1689$fn__1690.invoke(metrics_test.clj:288) ~[na:na]
at backtype.storm.metrics_test$fn__1689.invoke(metrics_test.clj:252) [na:na]
at clojure.test$test_var$fn__7145.invoke(test.clj:701) [clojure-1.5.1.jar:na]
at clojure.test$test_var.invoke(test.clj:701) [clojure-1.5.1.jar:na]
at clojure.test$test_all_vars$fn__7149$fn__7156.invoke(test.clj:717) [clojure-1.5.1.jar:na]
at clojure.test$default_fixture.invoke(test.clj:671) [clojure-1.5.1.jar:na]
at clojure.test$test_all_vars$fn__7149.invoke(test.clj:717) [clojure-1.5.1.jar:na]
at clojure.test$default_fixture.invoke(test.clj:671) [clojure-1.5.1.jar:na]
at clojure.test$test_all_vars.invoke(test.clj:713) [clojure-1.5.1.jar:na]
at clojure.test$test_ns.invoke(test.clj:736) [clojure-1.5.1.jar:na]
at clojure.core$map$fn__4207.invoke(core.clj:2487) [clojure-1.5.1.jar:na]
at clojure.lang.LazySeq.sval(LazySeq.java:42) [clojure-1.5.1.jar:na]
at clojure.lang.LazySeq.seq(LazySeq.java:60) [clojure-1.5.1.jar:na]
at clojure.lang.Cons.next(Cons.java:39) [clojure-1.5.1.jar:na]
at clojure.lang.RT.boundedLength(RT.java:1654) [clojure-1.5.1.jar:na]
at clojure.lang.RestFn.applyTo(RestFn.java:130) [clojure-1.5.1.jar:na]
at clojure.core$apply.invoke(core.clj:619) [clojure-1.5.1.jar:na]
at clojure.test$run_tests.doInvoke(test.clj:751) [clojure-1.5.1.jar:na]
at clojure.lang.RestFn.invoke(RestFn.java:408) [clojure-1.5.1.jar:na]
at com.theoryinpractise.clojure.testrunner$eval8077$fn__8082$fn__8118.invoke(run-test7173011794914230650.clj:102) [na:na]
at com.theoryinpractise.clojure.testrunner$eval8077$fn__8082.invoke(run-test7173011794914230650.clj:99) [na:na]
at com.theoryinpractise.clojure.testrunner$eval8077.invoke(run-test7173011794914230650.clj:72) [na:na]
at clojure.lang.Compiler.eval(Compiler.java:6619) [clojure-1.5.1.jar:na]
at clojure.lang.Compiler.load(Compiler.java:7064) [clojure-1.5.1.jar:na]
at clojure.lang.Compiler.loadFile(Compiler.java:7020) [clojure-1.5.1.jar:na]
at clojure.main$load_script.invoke(main.clj:294) [clojure-1.5.1.jar:na]
at clojure.main$script_opt.invoke(main.clj:356) [clojure-1.5.1.jar:na]
at clojure.main$main.doInvoke(main.clj:440) [clojure-1.5.1.jar:na]
at clojure.lang.RestFn.invoke(RestFn.java:408) [clojure-1.5.1.jar:na]
at clojure.lang.Var.invoke(Var.java:415) [clojure-1.5.1.jar:na]
at clojure.lang.AFn.applyToHelper(AFn.java:161) [clojure-1.5.1.jar:na]
at clojure.lang.Var.applyTo(Var.java:532) [clojure-1.5.1.jar:na]
at clojure.main.main(main.java:37) [clojure-1.5.1.jar:na]
114013 [main] INFO backtype.storm.daemon.nimbus - Shutting down master
114016 [main] INFO backtype.storm.daemon.nimbus - Shut down master
114022 [main] INFO backtype.storm.daemon.supervisor - Shutting down a9fd8203-4ad0-419a-8287-09acedc6d16b:f8b7e9ce-98c2-472f-8deb-c73223b3abd0
114022 [main] INFO backtype.storm.process-simulator - Killing process e3a0b0c3-d589-4861-8e46-0eeb660db114
114023 [main] INFO backtype.storm.daemon.worker - Shutting down worker metrics-tester-1-0 a9fd8203-4ad0-419a-8287-09acedc6d16b 1024
114023 [main] INFO backtype.storm.daemon.worker - Shutting down receive thread
114023 [main] INFO backtype.storm.messaging.loader - Shutting down receiving-thread: [metrics-tester-1-0, 1024]
114024 [main] INFO backtype.storm.messaging.loader - Waiting for receiving-thread:[metrics-tester-1-0, 1024] to die
114024 [Thread-621-worker-receiver-thread-0] INFO backtype.storm.messaging.loader - Receiving-thread:[metrics-tester-1-0, 1024] received shutdown notice
114024 [main] INFO backtype.storm.messaging.loader - Shutdown receiving-thread: [metrics-tester-1-0, 1024]
114024 [main] INFO backtype.storm.daemon.worker - Shut down receive thread
114024 [main] INFO backtype.storm.daemon.worker - Terminating messaging context
114025 [main] INFO backtype.storm.daemon.worker - Shutting down executors
114025 [main] INFO backtype.storm.daemon.executor - Shutting down executor __metricsclojure.storm.metric.testing.FakeMetricConsumer:[2 2]
114026 [Thread-611-disruptor-executor[2 2]-send-queue] INFO backtype.storm.util - Async loop interrupted!

========== below is jstack of main thread ==============================================================
"main" prio=10 tid=0x00007f0ce0006000 nid=0x288e in Object.wait() [0x00007f0ce8789000]
java.lang.Thread.State: WAITING (on object monitor)
at java.lang.Object.wait(Native Method)
- waiting on <0x00000000c4c0d1c0> (a java.lang.Thread)
at java.lang.Thread.join(Thread.java:1258)
- locked <0x00000000c4c0d1c0> (a java.lang.Thread)
at java.lang.Thread.join(Thread.java:1332)
at backtype.storm.util$async_loop$reify__459.join(util.clj:455)
at sun.reflect.GeneratedMethodAccessor53.invoke(Unknown Source)
at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:601)
at clojure.lang.Reflector.invokeMatchingMethod(Reflector.java:93)
at clojure.lang.Reflector.invokeNoArgInstanceMember(Reflector.java:298)
at backtype.storm.daemon.executor$mk_executor$reify__4104.shutdown(executor.clj:353)
at sun.reflect.GeneratedMethodAccessor71.invoke(Unknown Source)
at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:601)
at clojure.lang.Reflector.invokeMatchingMethod(Reflector.java:93)
at clojure.lang.Reflector.invokeNoArgInstanceMember(Reflector.java:298)
at backtype.storm.daemon.worker$fn__4530$exec_fn__1774__auto____4531$shutdown_STAR___4549.invoke(worker.clj:392)
at backtype.storm.daemon.worker$fn__4530$exec_fn__1774__auto__$reify__4575.shutdown(worker.clj:423)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:601)
at clojure.lang.Reflector.invokeMatchingMethod(Reflector.java:93)
at clojure.lang.Reflector.invokeNoArgInstanceMember(Reflector.java:298)
at backtype.storm.process_simulator$kill_process.invoke(process_simulator.clj:46)
- locked <0x00000000c4842af0> (a java.lang.Object)
at backtype.storm.daemon.supervisor$shutdown_worker.invoke(supervisor.clj:173)
at backtype.storm.daemon.supervisor$fn__4967$exec_fn__1774__auto__$reify__4976.shutdown_all_workers(supervisor.clj:411)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:601)
at clojure.lang.Reflector.invokeMatchingMethod(Reflector.java:93)
at clojure.lang.Reflector.invokeNoArgInstanceMember(Reflector.java:298)
at backtype.storm.testing$kill_local_storm_cluster.invoke(testing.clj:172)
at backtype.storm.metrics_test$fn__456$fn__457.invoke(metrics_test.clj:252)
at backtype.storm.metrics_test$fn__456.invoke(metrics_test.clj:252)
at clojure.test$test_var$fn__7145.invoke(test.clj:701)
at clojure.test$test_var.invoke(test.clj:701)
at clojure.test$test_all_vars$fn__7149$fn__7156.invoke(test.clj:717)
at clojure.test$default_fixture.invoke(test.clj:671)
at clojure.test$test_all_vars$fn__7149.invoke(test.clj:717)
at clojure.test$default_fixture.invoke(test.clj:671)
at clojure.test$test_all_vars.invoke(test.clj:713)
at clojure.test$test_ns.invoke(test.clj:736)
at clojure.core$map$fn__4207.invoke(core.clj:2487)
at clojure.lang.LazySeq.sval(LazySeq.java:42)
- locked <0x00000000c4806f10> (a clojure.lang.LazySeq)
at clojure.lang.LazySeq.seq(LazySeq.java:60)
- locked <0x00000000c4806f10> (a clojure.lang.LazySeq)
at clojure.lang.Cons.next(Cons.java:39)
at clojure.lang.RT.boundedLength(RT.java:1654)
at clojure.lang.RestFn.applyTo(RestFn.java:130)
at clojure.core$apply.invoke(core.clj:619)
at clojure.test$run_tests.doInvoke(test.clj:751)
at clojure.lang.RestFn.invoke(RestFn.java:408)
at com.theoryinpractise.clojure.testrunner$eval700$fn__705.invoke(run-test4685934117261617748.clj:48)
at com.theoryinpractise.clojure.testrunner$eval700.invoke(run-test4685934117261617748.clj:38)
at clojure.lang.Compiler.eval(Compiler.java:6619)
at clojure.lang.Compiler.load(Compiler.java:7064)
at clojure.lang.Compiler.loadFile(Compiler.java:7020)
at clojure.main$load_script.invoke(main.clj:294)
at clojure.main$script_opt.invoke(main.clj:356)
at clojure.main$main.doInvoke(main.clj:440)
at clojure.lang.RestFn.invoke(RestFn.java:408)
at clojure.lang.Var.invoke(Var.java:415)
at clojure.lang.AFn.applyToHelper(AFn.java:161)
at clojure.lang.Var.applyTo(Var.java:532)
at clojure.main.main(main.java:37)

@dashengju
Copy link
Contributor Author

new information:
under centos 6, I have found that: the hang was caused by test-builtin-metrics-2 and test-builtin-metrics-3, without those two test cases, others works well.

I am not familiar with with-simulated-time-local-cluster. compare with test-builtin-metrics-1, I found that test-builtin-metrics-2 and test-builtin-metrics-3 have follow code:
tracker (AckFailMapTracker.)
_ (.setAckFailDelegate feeder tracker)
@d2r , I do not know if this is help for you.

@d2r
Copy link

d2r commented Jul 1, 2014

@dashengju, thanks for trying it out.

But under centos 6, always hang by test-builtin-metrics-2 case. It seems your patch does not work.

========== below is the stdout error info ==============================================================
114011 [main] ERROR backtype.storm.metrics-test - Error in cluster
java.lang.AssertionError: Test timed out (30000ms)

The hang that was happening on your test should be fixed with the patch.

This time-out error is a different issue that existed before your changes, and it does not appear to be related to your changes specifically.

I think we should file a separate JIRA for this issue and proceed with this pull request. If you agree, would you merge the patch and update your branch so that this pull request is updated?

@dashengju
Copy link
Contributor Author

@d2r , I agree with you to file a separate JIRA for time-out error.

I have merged your patch and updated this pull request. Thanks for your review and your patch.

Please review again.

@revans2
Copy link
Contributor

revans2 commented Jul 1, 2014

I am +1 with the new changes.

@asfgit asfgit merged commit 573c42a into apache:master Jul 1, 2014
@revans2
Copy link
Contributor

revans2 commented Jul 1, 2014

I merged this into master, but I could only find a name of "DashengJu" on both github and JIRA. if you would like me to update the README with a different name to give you proper credit for your work please post a comment here and I'll be happy to update it.

@dashengju dashengju deleted the master branch October 21, 2014 10:45
knusbaum pushed a commit to knusbaum/incubator-storm that referenced this pull request Feb 11, 2015
Make Config.ThriftPurpose enum static.
dan-blanchard added a commit to pystorm/pystorm that referenced this pull request Oct 20, 2015
Parth-Brahmbhatt pushed a commit to Parth-Brahmbhatt/incubator-storm that referenced this pull request Mar 7, 2016
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

5 participants