New Storm metrics system #390

Merged
merged 14 commits into from Jan 24, 2013

2 participants

@jasonjckn
Collaborator

No description provided.

@jasonjckn jasonjckn closed this Oct 31, 2012
@jasonjckn jasonjckn reopened this Oct 31, 2012
@jasonjckn jasonjckn closed this Oct 31, 2012
@jasonjckn jasonjckn reopened this Oct 31, 2012
@jasonjckn
Collaborator

just did a git rebase

@nathanmarz nathanmarz and 1 other commented on an outdated diff Oct 31, 2012
src/jvm/backtype/storm/task/TopologyContext.java
@@ -190,4 +194,8 @@ public void addTaskHook(ITaskHook hook) {
public Collection<ITaskHook> getHooks() {
return _hooks;
}
+
+ public void registerMetric(String name, IMetric metric, int timeBucketSizeInSecs) {
@nathanmarz
Owner
nathanmarz added a line comment Oct 31, 2012

You should add a convenience method that takes in an IReducer and wraps it in ReducedMetric

@nathanmarz
Owner
nathanmarz added a line comment Oct 31, 2012

Let's also have this method return the IMetric that was passed in, so that you can do things like:

_metric = context.registerMetric("foo", new CountMetric(), 5);

instead of:

_metric = new CountMetric();
context.registerMetric("foo", _metric, 5);

When you pass in a reducer or combiner, it will return a ReducedMetric or CombinedMetric appropriately.

@jasonjckn
Collaborator
jasonjckn added a line comment Oct 31, 2012

NOTE: you can only call registerMetric during prepare or open, should we add a check for this? That's in clojure atom open-or-prepare-was-called a little cumbersome to access from this function.

I could use clojure.RT.var to do it.

@jasonjckn
Collaborator
jasonjckn added a line comment Oct 31, 2012

Do we want to add convenience method for composite metrics? _metric = context.registerMetric("foo", new CompositeMetric(), 5);

The interface would be ICompositeMetric { void doRegistration(TopologyContext context); }

@nathanmarz
Owner
nathanmarz added a line comment Oct 31, 2012

Yea, it would be good to check for this in the TopologyContext. Part of setup-metrics! could be to tell the TopologyContext to stop accepting new registerMetrics invocations.

@jasonjckn
Collaborator
jasonjckn added a line comment Oct 31, 2012

by 'CompositeMetric' I mean for example where you have 2 or more reducers aggregating the same values passed to .update, exporting e.g. myfoo_count and myfoo_avg

@nathanmarz
Owner
nathanmarz added a line comment Oct 31, 2012

Not for now. Let's come back to composite metrics once we see how people are using this.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
@nathanmarz nathanmarz commented on an outdated diff Oct 31, 2012
src/jvm/backtype/storm/metric/IMetricsConsumer.java
+import java.util.Map;
+
+public interface IMetricsConsumer {
+ public static class DataPoint {
+ public String srcWorkerHost;
+ public int srcWorkerPort;
+ public String srcComponentId;
+ public int srcTaskId;
+ public long timestamp;
+ public int updateIntervalSecs;
+ public String name;
+ public Object value;
+ }
+
+ void prepare(Map stormConf, Object registrationOptions, TopologyContext context);
+ void handleDataPoint(DataPoint dataPoint);
@nathanmarz
Owner
nathanmarz added a line comment Oct 31, 2012

I think we should change this interface actually. Rather than handle one data point at a time, it should handle all data points emitted in the same time interval from a task. So the interface would be like:

handleDataPoint(TaskInfo info, Collection dataPoints);

TaskInfo would contain the generic information like hostname, task id, port, timestamp, and updateIntervalSecs. Each data point would just contain the name and the value.

This also requires the metrics tuple to pack together all the data points for the same sized interval into one tuple.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
@nathanmarz nathanmarz commented on an outdated diff Oct 31, 2012
src/jvm/backtype/storm/metric/IncrementedMetric.java
@@ -0,0 +1,22 @@
+package backtype.storm.metric;
+
+public class IncrementedMetric implements IMetric {
@nathanmarz
Owner
nathanmarz added a line comment Oct 31, 2012

Can we rename this to CountMetric?

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
@nathanmarz
Owner

Overall, solid implementation. Left a couple of comments.

I also think we should add an ICombiner interface (a monoid interface with zero() and combine(X, X)

@nathanmarz
Owner

Oh, and also we need some unit tests for this. Time simulation should be useful here.

@jasonjckn
Collaborator

Do we want to split backtype.storm.metric namespace into an API and Impl sections?

@nathanmarz
Owner

Yea, that's a good idea.

Jason Jackson added some commits Oct 31, 2012
@nathanmarz nathanmarz and 1 other commented on an outdated diff Nov 2, 2012
test/clj/backtype/storm/metrics_test.clj
+ (bolt
+ (execute [tuple]
+ (.inc ack-count)
+ (ack! collector tuple)))))
+
+(def datapoints-buffer (atom nil))
+
+(defn metric-name->vals! [name]
+ (->> @datapoints-buffer
+ (mapcat (fn [[task-info data-points]] data-points))
+ (filter #(= name (.name %)))
+ (map #(.value %))
+ (into [])))
+
+(deftest test-time-buckets
+ (with-simulated-time-local-cluster
@nathanmarz
Owner
nathanmarz added a line comment Nov 2, 2012

The problem with this test is the lack of synchronization, so it can fail even when things are working properly. After each advance-cluster-time call, there's no guarantee that the tick tuple will be emitted, processed, and the resulting messages caught by your consumer before you do your assertion.

One way to solve this would be to use a global semaphore and use that to synchronize your assertions with the consumer actually receiving metrics tuples. The backtype.storm.utils.RegisteredGlobalState class can be useful for this.

@jasonjckn
Collaborator
jasonjckn added a line comment Nov 6, 2012

Why is it that 30secs tuple timeouts can be tested with advance-cluster-time without synchronization?

With global semaphore implementation, what happens if metrics tuple is not delivered, unit tests hang? My logic: If you decrement a semaphore when it's at 0, you block, until someone increments it (it's incremented when the tuple is delivered)

Here's another solution: advance-cluster-time blocks until all the time is simulated AND all queues are empty, all tuples were processed and acked, and IBolt::Execute functions returned. In other words, a more generic synchronization.

@nathanmarz
Owner
nathanmarz added a line comment Nov 6, 2012

The timeout tests wait until the spout receives the appropriate number of acks or fails, so it's not actually time-dependent.

Yes, with semaphore implementation, if the test is wrong or there's a bug it should hang, which is fine.

Your second proposal is a lot tricker than it seems. It's doable in theory, and in fact tracked topologies implements a bunch of this. We would have to add inter-task queue checking in order to accomplish this. I think the first approach will be much easier for you than trying to iron out every last detail of the second proposal.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
@nathanmarz
Owner

I commented on what's wrong with the unit test.

@nathanmarz
Owner

OK so the main things left on this issue are:

  1. Getting the standard Storm metrics into this system: execute latency, #executed, #acked, #failed, process latency, etc.
  2. Fixing the unit test
@nathanmarz
Owner

Was just looking at the diff, you should register the metrics classes for serialization in SerializationFactory, not in defaults.yaml. Those configs can be overridden.

@jasonjckn jasonjckn closed this Nov 14, 2012
@jasonjckn jasonjckn reopened this Nov 14, 2012
@jasonjckn jasonjckn closed this Nov 14, 2012
@jasonjckn jasonjckn reopened this Nov 14, 2012
@jasonjckn jasonjckn closed this Nov 14, 2012
@jasonjckn jasonjckn reopened this Nov 14, 2012
@jasonjckn jasonjckn closed this Nov 20, 2012
@jasonjckn
Collaborator

0.9.0 merge undone.

@jasonjckn jasonjckn reopened this Nov 20, 2012
@jasonjckn
Collaborator

all unit tests pass.

@nathanmarz
Owner

Merged into 0.9.0 with a few minor changes

@jasonjckn
Collaborator

can I merge metrics into 0.8.2 branch?

@jasonjckn
Collaborator

and deploy to prod?

@stass stass commented on the diff Dec 17, 2012
src/jvm/backtype/storm/Config.java
@@ -454,6 +461,11 @@
public static String TOPOLOGY_STATS_SAMPLE_RATE="topology.stats.sample.rate";
/**
+ * The time period that builtin metrics data in bucketed into.
@stass
stass added a line comment Dec 17, 2012

"in bucketed into" -- should it be "is bucketed into" instead?

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
@nathanmarz nathanmarz merged commit a9fd4ab into nathanmarz:master Jan 24, 2013
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment