You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by se...@apache.org on 2016/06/10 12:51:09 UTC

flink git commit: [hotfix] [core] Make sure Dropwizard reporters do not block operator creation/teardown while reporting

Repository: flink
Updated Branches:
  refs/heads/master 393bceb58 -> 2bffd7b9e


[hotfix] [core] Make sure Dropwizard reporters do not block operator creation/teardown while reporting

The "report()" method used to hold the same lock that was needed to create and shutdown metrics.
Holding that lock too long could delay operator creation and teardown.

This also removes the no longer needed dependency on dropwizard in flink-core.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/2bffd7b9
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/2bffd7b9
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/2bffd7b9

Branch: refs/heads/master
Commit: 2bffd7b9e558b0ee7efb1bc23b1047faed48c4be
Parents: 393bceb
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Jun 10 00:24:57 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Jun 10 14:50:36 2016 +0200

----------------------------------------------------------------------
 flink-core/pom.xml                               |  6 ------
 .../dropwizard/ScheduledDropwizardReporter.java  | 19 +++++++++++--------
 2 files changed, 11 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/2bffd7b9/flink-core/pom.xml
----------------------------------------------------------------------
diff --git a/flink-core/pom.xml b/flink-core/pom.xml
index b64f94e..1aadfbd 100644
--- a/flink-core/pom.xml
+++ b/flink-core/pom.xml
@@ -47,12 +47,6 @@ under the License.
 			<!-- managed version -->
 		</dependency>
 
-		<dependency>
-			<groupId>io.dropwizard.metrics</groupId>
-			<artifactId>metrics-core</artifactId>
-			<version>${metrics.version}</version>
-		</dependency>
-
 		<!-- Avro is needed for the interoperability with Avro types for serialization -->
 		<dependency>
 			<groupId>org.apache.avro</groupId>

http://git-wip-us.apache.org/repos/asf/flink/blob/2bffd7b9/flink-metric-reporters/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/ScheduledDropwizardReporter.java
----------------------------------------------------------------------
diff --git a/flink-metric-reporters/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/ScheduledDropwizardReporter.java b/flink-metric-reporters/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/ScheduledDropwizardReporter.java
index 74fdb85..d67f3e3 100644
--- a/flink-metric-reporters/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/ScheduledDropwizardReporter.java
+++ b/flink-metric-reporters/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/ScheduledDropwizardReporter.java
@@ -35,6 +35,7 @@ import org.apache.flink.metrics.reporter.Scheduled;
 
 import java.util.HashMap;
 import java.util.Map;
+import java.util.SortedMap;
 
 /**
  * Base class for {@link org.apache.flink.metrics.reporter.MetricReporter} that wraps a
@@ -123,14 +124,16 @@ public abstract class ScheduledDropwizardReporter implements MetricReporter, Sch
 
 	@Override
 	public void report() {
-		synchronized (this) {
-			this.reporter.report(
-				this.registry.getGauges(),
-				this.registry.getCounters(),
-				this.registry.getHistograms(),
-				this.registry.getMeters(),
-				this.registry.getTimers());
-		}
+		// we do not need to lock here, because the dropwizard registry is
+		// internally a concurrent map
+		@SuppressWarnings("rawtypes")
+		final SortedMap<String, com.codahale.metrics.Gauge> gauges = registry.getGauges();
+		final SortedMap<String, com.codahale.metrics.Counter> counters = registry.getCounters();
+		final SortedMap<String, com.codahale.metrics.Histogram> histograms = registry.getHistograms();
+		final SortedMap<String, com.codahale.metrics.Meter> meters = registry.getMeters();
+		final SortedMap<String, com.codahale.metrics.Timer> timers = registry.getTimers();
+
+		this.reporter.report(gauges, counters, histograms, meters, timers);
 	}
 
 	public abstract ScheduledReporter getReporter(Configuration config);