> On July 14, 2015, 9:44 p.m., Yan Fang wrote: > > samza-elasticsearch/src/main/java/org/apache/samza/system/elasticsearch/ElasticsearchSystemProducerMetrics.java, > > line 24 > > <https://reviews.apache.org/r/36473/diff/1/?file=1010788#file1010788line24> > > > > can this class extends MetricsHelper? This can simplifies a little. > > Roger Hoover wrote: > I don't see how it simplifies things because I have to implement all the > methods in the Scala trait. I'm having trouble getting the newGauge > signatures to match. > > ``` > public class ElasticsearchSystemProducerMetrics implements MetricsHelper { > public final Counter bulkSendSuccess; > public final Counter inserts; > public final Counter updates; > private final MetricsRegistry registry; > private final String group; > private final String systemName; > > public interface JFunction<R> { > R apply(); > } > > public ElasticsearchSystemProducerMetrics(String systemName, > MetricsRegistry registry) { > group = this.getClass().getName(); > this.registry = registry; > this.systemName = systemName; > > bulkSendSuccess = newCounter("bulk-send-success"); > inserts = newCounter("docs-inserted"); > updates = newCounter("docs-updated"); > } > > @Override > public Counter newCounter(String name) { > return MetricsHelper$class.newCounter(this, name); > } > > @Override > public <T> Gauge<T> newGauge(String name, T value) { > return MetricsHelper$class.newGauge(this, name, value); > } > > @Override > public <T> Gauge<T> newGauge(String name, JFunction<T> value) { > return null; > } > > @Override > public Timer newTimer(String name) { > return MetricsHelper$class.newTimer(this, name); > } > > @Override > public String getPrefix() { > return systemName + "-"; > } > > @Override > public MetricsRegistry registry() { > return registry; > } > > @Override > public String group() { > return group; > } > } > ```
We really only need counters for this class but have to figure out how to implement the Scala newGauge methods which are tricky. Would appreciate help if you know how to do it. - Roger ----------------------------------------------------------- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/36473/#review91670 ----------------------------------------------------------- On July 14, 2015, 6:12 a.m., Roger Hoover wrote: > > ----------------------------------------------------------- > This is an automatically generated e-mail. To reply, visit: > https://reviews.apache.org/r/36473/ > ----------------------------------------------------------- > > (Updated July 14, 2015, 6:12 a.m.) > > > Review request for samza. > > > Repository: samza > > > Description > ------- > > SAMZA-733 Add metrics to Elasticsearch System Producer > > > Diffs > ----- > > > samza-elasticsearch/src/main/java/org/apache/samza/system/elasticsearch/ElasticsearchSystemFactory.java > a277b69 > > samza-elasticsearch/src/main/java/org/apache/samza/system/elasticsearch/ElasticsearchSystemProducer.java > 7eb14a2 > > samza-elasticsearch/src/main/java/org/apache/samza/system/elasticsearch/ElasticsearchSystemProducerMetrics.java > PRE-CREATION > > Diff: https://reviews.apache.org/r/36473/diff/ > > > Testing > ------- > > Tested that metrics for Elasticsearch producer appear in JMX and the metrics > stream and that the metrics correctly count how many Elasticsearch documents > were created and indexed. > > > Thanks, > > Roger Hoover > >