This is an automated email from the ASF dual-hosted git repository. chesnay pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git
commit 874b0f477fdd252ac833203635b6776dcc107305 Author: Chesnay Schepler <[email protected]> AuthorDate: Sat Mar 13 15:06:06 2021 +0100 [FLINK-21773][metrics][statsd] Remove flink-runtime dependency --- flink-metrics/flink-metrics-statsd/pom.xml | 15 ----- .../flink/metrics/statsd/StatsDReporterTest.java | 78 ++++------------------ 2 files changed, 14 insertions(+), 79 deletions(-) diff --git a/flink-metrics/flink-metrics-statsd/pom.xml b/flink-metrics/flink-metrics-statsd/pom.xml index ecb4c8a..8401357 100644 --- a/flink-metrics/flink-metrics-statsd/pom.xml +++ b/flink-metrics/flink-metrics-statsd/pom.xml @@ -66,21 +66,6 @@ under the License. <dependency> <groupId>org.apache.flink</groupId> - <artifactId>flink-runtime_${scala.binary.version}</artifactId> - <version>${project.version}</version> - <scope>test</scope> - </dependency> - - <dependency> - <groupId>org.apache.flink</groupId> - <artifactId>flink-runtime_${scala.binary.version}</artifactId> - <version>${project.version}</version> - <scope>test</scope> - <type>test-jar</type> - </dependency> - - <dependency> - <groupId>org.apache.flink</groupId> <artifactId>flink-test-utils-junit</artifactId> </dependency> </dependencies> diff --git a/flink-metrics/flink-metrics-statsd/src/test/java/org/apache/flink/metrics/statsd/StatsDReporterTest.java b/flink-metrics/flink-metrics-statsd/src/test/java/org/apache/flink/metrics/statsd/StatsDReporterTest.java index 64bb864..d01eac4 100644 --- a/flink-metrics/flink-metrics-statsd/src/test/java/org/apache/flink/metrics/statsd/StatsDReporterTest.java +++ b/flink-metrics/flink-metrics-statsd/src/test/java/org/apache/flink/metrics/statsd/StatsDReporterTest.java @@ -18,10 +18,7 @@ package org.apache.flink.metrics.statsd; -import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.MetricOptions; import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.Metric; @@ -29,18 +26,10 @@ import org.apache.flink.metrics.MetricConfig; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.metrics.SimpleCounter; import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; -import org.apache.flink.metrics.reporter.MetricReporter; import org.apache.flink.metrics.util.TestCounter; import org.apache.flink.metrics.util.TestHistogram; import org.apache.flink.metrics.util.TestMeter; -import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; -import org.apache.flink.runtime.jobgraph.JobVertexID; -import org.apache.flink.runtime.metrics.MetricRegistryImpl; -import org.apache.flink.runtime.metrics.MetricRegistryTestUtils; -import org.apache.flink.runtime.metrics.ReporterSetup; -import org.apache.flink.runtime.metrics.groups.TaskManagerJobMetricGroup; -import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup; -import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; +import org.apache.flink.metrics.util.TestMetricGroup; import org.apache.flink.util.TestLogger; import org.junit.Test; @@ -50,9 +39,7 @@ import java.lang.reflect.InvocationTargetException; import java.net.DatagramPacket; import java.net.DatagramSocket; import java.net.SocketException; -import java.util.Collections; import java.util.HashSet; -import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; @@ -77,70 +64,33 @@ public class StatsDReporterTest extends TestLogger { /** Tests that the registered metrics' names don't contain invalid characters. */ @Test public void testAddingMetrics() throws Exception { - Configuration configuration = new Configuration(); - String taskName = "testTask"; - String jobName = "testJob:-!ax..?"; - String hostname = "local::host:"; - String taskManagerId = "tas:kMana::ger"; String counterName = "testCounter"; - configuration.setString(MetricOptions.SCOPE_NAMING_TASK, "<host>.<tm_id>.<job_name>"); - configuration.setString(MetricOptions.SCOPE_DELIMITER, "_"); - - MetricRegistryImpl metricRegistry = - new MetricRegistryImpl( - MetricRegistryTestUtils.fromConfiguration(configuration), - Collections.singletonList( - ReporterSetup.forReporter("test", new TestingStatsDReporter()))); - - char delimiter = metricRegistry.getDelimiter(); - - TaskManagerMetricGroup tmMetricGroup = - new TaskManagerMetricGroup(metricRegistry, hostname, taskManagerId); - TaskManagerJobMetricGroup tmJobMetricGroup = - new TaskManagerJobMetricGroup(metricRegistry, tmMetricGroup, new JobID(), jobName); - TaskMetricGroup taskMetricGroup = - new TaskMetricGroup( - metricRegistry, - tmJobMetricGroup, - new JobVertexID(), - new ExecutionAttemptID(), - taskName, - 0, - 0); + final String scope = "scope"; + final char delimiter = '_'; - SimpleCounter myCounter = new SimpleCounter(); - - taskMetricGroup.counter(counterName, myCounter); - - List<MetricReporter> reporters = metricRegistry.getReporters(); - - assertTrue(reporters.size() == 1); - - MetricReporter metricReporter = reporters.get(0); + MetricGroup metricGroup = + TestMetricGroup.newBuilder() + .setMetricIdentifierFunction( + (metricName, characterFilter) -> scope + delimiter + metricName) + .build(); - assertTrue( - "Reporter should be of type StatsDReporter", - metricReporter instanceof StatsDReporter); + TestingStatsDReporter reporter = new TestingStatsDReporter(); + reporter.open(new MetricConfig()); - TestingStatsDReporter reporter = (TestingStatsDReporter) metricReporter; + SimpleCounter myCounter = new SimpleCounter(); + reporter.notifyOfAddedMetric(myCounter, counterName, metricGroup); Map<Counter, String> counters = reporter.getCounters(); assertTrue(counters.containsKey(myCounter)); String expectedCounterName = - reporter.filterCharacters(hostname) - + delimiter - + reporter.filterCharacters(taskManagerId) - + delimiter - + reporter.filterCharacters(jobName) + reporter.filterCharacters(scope) + delimiter + reporter.filterCharacters(counterName); assertEquals(expectedCounterName, counters.get(myCounter)); - - metricRegistry.shutdown().get(); } /** Tests that histograms are properly reported via the StatsD reporter. */ @@ -270,7 +220,7 @@ public class StatsDReporterTest extends TestLogger { config.setProperty("port", String.valueOf(port)); reporter = new StatsDReporter(); - ReporterSetup.forReporter("test", config, reporter); + reporter.open(config); MetricGroup metricGroup = new UnregisteredMetricsGroup(); reporter.notifyOfAddedMetric(metric, metricName, metricGroup);
