You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@samza.apache.org by jg...@apache.org on 2014/03/18 20:11:08 UTC
git commit: SAMZA-191: Per-SSP poll count metric is overly expensive.
Repository: incubator-samza
Updated Branches:
refs/heads/master 2e7b91b20 -> f50312392
SAMZA-191: Per-SSP poll count metric is overly expensive.
Project: http://git-wip-us.apache.org/repos/asf/incubator-samza/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-samza/commit/f5031239
Tree: http://git-wip-us.apache.org/repos/asf/incubator-samza/tree/f5031239
Diff: http://git-wip-us.apache.org/repos/asf/incubator-samza/diff/f5031239
Branch: refs/heads/master
Commit: f503123929da77b944e3c4a1b2394f53ee09adc5
Parents: 2e7b91b
Author: Jakob Homan <jg...@gmail.com>
Authored: Tue Mar 18 12:10:03 2014 -0700
Committer: Jakob Homan <jg...@gmail.com>
Committed: Tue Mar 18 12:10:03 2014 -0700
----------------------------------------------------------------------
.../java/org/apache/samza/util/BlockingEnvelopeMap.java | 9 ---------
1 file changed, 9 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/f5031239/samza-api/src/main/java/org/apache/samza/util/BlockingEnvelopeMap.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/util/BlockingEnvelopeMap.java b/samza-api/src/main/java/org/apache/samza/util/BlockingEnvelopeMap.java
index 6dc1f86..7171088 100644
--- a/samza-api/src/main/java/org/apache/samza/util/BlockingEnvelopeMap.java
+++ b/samza-api/src/main/java/org/apache/samza/util/BlockingEnvelopeMap.java
@@ -108,8 +108,6 @@ public abstract class BlockingEnvelopeMap implements SystemConsumer {
IncomingMessageEnvelope envelope = null;
List<IncomingMessageEnvelope> systemStreamPartitionMessages = new ArrayList<IncomingMessageEnvelope>();
- metrics.incPoll(systemStreamPartition);
-
// First, drain all messages up to numMessages without blocking.
// Stop when we've filled the request (max numMessages), or when
// we get a null envelope back.
@@ -188,7 +186,6 @@ public abstract class BlockingEnvelopeMap implements SystemConsumer {
private final String group;
private final MetricsRegistry metricsRegistry;
private final ConcurrentHashMap<SystemStreamPartition, Gauge<Boolean>> noMoreMessageGaugeMap;
- private final ConcurrentHashMap<SystemStreamPartition, Counter> pollCountMap;
private final ConcurrentHashMap<SystemStreamPartition, Counter> blockingPollCountMap;
private final ConcurrentHashMap<SystemStreamPartition, Counter> blockingPollTimeoutCountMap;
private final Counter pollCount;
@@ -197,7 +194,6 @@ public abstract class BlockingEnvelopeMap implements SystemConsumer {
this.group = group;
this.metricsRegistry = metricsRegistry;
this.noMoreMessageGaugeMap = new ConcurrentHashMap<SystemStreamPartition, Gauge<Boolean>>();
- this.pollCountMap = new ConcurrentHashMap<SystemStreamPartition, Counter>();
this.blockingPollCountMap = new ConcurrentHashMap<SystemStreamPartition, Counter>();
this.blockingPollTimeoutCountMap = new ConcurrentHashMap<SystemStreamPartition, Counter>();
this.pollCount = metricsRegistry.newCounter(group, "poll-count");
@@ -205,7 +201,6 @@ public abstract class BlockingEnvelopeMap implements SystemConsumer {
public void initMetrics(SystemStreamPartition systemStreamPartition) {
this.noMoreMessageGaugeMap.putIfAbsent(systemStreamPartition, metricsRegistry.<Boolean> newGauge(group, "no-more-messages-" + systemStreamPartition, false));
- this.pollCountMap.putIfAbsent(systemStreamPartition, metricsRegistry.newCounter(group, "poll-count-" + systemStreamPartition));
this.blockingPollCountMap.putIfAbsent(systemStreamPartition, metricsRegistry.newCounter(group, "blocking-poll-count-" + systemStreamPartition));
this.blockingPollTimeoutCountMap.putIfAbsent(systemStreamPartition, metricsRegistry.newCounter(group, "blocking-poll-timeout-count-" + systemStreamPartition));
@@ -224,10 +219,6 @@ public abstract class BlockingEnvelopeMap implements SystemConsumer {
this.blockingPollTimeoutCountMap.get(systemStreamPartition).inc();
}
- public void incPoll(SystemStreamPartition systemStreamPartition) {
- this.pollCountMap.get(systemStreamPartition).inc();
- }
-
public void incPoll() {
this.pollCount.inc();
}