From d45b941280155a67aed3f3518b2a05eedc1dab2e Mon Sep 17 00:00:00 2001 From: Till Date: Thu, 19 Oct 2017 17:22:53 +0200 Subject: [PATCH] [FLINK-7876] Register TaskManagerMetricGroup under ResourceID This commit changes that TaskManagerMetricGroups are now registered under the TaskManager's ResourceID instead of the InstanceID. This allows to create the TaskManagerMetricGroup at startup of the TaskManager. Moreover, it pulls the MetricRegistry out of JobManager and TaskManager. This allows to reuse the same MetricRegistry across multiple instances (e.g. in the FlinkMiniCluster case). Moreover, it ensures proper cleanup of a potentially started MetricyQueryServiceActor. Change TaskManagersHandler to work with ResourceID instead of InstanceID Adapt MetricFetcher to use ResourceID instead of InstanceID This closes #4872. --- .../entrypoint/MesosJobClusterEntrypoint.java | 6 +- .../MesosSessionClusterEntrypoint.java | 6 +- .../MesosApplicationMasterRunner.java | 13 + .../MesosResourceManager.java | 4 +- .../clusterframework/MesosJobManager.scala | 7 +- .../clusterframework/MesosTaskManager.scala | 6 +- .../MesosResourceManagerTest.java | 8 +- .../ScheduledDropwizardReporterTest.java | 4 +- .../DropwizardFlinkHistogramWrapperTest.java | 6 +- .../flink/metrics/jmx/JMXReporterTest.java | 14 +- .../PrometheusReporterTaskScopeTest.java | 6 +- .../prometheus/PrometheusReporterTest.java | 16 +- .../metrics/slf4j/Slf4jReporterTest.java | 6 +- .../metrics/statsd/StatsDReporterTest.java | 12 +- .../webmonitor/WebRuntimeMonitorITCase.java | 2 + .../runtime/akka/AkkaJobManagerGateway.java | 10 +- .../flink/runtime/dispatcher/Dispatcher.java | 11 +- .../dispatcher/StandaloneDispatcher.java | 6 +- .../runtime/entrypoint/ClusterEntrypoint.java | 10 +- .../entrypoint/JobClusterEntrypoint.java | 8 +- .../entrypoint/SessionClusterEntrypoint.java | 8 +- .../StandaloneSessionClusterEntrypoint.java | 4 +- .../runtime/jobmaster/JobManagerGateway.java | 6 +- .../runtime/jobmaster/JobManagerRunner.java | 4 +- .../flink/runtime/metrics/MetricRegistry.java | 378 +--------------- .../metrics/MetricRegistryConfiguration.java | 2 +- .../runtime/metrics/MetricRegistryImpl.java | 412 ++++++++++++++++++ .../metrics/groups/AbstractMetricGroup.java | 2 +- .../runtime/minicluster/MiniCluster.java | 12 +- .../minicluster/MiniClusterJobDispatcher.java | 8 +- .../minicluster/StandaloneMiniCluster.java | 29 +- .../resourcemanager/ResourceManager.java | 12 +- .../ResourceManagerGateway.java | 2 +- .../ResourceManagerRunner.java | 4 +- .../StandaloneResourceManager.java | 4 +- .../handler/legacy/TaskManagerLogHandler.java | 8 +- .../handler/legacy/TaskManagersHandler.java | 21 +- .../handler/legacy/metrics/MetricFetcher.java | 10 +- .../runtime/taskexecutor/TaskExecutor.java | 6 +- .../taskexecutor/TaskManagerRunner.java | 24 +- .../taskexecutor/TaskManagerServices.java | 18 +- .../TaskManagerServicesConfiguration.java | 13 - .../runtime/webmonitor/RestfulGateway.java | 4 +- .../ContaineredJobManager.scala | 7 +- .../flink/runtime/jobmanager/JobManager.scala | 113 ++--- .../runtime/messages/JobManagerMessages.scala | 7 +- .../minicluster/FlinkMiniCluster.scala | 8 + .../minicluster/LocalFlinkMiniCluster.scala | 52 ++- .../runtime/taskmanager/TaskManager.scala | 48 +- .../clusterframework/ResourceManagerTest.java | 6 +- .../runtime/dispatcher/DispatcherTest.java | 10 +- .../jobmanager/JobManagerHARecoveryTest.java | 13 +- .../runtime/jobmanager/JobManagerTest.java | 11 + .../runtime/jobmanager/JobSubmitTest.java | 2 + .../jobmaster/JobManagerRunnerMockTest.java | 4 +- .../JobManagerLeaderElectionTest.java | 5 +- ...yTest.java => MetricRegistryImplTest.java} | 26 +- .../runtime/metrics/NoOpMetricRegistry.java | 60 +++ .../metrics/TaskManagerMetricsTest.java | 16 +- .../metrics/dump/MetricQueryServiceTest.java | 4 +- .../groups/AbstractMetricGroupTest.java | 8 +- .../metrics/groups/JobManagerGroupTest.java | 12 +- .../groups/JobManagerJobGroupTest.java | 10 +- .../groups/MetricGroupRegistrationTest.java | 9 +- .../metrics/groups/MetricGroupTest.java | 11 +- .../metrics/groups/OperatorGroupTest.java | 12 +- .../metrics/groups/TaskManagerGroupTest.java | 12 +- .../groups/TaskManagerJobGroupTest.java | 10 +- .../metrics/groups/TaskMetricGroupTest.java | 14 +- .../UnregisteredTaskMetricsGroup.java | 4 +- .../ResourceManagerHATest.java | 4 +- .../ResourceManagerJobMasterTest.java | 4 +- .../ResourceManagerTaskExecutorTest.java | 4 +- .../legacy/TaskManagerLogHandlerTest.java | 2 +- .../legacy/metrics/MetricFetcherTest.java | 10 +- .../taskexecutor/TaskExecutorITCase.java | 4 +- .../taskexecutor/TaskExecutorTest.java | 24 +- .../taskexecutor/TaskManagerServicesTest.java | 3 +- ...kManagerComponentsStartupShutdownTest.java | 7 +- .../TaskManagerProcessReapingTestBase.java | 20 +- .../taskmanager/TaskManagerStartupTest.java | 2 + .../JobManagerRegistrationTest.scala | 17 +- .../runtime/testingUtils/TestingCluster.scala | 7 +- .../testingUtils/TestingJobManager.scala | 6 +- .../testingUtils/TestingTaskManager.scala | 10 +- .../runtime/testingUtils/TestingUtils.scala | 21 +- ...TaskManagerProcessFailureRecoveryTest.java | 2 + .../JobManagerHACheckpointRecoveryITCase.java | 2 + .../JobManagerHAJobGraphRecoveryITCase.java | 2 + ...erHAProcessFailureBatchRecoveryITCase.java | 2 + .../ProcessFailureCancelingITCase.java | 2 + .../AbstractOperatorRestoreTestBase.java | 3 + .../flink/yarn/TestingYarnJobManager.scala | 6 +- .../flink/yarn/TestingYarnTaskManager.scala | 6 +- .../yarn/YarnApplicationMasterRunner.java | 13 + .../flink/yarn/YarnResourceManager.java | 4 +- .../entrypoint/YarnJobClusterEntrypoint.java | 4 +- .../YarnSessionClusterEntrypoint.java | 4 +- .../apache/flink/yarn/YarnJobManager.scala | 7 +- .../apache/flink/yarn/YarnTaskManager.scala | 6 +- 100 files changed, 1020 insertions(+), 834 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistryImpl.java rename flink-runtime/src/test/java/org/apache/flink/runtime/metrics/{MetricRegistryTest.java => MetricRegistryImplTest.java} (93%) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/metrics/NoOpMetricRegistry.java diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java index 9a3639dde1577..b98adff4d0cff 100755 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java @@ -34,7 +34,7 @@ import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.resourcemanager.ResourceManager; import org.apache.flink.runtime.resourcemanager.ResourceManagerConfiguration; import org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServices; @@ -112,7 +112,7 @@ protected void initializeServices(Configuration config) throws Exception { } @Override - protected void startClusterComponents(Configuration configuration, RpcService rpcService, HighAvailabilityServices highAvailabilityServices, BlobServer blobServer, HeartbeatServices heartbeatServices, MetricRegistry metricRegistry) throws Exception { + protected void startClusterComponents(Configuration configuration, RpcService rpcService, HighAvailabilityServices highAvailabilityServices, BlobServer blobServer, HeartbeatServices heartbeatServices, MetricRegistryImpl metricRegistry) throws Exception { super.startClusterComponents(configuration, rpcService, highAvailabilityServices, blobServer, heartbeatServices, metricRegistry); } @@ -123,7 +123,7 @@ protected ResourceManager createResourceManager( RpcService rpcService, HighAvailabilityServices highAvailabilityServices, HeartbeatServices heartbeatServices, - MetricRegistry metricRegistry, + MetricRegistryImpl metricRegistry, FatalErrorHandler fatalErrorHandler) throws Exception { final ResourceManagerConfiguration rmConfiguration = ResourceManagerConfiguration.fromConfiguration(configuration); final ResourceManagerRuntimeServicesConfiguration rmServicesConfiguration = ResourceManagerRuntimeServicesConfiguration.fromConfiguration(configuration); diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java index aa511b0de4e98..0cf0fceeb716b 100755 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java @@ -33,7 +33,7 @@ import org.apache.flink.runtime.entrypoint.SessionClusterEntrypoint; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.resourcemanager.ResourceManager; import org.apache.flink.runtime.resourcemanager.ResourceManagerConfiguration; import org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServices; @@ -102,7 +102,7 @@ protected void initializeServices(Configuration config) throws Exception { } @Override - protected void startClusterComponents(Configuration configuration, RpcService rpcService, HighAvailabilityServices highAvailabilityServices, BlobServer blobServer, HeartbeatServices heartbeatServices, MetricRegistry metricRegistry) throws Exception { + protected void startClusterComponents(Configuration configuration, RpcService rpcService, HighAvailabilityServices highAvailabilityServices, BlobServer blobServer, HeartbeatServices heartbeatServices, MetricRegistryImpl metricRegistry) throws Exception { super.startClusterComponents(configuration, rpcService, highAvailabilityServices, blobServer, heartbeatServices, metricRegistry); } @@ -113,7 +113,7 @@ protected ResourceManager createResourceManager( RpcService rpcService, HighAvailabilityServices highAvailabilityServices, HeartbeatServices heartbeatServices, - MetricRegistry metricRegistry, + MetricRegistryImpl metricRegistry, FatalErrorHandler fatalErrorHandler) throws Exception { final ResourceManagerConfiguration rmConfiguration = ResourceManagerConfiguration.fromConfiguration(configuration); final ResourceManagerRuntimeServicesConfiguration rmServicesConfiguration = ResourceManagerRuntimeServicesConfiguration.fromConfiguration(configuration); diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java index a6ea1335b3536..9887d97f7764d 100755 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java @@ -39,6 +39,8 @@ import org.apache.flink.runtime.jobmanager.JobManager; import org.apache.flink.runtime.jobmanager.MemoryArchivist; import org.apache.flink.runtime.jobmaster.JobMaster; +import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.process.ProcessReaper; import org.apache.flink.runtime.security.SecurityConfiguration; import org.apache.flink.runtime.security.SecurityUtils; @@ -200,6 +202,7 @@ protected int runPrivileged(Configuration config, Configuration dynamicPropertie ExecutorService ioExecutor = null; MesosServices mesosServices = null; HighAvailabilityServices highAvailabilityServices = null; + MetricRegistryImpl metricRegistry = null; try { // ------- (1) load and parse / validate all configurations ------- @@ -304,6 +307,11 @@ protected int runPrivileged(Configuration config, Configuration dynamicPropertie // 2: the JobManager LOG.debug("Starting JobManager actor"); + metricRegistry = new MetricRegistryImpl( + MetricRegistryConfiguration.fromConfiguration(config)); + + metricRegistry.startQueryService(actorSystem, null); + // we start the JobManager with its standard name ActorRef jobManager = JobManager.startJobManagerActors( config, @@ -311,6 +319,7 @@ protected int runPrivileged(Configuration config, Configuration dynamicPropertie futureExecutor, ioExecutor, highAvailabilityServices, + metricRegistry, webMonitor != null ? Option.apply(webMonitor.getRestAddress()) : Option.empty(), Option.apply(JobMaster.JOB_MANAGER_NAME), Option.apply(JobMaster.ARCHIVE_NAME), @@ -422,6 +431,10 @@ protected int runPrivileged(Configuration config, Configuration dynamicPropertie } } + if (metricRegistry != null) { + metricRegistry.shutdown(); + } + org.apache.flink.runtime.concurrent.Executors.gracefulShutdown( AkkaUtils.getTimeout(config).toMillis(), TimeUnit.MILLISECONDS, diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosResourceManager.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosResourceManager.java index 1e32b2c973f03..7ea49083f22cb 100644 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosResourceManager.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosResourceManager.java @@ -49,7 +49,7 @@ import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.resourcemanager.JobLeaderIdService; import org.apache.flink.runtime.resourcemanager.ResourceManager; import org.apache.flink.runtime.resourcemanager.ResourceManagerConfiguration; @@ -145,7 +145,7 @@ public MesosResourceManager( HighAvailabilityServices highAvailabilityServices, HeartbeatServices heartbeatServices, SlotManager slotManager, - MetricRegistry metricRegistry, + MetricRegistryImpl metricRegistry, JobLeaderIdService jobLeaderIdService, FatalErrorHandler fatalErrorHandler, // Mesos specifics diff --git a/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosJobManager.scala b/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosJobManager.scala index 9ad8eb21e1fa6..c6230e70d6f8a 100644 --- a/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosJobManager.scala +++ b/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosJobManager.scala @@ -31,7 +31,8 @@ import org.apache.flink.runtime.instance.InstanceManager import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore import org.apache.flink.runtime.jobmanager.scheduler.{Scheduler => FlinkScheduler} import org.apache.flink.runtime.leaderelection.LeaderElectionService -import org.apache.flink.runtime.metrics.{MetricRegistry => FlinkMetricRegistry} +import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup +import org.apache.flink.runtime.metrics.{MetricRegistryImpl => FlinkMetricRegistry} import scala.concurrent.duration._ @@ -66,7 +67,7 @@ class MesosJobManager( submittedJobGraphs : SubmittedJobGraphStore, checkpointRecoveryFactory : CheckpointRecoveryFactory, jobRecoveryTimeout: FiniteDuration, - metricsRegistry: Option[FlinkMetricRegistry], + jobManagerMetricGroup: JobManagerMetricGroup, optRestAddress: Option[String]) extends ContaineredJobManager( flinkConfiguration, @@ -83,7 +84,7 @@ class MesosJobManager( submittedJobGraphs, checkpointRecoveryFactory, jobRecoveryTimeout, - metricsRegistry, + jobManagerMetricGroup, optRestAddress) { val jobPollingInterval: FiniteDuration = 5 seconds diff --git a/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala b/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala index 78346394ec03a..e69472e7c0d1e 100644 --- a/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala +++ b/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala @@ -23,7 +23,7 @@ import org.apache.flink.runtime.highavailability.HighAvailabilityServices import org.apache.flink.runtime.io.disk.iomanager.IOManager import org.apache.flink.runtime.io.network.NetworkEnvironment import org.apache.flink.runtime.memory.MemoryManager -import org.apache.flink.runtime.metrics.MetricRegistry +import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup import org.apache.flink.runtime.taskexecutor.TaskManagerConfiguration import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerLocation} @@ -39,7 +39,7 @@ class MesosTaskManager( network: NetworkEnvironment, numberOfSlots: Int, highAvailabilityServices: HighAvailabilityServices, - metricRegistry : MetricRegistry) + taskManagerMetricGroup : TaskManagerMetricGroup) extends TaskManager( config, resourceID, @@ -49,7 +49,7 @@ class MesosTaskManager( network, numberOfSlots, highAvailabilityServices, - metricRegistry) { + taskManagerMetricGroup) { override def handleMessage: Receive = { super.handleMessage diff --git a/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosResourceManagerTest.java b/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosResourceManagerTest.java index 4bdd9a3f248ca..1cdd0879f0db8 100644 --- a/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosResourceManagerTest.java +++ b/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosResourceManagerTest.java @@ -52,7 +52,7 @@ import org.apache.flink.runtime.jobmaster.JobMasterRegistrationSuccess; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.registration.RegistrationResponse; import org.apache.flink.runtime.resourcemanager.JobLeaderIdService; import org.apache.flink.runtime.resourcemanager.ResourceManagerConfiguration; @@ -160,7 +160,7 @@ public TestingMesosResourceManager( HighAvailabilityServices highAvailabilityServices, HeartbeatServices heartbeatServices, SlotManager slotManager, - MetricRegistry metricRegistry, + MetricRegistryImpl metricRegistry, JobLeaderIdService jobLeaderIdService, FatalErrorHandler fatalErrorHandler, @@ -306,7 +306,7 @@ class MockResourceManagerRuntimeServices { public final ScheduledExecutor scheduledExecutor; public final TestingHighAvailabilityServices highAvailabilityServices; public final HeartbeatServices heartbeatServices; - public final MetricRegistry metricRegistry; + public final MetricRegistryImpl metricRegistry; public final TestingLeaderElectionService rmLeaderElectionService; public final JobLeaderIdService jobLeaderIdService; public final SlotManager slotManager; @@ -321,7 +321,7 @@ class MockResourceManagerRuntimeServices { rmLeaderElectionService = new TestingLeaderElectionService(); highAvailabilityServices.setResourceManagerLeaderElectionService(rmLeaderElectionService); heartbeatServices = new TestingHeartbeatServices(5L, 5L, scheduledExecutor); - metricRegistry = mock(MetricRegistry.class); + metricRegistry = mock(MetricRegistryImpl.class); slotManager = mock(SlotManager.class); slotManagerStarted = new CompletableFuture<>(); jobLeaderIdService = new JobLeaderIdService( diff --git a/flink-metrics/flink-metrics-dropwizard/src/test/java/org/apache/flink/dropwizard/ScheduledDropwizardReporterTest.java b/flink-metrics/flink-metrics-dropwizard/src/test/java/org/apache/flink/dropwizard/ScheduledDropwizardReporterTest.java index 5ed6de27db5e8..e6d5e27db0bfd 100644 --- a/flink-metrics/flink-metrics-dropwizard/src/test/java/org/apache/flink/dropwizard/ScheduledDropwizardReporterTest.java +++ b/flink-metrics/flink-metrics-dropwizard/src/test/java/org/apache/flink/dropwizard/ScheduledDropwizardReporterTest.java @@ -34,7 +34,7 @@ import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; import org.apache.flink.metrics.reporter.MetricReporter; import org.apache.flink.runtime.jobgraph.JobVertexID; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; import org.apache.flink.runtime.metrics.groups.TaskManagerJobMetricGroup; import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup; @@ -92,7 +92,7 @@ public void testAddingMetrics() throws NoSuchFieldException, IllegalAccessExcept MetricRegistryConfiguration metricRegistryConfiguration = MetricRegistryConfiguration.fromConfiguration(configuration); - MetricRegistry metricRegistry = new MetricRegistry(metricRegistryConfiguration); + MetricRegistryImpl metricRegistry = new MetricRegistryImpl(metricRegistryConfiguration); char delimiter = metricRegistry.getDelimiter(); diff --git a/flink-metrics/flink-metrics-dropwizard/src/test/java/org/apache/flink/dropwizard/metrics/DropwizardFlinkHistogramWrapperTest.java b/flink-metrics/flink-metrics-dropwizard/src/test/java/org/apache/flink/dropwizard/metrics/DropwizardFlinkHistogramWrapperTest.java index 63765ae2a18e9..a927a30eb6db8 100644 --- a/flink-metrics/flink-metrics-dropwizard/src/test/java/org/apache/flink/dropwizard/metrics/DropwizardFlinkHistogramWrapperTest.java +++ b/flink-metrics/flink-metrics-dropwizard/src/test/java/org/apache/flink/dropwizard/metrics/DropwizardFlinkHistogramWrapperTest.java @@ -24,7 +24,7 @@ import org.apache.flink.dropwizard.ScheduledDropwizardReporter; import org.apache.flink.metrics.MetricConfig; import org.apache.flink.metrics.reporter.MetricReporter; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup; import org.apache.flink.util.TestLogger; @@ -105,12 +105,12 @@ public void testDropwizardHistogramWrapperReporting() throws Exception { config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "my_reporter." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestingReporter.class.getName()); config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "my_reporter." + ConfigConstants.METRICS_REPORTER_INTERVAL_SUFFIX, reportingInterval + " MILLISECONDS"); - MetricRegistry registry = null; + MetricRegistryImpl registry = null; MetricRegistryConfiguration metricRegistryConfiguration = MetricRegistryConfiguration.fromConfiguration(config); try { - registry = new MetricRegistry(metricRegistryConfiguration); + registry = new MetricRegistryImpl(metricRegistryConfiguration); DropwizardHistogramWrapper histogramWrapper = new DropwizardHistogramWrapper( new com.codahale.metrics.Histogram(new SlidingWindowReservoir(size))); diff --git a/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/metrics/jmx/JMXReporterTest.java b/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/metrics/jmx/JMXReporterTest.java index f10769a6de1de..4c97055b4cec4 100644 --- a/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/metrics/jmx/JMXReporterTest.java +++ b/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/metrics/jmx/JMXReporterTest.java @@ -24,7 +24,7 @@ import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.reporter.MetricReporter; import org.apache.flink.metrics.util.TestMeter; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; import org.apache.flink.runtime.metrics.groups.FrontMetricGroup; import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup; @@ -108,7 +108,7 @@ public void testPortConflictHandling() throws Exception { cfg.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test2." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, JMXReporter.class.getName()); cfg.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test2.port", "9020-9035"); - MetricRegistry reg = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(cfg)); + MetricRegistryImpl reg = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(cfg)); TaskManagerMetricGroup mg = new TaskManagerMetricGroup(reg, "host", "tm"); @@ -168,7 +168,7 @@ public void testJMXAvailability() throws Exception { cfg.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test2." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, JMXReporter.class.getName()); cfg.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test2.port", "9040-9055"); - MetricRegistry reg = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(cfg)); + MetricRegistryImpl reg = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(cfg)); TaskManagerMetricGroup mg = new TaskManagerMetricGroup(reg, "host", "tm"); @@ -231,7 +231,7 @@ public Integer getValue() { */ @Test public void testHistogramReporting() throws Exception { - MetricRegistry registry = null; + MetricRegistryImpl registry = null; String histogramName = "histogram"; try { @@ -239,7 +239,7 @@ public void testHistogramReporting() throws Exception { config.setString(MetricOptions.REPORTERS_LIST, "jmx_test"); config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "jmx_test." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, JMXReporter.class.getName()); - registry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(config)); + registry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(config)); TaskManagerMetricGroup metricGroup = new TaskManagerMetricGroup(registry, "localhost", "tmId"); @@ -281,7 +281,7 @@ public void testHistogramReporting() throws Exception { */ @Test public void testMeterReporting() throws Exception { - MetricRegistry registry = null; + MetricRegistryImpl registry = null; String meterName = "meter"; try { @@ -289,7 +289,7 @@ public void testMeterReporting() throws Exception { config.setString(MetricOptions.REPORTERS_LIST, "jmx_test"); config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "jmx_test." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, JMXReporter.class.getName()); - registry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(config)); + registry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(config)); TaskManagerMetricGroup metricGroup = new TaskManagerMetricGroup(registry, "localhost", "tmId"); diff --git a/flink-metrics/flink-metrics-prometheus/src/test/java/org/apache/flink/metrics/prometheus/PrometheusReporterTaskScopeTest.java b/flink-metrics/flink-metrics-prometheus/src/test/java/org/apache/flink/metrics/prometheus/PrometheusReporterTaskScopeTest.java index 0ae8fc74cfd61..55ddc004ccd4c 100644 --- a/flink-metrics/flink-metrics-prometheus/src/test/java/org/apache/flink/metrics/prometheus/PrometheusReporterTaskScopeTest.java +++ b/flink-metrics/flink-metrics-prometheus/src/test/java/org/apache/flink/metrics/prometheus/PrometheusReporterTaskScopeTest.java @@ -26,8 +26,8 @@ import org.apache.flink.metrics.SimpleCounter; import org.apache.flink.metrics.util.TestMeter; import org.apache.flink.runtime.jobgraph.JobVertexID; -import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.metrics.groups.TaskManagerJobMetricGroup; import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup; import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; @@ -75,12 +75,12 @@ public class PrometheusReporterTaskScopeTest { private TaskMetricGroup taskMetricGroup1; private TaskMetricGroup taskMetricGroup2; - private MetricRegistry registry; + private MetricRegistryImpl registry; private PrometheusReporter reporter; @Before public void setupReporter() { - registry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(createConfigWithOneReporter("test1", "9400-9500"))); + registry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(createConfigWithOneReporter("test1", "9400-9500"))); reporter = (PrometheusReporter) registry.getReporters().get(0); TaskManagerMetricGroup tmMetricGroup = new TaskManagerMetricGroup(registry, TASK_MANAGER_HOST, TASK_MANAGER_ID); diff --git a/flink-metrics/flink-metrics-prometheus/src/test/java/org/apache/flink/metrics/prometheus/PrometheusReporterTest.java b/flink-metrics/flink-metrics-prometheus/src/test/java/org/apache/flink/metrics/prometheus/PrometheusReporterTest.java index 0d7be6dea3af1..6704189da2713 100644 --- a/flink-metrics/flink-metrics-prometheus/src/test/java/org/apache/flink/metrics/prometheus/PrometheusReporterTest.java +++ b/flink-metrics/flink-metrics-prometheus/src/test/java/org/apache/flink/metrics/prometheus/PrometheusReporterTest.java @@ -28,8 +28,8 @@ import org.apache.flink.metrics.Metric; import org.apache.flink.metrics.SimpleCounter; import org.apache.flink.metrics.util.TestMeter; -import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.metrics.groups.FrontMetricGroup; import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup; import org.apache.flink.runtime.metrics.util.TestingHistogram; @@ -69,13 +69,13 @@ public class PrometheusReporterTest extends TestLogger { @Rule public ExpectedException thrown = ExpectedException.none(); - private MetricRegistry registry; + private MetricRegistryImpl registry; private FrontMetricGroup metricGroup; private PrometheusReporter reporter; @Before public void setupReporter() { - registry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(createConfigWithOneReporter("test1", "9400-9500"))); + registry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(createConfigWithOneReporter("test1", "9400-9500"))); metricGroup = new FrontMetricGroup<>(0, new TaskManagerMetricGroup(registry, HOST_NAME, TASK_MANAGER)); reporter = (PrometheusReporter) registry.getReporters().get(0); } @@ -158,7 +158,7 @@ public void histogramIsReportedAsPrometheusSummary() throws UnirestException { @Test public void endpointIsUnavailableAfterReporterIsClosed() throws UnirestException { - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(createConfigWithOneReporter("test1", "9400-9500"))); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(createConfigWithOneReporter("test1", "9400-9500"))); PrometheusReporter reporter = (PrometheusReporter) registry.getReporters().get(0); reporter.close(); thrown.expect(UnirestException.class); @@ -244,12 +244,12 @@ class SomeMetricType implements Metric{} @Test public void cannotStartTwoReportersOnSamePort() { - final MetricRegistry fixedPort1 = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(createConfigWithOneReporter("test1", "9400-9500"))); + final MetricRegistryImpl fixedPort1 = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(createConfigWithOneReporter("test1", "9400-9500"))); assertThat(fixedPort1.getReporters(), hasSize(1)); PrometheusReporter firstReporter = (PrometheusReporter) fixedPort1.getReporters().get(0); - final MetricRegistry fixedPort2 = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(createConfigWithOneReporter("test2", String.valueOf(firstReporter.getPort())))); + final MetricRegistryImpl fixedPort2 = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(createConfigWithOneReporter("test2", String.valueOf(firstReporter.getPort())))); assertThat(fixedPort2.getReporters(), hasSize(0)); fixedPort1.shutdown(); @@ -258,8 +258,8 @@ public void cannotStartTwoReportersOnSamePort() { @Test public void canStartTwoReportersWhenUsingPortRange() { - final MetricRegistry portRange1 = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(createConfigWithOneReporter("test1", "9200-9300"))); - final MetricRegistry portRange2 = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(createConfigWithOneReporter("test2", "9200-9300"))); + final MetricRegistryImpl portRange1 = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(createConfigWithOneReporter("test1", "9200-9300"))); + final MetricRegistryImpl portRange2 = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(createConfigWithOneReporter("test2", "9200-9300"))); assertThat(portRange1.getReporters(), hasSize(1)); assertThat(portRange2.getReporters(), hasSize(1)); diff --git a/flink-metrics/flink-metrics-slf4j/src/test/java/org/apache/flink/metrics/slf4j/Slf4jReporterTest.java b/flink-metrics/flink-metrics-slf4j/src/test/java/org/apache/flink/metrics/slf4j/Slf4jReporterTest.java index 51724bd750872..ba7c5a19af145 100644 --- a/flink-metrics/flink-metrics-slf4j/src/test/java/org/apache/flink/metrics/slf4j/Slf4jReporterTest.java +++ b/flink-metrics/flink-metrics-slf4j/src/test/java/org/apache/flink/metrics/slf4j/Slf4jReporterTest.java @@ -29,8 +29,8 @@ import org.apache.flink.metrics.SimpleCounter; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.jobgraph.JobVertexID; -import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup; import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.metrics.util.TestingHistogram; @@ -53,7 +53,7 @@ public class Slf4jReporterTest extends TestLogger { private static final String TASK_MANAGER_ID = "tm01"; private static final String JOB_NAME = "jn01"; private static final String TASK_NAME = "tn01"; - private static MetricRegistry registry; + private static MetricRegistryImpl registry; private static char delimiter; private static TaskMetricGroup taskMetricGroup; private static Slf4jReporter reporter; @@ -68,7 +68,7 @@ public static void setUp() { ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, Slf4jReporter.class.getName()); configuration.setString(MetricOptions.SCOPE_NAMING_TASK, ".."); - registry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(configuration)); + registry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(configuration)); delimiter = registry.getDelimiter(); taskMetricGroup = new TaskManagerMetricGroup(registry, HOST_NAME, TASK_MANAGER_ID) 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 94de9a9db80a8..f460abd466776 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 @@ -30,7 +30,7 @@ import org.apache.flink.metrics.reporter.MetricReporter; import org.apache.flink.metrics.util.TestMeter; import org.apache.flink.runtime.jobgraph.JobVertexID; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; import org.apache.flink.runtime.metrics.groups.TaskManagerJobMetricGroup; import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup; @@ -89,7 +89,7 @@ public void testAddingMetrics() throws NoSuchFieldException, IllegalAccessExcept configuration.setString(MetricOptions.SCOPE_NAMING_TASK, ".."); configuration.setString(MetricOptions.SCOPE_DELIMITER, "_"); - MetricRegistry metricRegistry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(configuration)); + MetricRegistryImpl metricRegistry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(configuration)); char delimiter = metricRegistry.getDelimiter(); @@ -133,7 +133,7 @@ public void testAddingMetrics() throws NoSuchFieldException, IllegalAccessExcept */ @Test public void testStatsDHistogramReporting() throws Exception { - MetricRegistry registry = null; + MetricRegistryImpl registry = null; DatagramSocketReceiver receiver = null; Thread receiverThread = null; long timeout = 5000; @@ -157,7 +157,7 @@ public void testStatsDHistogramReporting() throws Exception { config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test.host", "localhost"); config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test.port", "" + port); - registry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(config)); + registry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(config)); TaskManagerMetricGroup metricGroup = new TaskManagerMetricGroup(registry, "localhost", "tmId"); @@ -207,7 +207,7 @@ public void testStatsDHistogramReporting() throws Exception { */ @Test public void testStatsDMetersReporting() throws Exception { - MetricRegistry registry = null; + MetricRegistryImpl registry = null; DatagramSocketReceiver receiver = null; Thread receiverThread = null; long timeout = 5000; @@ -231,7 +231,7 @@ public void testStatsDMetersReporting() throws Exception { config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test.host", "localhost"); config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test.port", "" + port); - registry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(config)); + registry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(config)); TaskManagerMetricGroup metricGroup = new TaskManagerMetricGroup(registry, "localhost", "tmId"); TestMeter meter = new TestMeter(); metricGroup.meter(meterName, meter); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java index f9dd98e2778cb..75a844caa5a77 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java @@ -30,6 +30,7 @@ import org.apache.flink.runtime.jobmaster.JobManagerGateway; import org.apache.flink.runtime.leaderelection.TestingListener; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; +import org.apache.flink.runtime.metrics.NoOpMetricRegistry; import org.apache.flink.runtime.rest.handler.util.MimeTypes; import org.apache.flink.runtime.testingUtils.TestingCluster; import org.apache.flink.runtime.testingUtils.TestingUtils; @@ -193,6 +194,7 @@ public void testRedirectToLeader() throws Exception { TestingUtils.defaultExecutor(), TestingUtils.defaultExecutor(), highAvailabilityServices, + new NoOpMetricRegistry(), Option.apply(webMonitor[i].getRestAddress()), JobManager.class, MemoryArchivist.class)._1(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/akka/AkkaJobManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/akka/AkkaJobManagerGateway.java index 68968528ef4b8..08946ed6c994d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/akka/AkkaJobManagerGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/akka/AkkaJobManagerGateway.java @@ -21,11 +21,11 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.instance.Instance; -import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmaster.JobManagerGateway; import org.apache.flink.runtime.messages.Acknowledge; @@ -177,10 +177,10 @@ public CompletableFuture stopJob(JobID jobId, Time timeout) { //-------------------------------------------------------------------------------- @Override - public CompletableFuture> requestTaskManagerInstance(InstanceID instanceId, Time timeout) { + public CompletableFuture> requestTaskManagerInstance(ResourceID resourceId, Time timeout) { return FutureUtils.toJava( jobManagerGateway - .ask(new JobManagerMessages.RequestTaskManagerInstance(instanceId), FutureUtils.toFiniteDuration(timeout)) + .ask(new JobManagerMessages.RequestTaskManagerInstance(resourceId), FutureUtils.toFiniteDuration(timeout)) .mapTo(ClassTag$.MODULE$.apply(JobManagerMessages.TaskManagerInstance.class))) .thenApply( (JobManagerMessages.TaskManagerInstance taskManagerResponse) -> { @@ -265,7 +265,7 @@ public CompletableFuture> requestMetricQueryServicePaths(Time } @Override - public CompletableFuture>> requestTaskManagerMetricQueryServicePaths(Time timeout) { + public CompletableFuture>> requestTaskManagerMetricQueryServicePaths(Time timeout) { return requestTaskManagerInstances(timeout) .thenApply( (Collection instances) -> @@ -277,7 +277,7 @@ public CompletableFuture>> requestTaskMana final String taskManagerMetricQuerServicePath = taskManagerAddress.substring(0, taskManagerAddress.lastIndexOf('/') + 1) + MetricQueryService.METRIC_QUERY_SERVICE_NAME + '_' + instance.getTaskManagerID().getResourceIdString(); - return Tuple2.of(instance.getId(), taskManagerMetricQuerServicePath); + return Tuple2.of(instance.getTaskManagerID(), taskManagerMetricQuerServicePath); }) .collect(Collectors.toList())); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java index dda02751b9411..c2f8539a99ca2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java @@ -31,7 +31,6 @@ import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.RunningJobsRegistry; -import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobmanager.OnCompletionActions; @@ -46,7 +45,7 @@ import org.apache.flink.runtime.messages.webmonitor.ClusterOverview; import org.apache.flink.runtime.messages.webmonitor.JobDetails; import org.apache.flink.runtime.messages.webmonitor.MultipleJobsDetails; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.resourcemanager.ResourceOverview; import org.apache.flink.runtime.rpc.FatalErrorHandler; @@ -89,7 +88,7 @@ public abstract class Dispatcher extends FencedRpcEndpoint impleme private final ResourceManagerGateway resourceManagerGateway; private final JobManagerServices jobManagerServices; private final HeartbeatServices heartbeatServices; - private final MetricRegistry metricRegistry; + private final MetricRegistryImpl metricRegistry; private final FatalErrorHandler fatalErrorHandler; @@ -107,7 +106,7 @@ protected Dispatcher( ResourceManagerGateway resourceManagerGateway, BlobServer blobServer, HeartbeatServices heartbeatServices, - MetricRegistry metricRegistry, + MetricRegistryImpl metricRegistry, FatalErrorHandler fatalErrorHandler, Optional restAddress) throws Exception { super(rpcService, endpointId); @@ -384,7 +383,7 @@ public CompletableFuture> requestMetricQueryServicePaths(Time } @Override - public CompletableFuture>> requestTaskManagerMetricQueryServicePaths(Time timeout) { + public CompletableFuture>> requestTaskManagerMetricQueryServicePaths(Time timeout) { return resourceManagerGateway.requestTaskManagerMetricQueryServicePaths(timeout); } @@ -480,7 +479,7 @@ protected abstract JobManagerRunner createJobManagerRunner( HighAvailabilityServices highAvailabilityServices, HeartbeatServices heartbeatServices, JobManagerServices jobManagerServices, - MetricRegistry metricRegistry, + MetricRegistryImpl metricRegistry, OnCompletionActions onCompleteActions, FatalErrorHandler fatalErrorHandler) throws Exception; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/StandaloneDispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/StandaloneDispatcher.java index 5a6889ef36aae..ee92663beaae7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/StandaloneDispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/StandaloneDispatcher.java @@ -28,7 +28,7 @@ import org.apache.flink.runtime.jobmaster.JobManagerRunner; import org.apache.flink.runtime.jobmaster.JobManagerServices; import org.apache.flink.runtime.jobmaster.JobMaster; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; @@ -49,7 +49,7 @@ public StandaloneDispatcher( ResourceManagerGateway resourceManagerGateway, BlobServer blobServer, HeartbeatServices heartbeatServices, - MetricRegistry metricRegistry, + MetricRegistryImpl metricRegistry, FatalErrorHandler fatalErrorHandler, Optional restAddress) throws Exception { super( @@ -74,7 +74,7 @@ protected JobManagerRunner createJobManagerRunner( HighAvailabilityServices highAvailabilityServices, HeartbeatServices heartbeatServices, JobManagerServices jobManagerServices, - MetricRegistry metricRegistry, + MetricRegistryImpl metricRegistry, OnCompletionActions onCompleteActions, FatalErrorHandler fatalErrorHandler) throws Exception { // create the standard job manager runner diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java index b2ddf1deef79a..1a0e2ae709600 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java @@ -30,8 +30,8 @@ import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; -import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.akka.AkkaRpcService; @@ -76,7 +76,7 @@ public abstract class ClusterEntrypoint implements FatalErrorHandler { private final CompletableFuture terminationFuture; @GuardedBy("lock") - private MetricRegistry metricRegistry = null; + private MetricRegistryImpl metricRegistry = null; @GuardedBy("lock") private HighAvailabilityServices haServices = null; @@ -204,8 +204,8 @@ protected HeartbeatServices createHeartbeatServices(Configuration configuration) return HeartbeatServices.fromConfiguration(configuration); } - protected MetricRegistry createMetricRegistry(Configuration configuration) { - return new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(configuration)); + protected MetricRegistryImpl createMetricRegistry(Configuration configuration) { + return new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(configuration)); } protected void shutDown(boolean cleanupHaData) throws FlinkException { @@ -278,7 +278,7 @@ protected abstract void startClusterComponents( HighAvailabilityServices highAvailabilityServices, BlobServer blobServer, HeartbeatServices heartbeatServices, - MetricRegistry metricRegistry) throws Exception; + MetricRegistryImpl metricRegistry) throws Exception; protected void stopClusterComponents(boolean cleanupHaData) throws Exception { } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/JobClusterEntrypoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/JobClusterEntrypoint.java index 124c6c62fcaba..50d29daf6606c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/JobClusterEntrypoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/JobClusterEntrypoint.java @@ -29,7 +29,7 @@ import org.apache.flink.runtime.jobmanager.OnCompletionActions; import org.apache.flink.runtime.jobmaster.JobManagerRunner; import org.apache.flink.runtime.jobmaster.JobManagerServices; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.resourcemanager.ResourceManager; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; @@ -59,7 +59,7 @@ protected void startClusterComponents( HighAvailabilityServices highAvailabilityServices, BlobServer blobServer, HeartbeatServices heartbeatServices, - MetricRegistry metricRegistry) throws Exception { + MetricRegistryImpl metricRegistry) throws Exception { resourceManager = createResourceManager( configuration, @@ -96,7 +96,7 @@ protected JobManagerRunner createJobManagerRunner( HighAvailabilityServices highAvailabilityServices, JobManagerServices jobManagerServices, HeartbeatServices heartbeatServices, - MetricRegistry metricRegistry, + MetricRegistryImpl metricRegistry, FatalErrorHandler fatalErrorHandler) throws Exception { JobGraph jobGraph = retrieveJobGraph(configuration); @@ -163,7 +163,7 @@ protected abstract ResourceManager createResourceManager( RpcService rpcService, HighAvailabilityServices highAvailabilityServices, HeartbeatServices heartbeatServices, - MetricRegistry metricRegistry, + MetricRegistryImpl metricRegistry, FatalErrorHandler fatalErrorHandler) throws Exception; protected abstract JobGraph retrieveJobGraph(Configuration configuration) throws FlinkException; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/SessionClusterEntrypoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/SessionClusterEntrypoint.java index e24e01addbbde..8a488646ad00f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/SessionClusterEntrypoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/SessionClusterEntrypoint.java @@ -30,7 +30,7 @@ import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.resourcemanager.ResourceManager; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.rest.RestServerEndpointConfiguration; @@ -69,7 +69,7 @@ protected void startClusterComponents( HighAvailabilityServices highAvailabilityServices, BlobServer blobServer, HeartbeatServices heartbeatServices, - MetricRegistry metricRegistry) throws Exception { + MetricRegistryImpl metricRegistry) throws Exception { dispatcherLeaderRetrievalService = highAvailabilityServices.getDispatcherLeaderRetriever(); @@ -173,7 +173,7 @@ protected Dispatcher createDispatcher( ResourceManagerGateway resourceManagerGateway, BlobServer blobServer, HeartbeatServices heartbeatServices, - MetricRegistry metricRegistry, + MetricRegistryImpl metricRegistry, FatalErrorHandler fatalErrorHandler, Optional restAddress) throws Exception { @@ -197,6 +197,6 @@ protected abstract ResourceManager createResourceManager( RpcService rpcService, HighAvailabilityServices highAvailabilityServices, HeartbeatServices heartbeatServices, - MetricRegistry metricRegistry, + MetricRegistryImpl metricRegistry, FatalErrorHandler fatalErrorHandler) throws Exception; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/StandaloneSessionClusterEntrypoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/StandaloneSessionClusterEntrypoint.java index e7c981687a7a4..7d4373d0fd06e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/StandaloneSessionClusterEntrypoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/StandaloneSessionClusterEntrypoint.java @@ -23,7 +23,7 @@ import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.resourcemanager.ResourceManager; import org.apache.flink.runtime.resourcemanager.ResourceManagerConfiguration; import org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServices; @@ -51,7 +51,7 @@ protected ResourceManager createResourceManager( RpcService rpcService, HighAvailabilityServices highAvailabilityServices, HeartbeatServices heartbeatServices, - MetricRegistry metricRegistry, + MetricRegistryImpl metricRegistry, FatalErrorHandler fatalErrorHandler) throws Exception { final ResourceManagerConfiguration resourceManagerConfiguration = ResourceManagerConfiguration.fromConfiguration(configuration); final ResourceManagerRuntimeServicesConfiguration resourceManagerRuntimeServicesConfiguration = ResourceManagerRuntimeServicesConfiguration.fromConfiguration(configuration); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerGateway.java index 782d6d005dd8c..2527e4660b912 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerGateway.java @@ -21,8 +21,8 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.akka.ListeningBehaviour; +import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.instance.Instance; -import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.messages.JobManagerMessages; @@ -107,11 +107,11 @@ public interface JobManagerGateway extends RestfulGateway { * Requests the TaskManager instance registered under the given instanceId from the JobManager. * If there is no Instance registered, then {@link Optional#empty()} is returned. * - * @param instanceId for which to retrieve the Instance + * @param resourceId identifying the TaskManager which shall be retrieved * @param timeout for the asynchronous operation * @return Future containing the TaskManager instance registered under instanceId, otherwise {@link Optional#empty()} */ - CompletableFuture> requestTaskManagerInstance(InstanceID instanceId, Time timeout); + CompletableFuture> requestTaskManagerInstance(ResourceID resourceId, Time timeout); /** * Requests all currently registered TaskManager instances from the JobManager. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java index 14baa6f0f88c1..0a85bbe873068 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java @@ -34,7 +34,7 @@ import org.apache.flink.runtime.leaderelection.LeaderContender; import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; @@ -111,7 +111,7 @@ public JobManagerRunner( final HighAvailabilityServices haServices, final HeartbeatServices heartbeatServices, final JobManagerServices jobManagerServices, - final MetricRegistry metricRegistry, + final MetricRegistryImpl metricRegistry, final OnCompletionActions toNotifyOnComplete, final FatalErrorHandler errorHandler) throws Exception { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistry.java index 278292d313c3d..9aa97cb123e86 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistry.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistry.java @@ -18,283 +18,34 @@ package org.apache.flink.runtime.metrics; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.Configuration; import org.apache.flink.metrics.Metric; -import org.apache.flink.metrics.MetricConfig; -import org.apache.flink.metrics.MetricGroup; -import org.apache.flink.metrics.View; -import org.apache.flink.metrics.reporter.MetricReporter; -import org.apache.flink.metrics.reporter.Scheduled; -import org.apache.flink.runtime.akka.AkkaUtils; -import org.apache.flink.runtime.clusterframework.types.ResourceID; -import org.apache.flink.runtime.metrics.dump.MetricQueryService; import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup; -import org.apache.flink.runtime.metrics.groups.FrontMetricGroup; import org.apache.flink.runtime.metrics.scope.ScopeFormats; -import org.apache.flink.runtime.util.ExecutorThreadFactory; -import org.apache.flink.util.Preconditions; - -import akka.actor.ActorRef; -import akka.actor.ActorSystem; -import akka.actor.Kill; -import akka.pattern.Patterns; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.annotation.Nullable; - -import java.util.ArrayList; -import java.util.List; -import java.util.TimerTask; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; - -import scala.concurrent.Await; -import scala.concurrent.Future; -import scala.concurrent.duration.FiniteDuration; /** - * A MetricRegistry keeps track of all registered {@link Metric Metrics}. It serves as the - * connection between {@link MetricGroup MetricGroups} and {@link MetricReporter MetricReporters}. + * Interface for a metric registry. */ -public class MetricRegistry { - static final Logger LOG = LoggerFactory.getLogger(MetricRegistry.class); - - private final Object lock = new Object(); - - private List reporters; - private ScheduledExecutorService executor; - - @Nullable - private ActorRef queryService; - - @Nullable - private String metricQueryServicePath; - - private ViewUpdater viewUpdater; - - private final ScopeFormats scopeFormats; - private final char globalDelimiter; - private final List delimiters = new ArrayList<>(); - - /** - * Creates a new MetricRegistry and starts the configured reporter. - */ - public MetricRegistry(MetricRegistryConfiguration config) { - this.scopeFormats = config.getScopeFormats(); - this.globalDelimiter = config.getDelimiter(); - - // second, instantiate any custom configured reporters - this.reporters = new ArrayList<>(); - - List> reporterConfigurations = config.getReporterConfigurations(); - - this.executor = Executors.newSingleThreadScheduledExecutor(new ExecutorThreadFactory("Flink-MetricRegistry")); - - this.queryService = null; - this.metricQueryServicePath = null; - - if (reporterConfigurations.isEmpty()) { - // no reporters defined - // by default, don't report anything - LOG.info("No metrics reporter configured, no metrics will be exposed/reported."); - } else { - // we have some reporters so - for (Tuple2 reporterConfiguration: reporterConfigurations) { - String namedReporter = reporterConfiguration.f0; - Configuration reporterConfig = reporterConfiguration.f1; - - final String className = reporterConfig.getString(ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, null); - if (className == null) { - LOG.error("No reporter class set for reporter " + namedReporter + ". Metrics might not be exposed/reported."); - continue; - } - - try { - String configuredPeriod = reporterConfig.getString(ConfigConstants.METRICS_REPORTER_INTERVAL_SUFFIX, null); - TimeUnit timeunit = TimeUnit.SECONDS; - long period = 10; - - if (configuredPeriod != null) { - try { - String[] interval = configuredPeriod.split(" "); - period = Long.parseLong(interval[0]); - timeunit = TimeUnit.valueOf(interval[1]); - } - catch (Exception e) { - LOG.error("Cannot parse report interval from config: " + configuredPeriod + - " - please use values like '10 SECONDS' or '500 MILLISECONDS'. " + - "Using default reporting interval."); - } - } - - Class reporterClass = Class.forName(className); - MetricReporter reporterInstance = (MetricReporter) reporterClass.newInstance(); - - MetricConfig metricConfig = new MetricConfig(); - reporterConfig.addAllToProperties(metricConfig); - LOG.info("Configuring {} with {}.", reporterClass.getSimpleName(), metricConfig); - reporterInstance.open(metricConfig); - - if (reporterInstance instanceof Scheduled) { - LOG.info("Periodically reporting metrics in intervals of {} {} for reporter {} of type {}.", period, timeunit.name(), namedReporter, className); - - executor.scheduleWithFixedDelay( - new MetricRegistry.ReporterTask((Scheduled) reporterInstance), period, period, timeunit); - } else { - LOG.info("Reporting metrics for reporter {} of type {}.", namedReporter, className); - } - reporters.add(reporterInstance); - - String delimiterForReporter = reporterConfig.getString(ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER, String.valueOf(globalDelimiter)); - if (delimiterForReporter.length() != 1) { - LOG.warn("Failed to parse delimiter '{}' for reporter '{}', using global delimiter '{}'.", delimiterForReporter, namedReporter, globalDelimiter); - delimiterForReporter = String.valueOf(globalDelimiter); - } - this.delimiters.add(delimiterForReporter.charAt(0)); - } - catch (Throwable t) { - LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", namedReporter, t); - } - } - } - } - - /** - * Initializes the MetricQueryService. - * - * @param actorSystem ActorSystem to create the MetricQueryService on - * @param resourceID resource ID used to disambiguate the actor name - */ - public void startQueryService(ActorSystem actorSystem, ResourceID resourceID) { - synchronized (lock) { - Preconditions.checkState(!isShutdown(), "The metric registry has already been shut down."); - - try { - queryService = MetricQueryService.startMetricQueryService(actorSystem, resourceID); - metricQueryServicePath = AkkaUtils.getAkkaURL(actorSystem, queryService); - } catch (Exception e) { - LOG.warn("Could not start MetricDumpActor. No metrics will be submitted to the WebInterface.", e); - } - } - } - - /** - * Returns the address under which the {@link MetricQueryService} is reachable. - * - * @return address of the metric query service - */ - @Nullable - public String getMetricQueryServicePath() { - return metricQueryServicePath; - } +public interface MetricRegistry { /** * Returns the global delimiter. * * @return global delimiter */ - public char getDelimiter() { - return this.globalDelimiter; - } + char getDelimiter(); /** * Returns the configured delimiter for the reporter with the given index. * - * @param reporterIndex index of the reporter whose delimiter should be used + * @param index index of the reporter whose delimiter should be used * @return configured reporter delimiter, or global delimiter if index is invalid */ - public char getDelimiter(int reporterIndex) { - try { - return delimiters.get(reporterIndex); - } catch (IndexOutOfBoundsException e) { - LOG.warn("Delimiter for reporter index {} not found, returning global delimiter.", reporterIndex); - return this.globalDelimiter; - } - } - - public List getReporters() { - return reporters; - } - - /** - * Returns whether this registry has been shutdown. - * - * @return true, if this registry was shutdown, otherwise false - */ - public boolean isShutdown() { - synchronized (lock) { - return reporters == null && executor.isShutdown(); - } - } + char getDelimiter(int index); /** - * Shuts down this registry and the associated {@link MetricReporter}. + * Returns the number of registered reporters. */ - public void shutdown() { - synchronized (lock) { - Future stopFuture = null; - FiniteDuration stopTimeout = null; - - if (queryService != null) { - stopTimeout = new FiniteDuration(1L, TimeUnit.SECONDS); - stopFuture = Patterns.gracefulStop(queryService, stopTimeout); - } - - if (reporters != null) { - for (MetricReporter reporter : reporters) { - try { - reporter.close(); - } catch (Throwable t) { - LOG.warn("Metrics reporter did not shut down cleanly", t); - } - } - reporters = null; - } - shutdownExecutor(); - - if (stopFuture != null) { - boolean stopped = false; - - try { - stopped = Await.result(stopFuture, stopTimeout); - } catch (Exception e) { - LOG.warn("Query actor did not properly stop.", e); - } - - if (!stopped) { - // the query actor did not stop in time, let's kill him - queryService.tell(Kill.getInstance(), ActorRef.noSender()); - } - } - } - } - - private void shutdownExecutor() { - if (executor != null) { - executor.shutdown(); - - try { - if (!executor.awaitTermination(1L, TimeUnit.SECONDS)) { - executor.shutdownNow(); - } - } catch (InterruptedException e) { - executor.shutdownNow(); - } - } - } - - public ScopeFormats getScopeFormats() { - return scopeFormats; - } - - // ------------------------------------------------------------------------ - // Metrics (de)registration - // ------------------------------------------------------------------------ + int getNumberReporters(); /** * Registers a new {@link Metric} with this registry. @@ -303,44 +54,7 @@ public ScopeFormats getScopeFormats() { * @param metricName the name of the metric * @param group the group that contains the metric */ - public void register(Metric metric, String metricName, AbstractMetricGroup group) { - synchronized (lock) { - if (isShutdown()) { - LOG.warn("Cannot register metric, because the MetricRegistry has already been shut down."); - } else { - if (reporters != null) { - for (int i = 0; i < reporters.size(); i++) { - MetricReporter reporter = reporters.get(i); - try { - if (reporter != null) { - FrontMetricGroup front = new FrontMetricGroup>(i, group); - reporter.notifyOfAddedMetric(metric, metricName, front); - } - } catch (Exception e) { - LOG.warn("Error while registering metric.", e); - } - } - } - try { - if (queryService != null) { - MetricQueryService.notifyOfAddedMetric(queryService, metric, metricName, group); - } - } catch (Exception e) { - LOG.warn("Error while registering metric.", e); - } - try { - if (metric instanceof View) { - if (viewUpdater == null) { - viewUpdater = new ViewUpdater(executor); - } - viewUpdater.notifyOfAddedView((View) metric); - } - } catch (Exception e) { - LOG.warn("Error while registering metric.", e); - } - } - } - } + void register(Metric metric, String metricName, AbstractMetricGroup group); /** * Un-registers the given {@link Metric} with this registry. @@ -349,79 +63,7 @@ public void register(Metric metric, String metricName, AbstractMetricGroup group * @param metricName the name of the metric * @param group the group that contains the metric */ - public void unregister(Metric metric, String metricName, AbstractMetricGroup group) { - synchronized (lock) { - if (isShutdown()) { - LOG.warn("Cannot unregister metric, because the MetricRegistry has already been shut down."); - } else { - if (reporters != null) { - for (int i = 0; i < reporters.size(); i++) { - try { - MetricReporter reporter = reporters.get(i); - if (reporter != null) { - FrontMetricGroup front = new FrontMetricGroup>(i, group); - reporter.notifyOfRemovedMetric(metric, metricName, front); - } - } catch (Exception e) { - LOG.warn("Error while registering metric.", e); - } - } - } - try { - if (queryService != null) { - MetricQueryService.notifyOfRemovedMetric(queryService, metric); - } - } catch (Exception e) { - LOG.warn("Error while registering metric.", e); - } - try { - if (metric instanceof View) { - if (viewUpdater != null) { - viewUpdater.notifyOfRemovedView((View) metric); - } - } - } catch (Exception e) { - LOG.warn("Error while registering metric.", e); - } - } - } - } - - // ------------------------------------------------------------------------ - - @VisibleForTesting - @Nullable - public ActorRef getQueryService() { - return queryService; - } - - // ------------------------------------------------------------------------ - - /** - * This task is explicitly a static class, so that it does not hold any references to the enclosing - * MetricsRegistry instance. - * - *

This is a subtle difference, but very important: With this static class, the enclosing class instance - * may become garbage-collectible, whereas with an anonymous inner class, the timer thread - * (which is a GC root) will hold a reference via the timer task and its enclosing instance pointer. - * Making the MetricsRegistry garbage collectible makes the java.util.Timer garbage collectible, - * which acts as a fail-safe to stop the timer thread and prevents resource leaks. - */ - private static final class ReporterTask extends TimerTask { - - private final Scheduled reporter; - - private ReporterTask(Scheduled reporter) { - this.reporter = reporter; - } + void unregister(Metric metric, String metricName, AbstractMetricGroup group); - @Override - public void run() { - try { - reporter.report(); - } catch (Throwable t) { - LOG.warn("Error while reporting metrics", t); - } - } - } + ScopeFormats getScopeFormats(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistryConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistryConfiguration.java index e72a980ebce74..d07cb6599357b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistryConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistryConfiguration.java @@ -35,7 +35,7 @@ import java.util.regex.Pattern; /** - * Configuration object for {@link MetricRegistry}. + * Configuration object for {@link MetricRegistryImpl}. */ public class MetricRegistryConfiguration { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistryImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistryImpl.java new file mode 100644 index 0000000000000..407fa8b678b5e --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistryImpl.java @@ -0,0 +1,412 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.metrics; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.MetricConfig; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.View; +import org.apache.flink.metrics.reporter.MetricReporter; +import org.apache.flink.metrics.reporter.Scheduled; +import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.metrics.dump.MetricQueryService; +import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup; +import org.apache.flink.runtime.metrics.groups.FrontMetricGroup; +import org.apache.flink.runtime.metrics.scope.ScopeFormats; +import org.apache.flink.runtime.util.ExecutorThreadFactory; +import org.apache.flink.util.Preconditions; + +import akka.actor.ActorRef; +import akka.actor.ActorSystem; +import akka.actor.Kill; +import akka.pattern.Patterns; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.List; +import java.util.TimerTask; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +import scala.concurrent.Await; +import scala.concurrent.Future; +import scala.concurrent.duration.FiniteDuration; + +/** + * A MetricRegistry keeps track of all registered {@link Metric Metrics}. It serves as the + * connection between {@link MetricGroup MetricGroups} and {@link MetricReporter MetricReporters}. + */ +public class MetricRegistryImpl implements MetricRegistry { + static final Logger LOG = LoggerFactory.getLogger(MetricRegistryImpl.class); + + private final Object lock = new Object(); + + private List reporters; + private ScheduledExecutorService executor; + + @Nullable + private ActorRef queryService; + + @Nullable + private String metricQueryServicePath; + + private ViewUpdater viewUpdater; + + private final ScopeFormats scopeFormats; + private final char globalDelimiter; + private final List delimiters = new ArrayList<>(); + + /** + * Creates a new MetricRegistry and starts the configured reporter. + */ + public MetricRegistryImpl(MetricRegistryConfiguration config) { + this.scopeFormats = config.getScopeFormats(); + this.globalDelimiter = config.getDelimiter(); + + // second, instantiate any custom configured reporters + this.reporters = new ArrayList<>(); + + List> reporterConfigurations = config.getReporterConfigurations(); + + this.executor = Executors.newSingleThreadScheduledExecutor(new ExecutorThreadFactory("Flink-MetricRegistry")); + + this.queryService = null; + this.metricQueryServicePath = null; + + if (reporterConfigurations.isEmpty()) { + // no reporters defined + // by default, don't report anything + LOG.info("No metrics reporter configured, no metrics will be exposed/reported."); + } else { + // we have some reporters so + for (Tuple2 reporterConfiguration: reporterConfigurations) { + String namedReporter = reporterConfiguration.f0; + Configuration reporterConfig = reporterConfiguration.f1; + + final String className = reporterConfig.getString(ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, null); + if (className == null) { + LOG.error("No reporter class set for reporter " + namedReporter + ". Metrics might not be exposed/reported."); + continue; + } + + try { + String configuredPeriod = reporterConfig.getString(ConfigConstants.METRICS_REPORTER_INTERVAL_SUFFIX, null); + TimeUnit timeunit = TimeUnit.SECONDS; + long period = 10; + + if (configuredPeriod != null) { + try { + String[] interval = configuredPeriod.split(" "); + period = Long.parseLong(interval[0]); + timeunit = TimeUnit.valueOf(interval[1]); + } + catch (Exception e) { + LOG.error("Cannot parse report interval from config: " + configuredPeriod + + " - please use values like '10 SECONDS' or '500 MILLISECONDS'. " + + "Using default reporting interval."); + } + } + + Class reporterClass = Class.forName(className); + MetricReporter reporterInstance = (MetricReporter) reporterClass.newInstance(); + + MetricConfig metricConfig = new MetricConfig(); + reporterConfig.addAllToProperties(metricConfig); + LOG.info("Configuring {} with {}.", reporterClass.getSimpleName(), metricConfig); + reporterInstance.open(metricConfig); + + if (reporterInstance instanceof Scheduled) { + LOG.info("Periodically reporting metrics in intervals of {} {} for reporter {} of type {}.", period, timeunit.name(), namedReporter, className); + + executor.scheduleWithFixedDelay( + new MetricRegistryImpl.ReporterTask((Scheduled) reporterInstance), period, period, timeunit); + } else { + LOG.info("Reporting metrics for reporter {} of type {}.", namedReporter, className); + } + reporters.add(reporterInstance); + + String delimiterForReporter = reporterConfig.getString(ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER, String.valueOf(globalDelimiter)); + if (delimiterForReporter.length() != 1) { + LOG.warn("Failed to parse delimiter '{}' for reporter '{}', using global delimiter '{}'.", delimiterForReporter, namedReporter, globalDelimiter); + delimiterForReporter = String.valueOf(globalDelimiter); + } + this.delimiters.add(delimiterForReporter.charAt(0)); + } + catch (Throwable t) { + LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", namedReporter, t); + } + } + } + } + + /** + * Initializes the MetricQueryService. + * + * @param actorSystem ActorSystem to create the MetricQueryService on + * @param resourceID resource ID used to disambiguate the actor name + */ + public void startQueryService(ActorSystem actorSystem, ResourceID resourceID) { + synchronized (lock) { + Preconditions.checkState(!isShutdown(), "The metric registry has already been shut down."); + + try { + queryService = MetricQueryService.startMetricQueryService(actorSystem, resourceID); + metricQueryServicePath = AkkaUtils.getAkkaURL(actorSystem, queryService); + } catch (Exception e) { + LOG.warn("Could not start MetricDumpActor. No metrics will be submitted to the WebInterface.", e); + } + } + } + + /** + * Returns the address under which the {@link MetricQueryService} is reachable. + * + * @return address of the metric query service + */ + @Nullable + public String getMetricQueryServicePath() { + return metricQueryServicePath; + } + + @Override + public char getDelimiter() { + return this.globalDelimiter; + } + + @Override + public char getDelimiter(int reporterIndex) { + try { + return delimiters.get(reporterIndex); + } catch (IndexOutOfBoundsException e) { + LOG.warn("Delimiter for reporter index {} not found, returning global delimiter.", reporterIndex); + return this.globalDelimiter; + } + } + + @Override + public int getNumberReporters() { + return reporters.size(); + } + + public List getReporters() { + return reporters; + } + + /** + * Returns whether this registry has been shutdown. + * + * @return true, if this registry was shutdown, otherwise false + */ + public boolean isShutdown() { + synchronized (lock) { + return reporters == null && executor.isShutdown(); + } + } + + /** + * Shuts down this registry and the associated {@link MetricReporter}. + */ + public void shutdown() { + synchronized (lock) { + Future stopFuture = null; + FiniteDuration stopTimeout = null; + + if (queryService != null) { + stopTimeout = new FiniteDuration(1L, TimeUnit.SECONDS); + stopFuture = Patterns.gracefulStop(queryService, stopTimeout); + } + + if (reporters != null) { + for (MetricReporter reporter : reporters) { + try { + reporter.close(); + } catch (Throwable t) { + LOG.warn("Metrics reporter did not shut down cleanly", t); + } + } + reporters = null; + } + shutdownExecutor(); + + if (stopFuture != null) { + boolean stopped = false; + + try { + stopped = Await.result(stopFuture, stopTimeout); + } catch (Exception e) { + LOG.warn("Query actor did not properly stop.", e); + } + + if (!stopped) { + // the query actor did not stop in time, let's kill him + queryService.tell(Kill.getInstance(), ActorRef.noSender()); + } + } + } + } + + private void shutdownExecutor() { + if (executor != null) { + executor.shutdown(); + + try { + if (!executor.awaitTermination(1L, TimeUnit.SECONDS)) { + executor.shutdownNow(); + } + } catch (InterruptedException e) { + executor.shutdownNow(); + } + } + } + + @Override + public ScopeFormats getScopeFormats() { + return scopeFormats; + } + + // ------------------------------------------------------------------------ + // Metrics (de)registration + // ------------------------------------------------------------------------ + + @Override + public void register(Metric metric, String metricName, AbstractMetricGroup group) { + synchronized (lock) { + if (isShutdown()) { + LOG.warn("Cannot register metric, because the MetricRegistry has already been shut down."); + } else { + if (reporters != null) { + for (int i = 0; i < reporters.size(); i++) { + MetricReporter reporter = reporters.get(i); + try { + if (reporter != null) { + FrontMetricGroup front = new FrontMetricGroup>(i, group); + reporter.notifyOfAddedMetric(metric, metricName, front); + } + } catch (Exception e) { + LOG.warn("Error while registering metric.", e); + } + } + } + try { + if (queryService != null) { + MetricQueryService.notifyOfAddedMetric(queryService, metric, metricName, group); + } + } catch (Exception e) { + LOG.warn("Error while registering metric.", e); + } + try { + if (metric instanceof View) { + if (viewUpdater == null) { + viewUpdater = new ViewUpdater(executor); + } + viewUpdater.notifyOfAddedView((View) metric); + } + } catch (Exception e) { + LOG.warn("Error while registering metric.", e); + } + } + } + } + + @Override + public void unregister(Metric metric, String metricName, AbstractMetricGroup group) { + synchronized (lock) { + if (isShutdown()) { + LOG.warn("Cannot unregister metric, because the MetricRegistry has already been shut down."); + } else { + if (reporters != null) { + for (int i = 0; i < reporters.size(); i++) { + try { + MetricReporter reporter = reporters.get(i); + if (reporter != null) { + FrontMetricGroup front = new FrontMetricGroup>(i, group); + reporter.notifyOfRemovedMetric(metric, metricName, front); + } + } catch (Exception e) { + LOG.warn("Error while registering metric.", e); + } + } + } + try { + if (queryService != null) { + MetricQueryService.notifyOfRemovedMetric(queryService, metric); + } + } catch (Exception e) { + LOG.warn("Error while registering metric.", e); + } + try { + if (metric instanceof View) { + if (viewUpdater != null) { + viewUpdater.notifyOfRemovedView((View) metric); + } + } + } catch (Exception e) { + LOG.warn("Error while registering metric.", e); + } + } + } + } + + // ------------------------------------------------------------------------ + + @VisibleForTesting + @Nullable + public ActorRef getQueryService() { + return queryService; + } + + // ------------------------------------------------------------------------ + + /** + * This task is explicitly a static class, so that it does not hold any references to the enclosing + * MetricsRegistry instance. + * + *

This is a subtle difference, but very important: With this static class, the enclosing class instance + * may become garbage-collectible, whereas with an anonymous inner class, the timer thread + * (which is a GC root) will hold a reference via the timer task and its enclosing instance pointer. + * Making the MetricsRegistry garbage collectible makes the java.util.Timer garbage collectible, + * which acts as a fail-safe to stop the timer thread and prevents resource leaks. + */ + private static final class ReporterTask extends TimerTask { + + private final Scheduled reporter; + + private ReporterTask(Scheduled reporter) { + this.reporter = reporter; + } + + @Override + public void run() { + try { + reporter.report(); + } catch (Throwable t) { + LOG.warn("Error while reporting metrics", t); + } + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroup.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroup.java index ab59977e4edda..66eace588582a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroup.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroup.java @@ -106,7 +106,7 @@ public AbstractMetricGroup(MetricRegistry registry, String[] scope, A parent) { this.registry = checkNotNull(registry); this.scopeComponents = checkNotNull(scope); this.parent = parent; - this.scopeStrings = new String[registry.getReporters() == null ? 0 : registry.getReporters().size()]; + this.scopeStrings = new String[registry.getNumberReporters()]; } public Map getAllVariables() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java index dd352bb718d2a..d4248eeb14489 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java @@ -32,7 +32,7 @@ import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.leaderelection.LeaderAddressAndId; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.resourcemanager.ResourceManagerId; @@ -68,7 +68,7 @@ public class MiniCluster { private final MiniClusterConfiguration miniClusterConfiguration; @GuardedBy("lock") - private MetricRegistry metricRegistry; + private MetricRegistryImpl metricRegistry; @GuardedBy("lock") private RpcService commonRpcService; @@ -464,8 +464,8 @@ public JobExecutionResult runJobBlocking(JobGraph job) throws JobExecutionExcept * * @param config The configuration of the mini cluster */ - protected MetricRegistry createMetricRegistry(Configuration config) { - return new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(config)); + protected MetricRegistryImpl createMetricRegistry(Configuration config) { + return new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(config)); } /** @@ -502,7 +502,7 @@ protected ResourceManagerRunner[] startResourceManagers( Configuration configuration, HighAvailabilityServices haServices, HeartbeatServices heartbeatServices, - MetricRegistry metricRegistry, + MetricRegistryImpl metricRegistry, int numResourceManagers, RpcService[] resourceManagerRpcServices) throws Exception { @@ -528,7 +528,7 @@ protected ResourceManagerRunner[] startResourceManagers( protected TaskExecutor[] startTaskManagers( Configuration configuration, HighAvailabilityServices haServices, - MetricRegistry metricRegistry, + MetricRegistryImpl metricRegistry, int numTaskManagers, RpcService[] taskManagerRpcServices) throws Exception { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterJobDispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterJobDispatcher.java index 60d9a6692609a..ca042b624350a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterJobDispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterJobDispatcher.java @@ -30,7 +30,7 @@ import org.apache.flink.runtime.jobmanager.OnCompletionActions; import org.apache.flink.runtime.jobmaster.JobManagerRunner; import org.apache.flink.runtime.jobmaster.JobManagerServices; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.util.ExceptionUtils; @@ -75,7 +75,7 @@ public class MiniClusterJobDispatcher { private final JobManagerServices jobManagerServices; /** Registry for all metrics in the mini cluster */ - private final MetricRegistry metricRegistry; + private final MetricRegistryImpl metricRegistry; /** The number of JobManagers to launch (more than one simulates a high-availability setup) */ private final int numJobManagers; @@ -104,7 +104,7 @@ public MiniClusterJobDispatcher( HighAvailabilityServices haServices, BlobServer blobServer, HeartbeatServices heartbeatServices, - MetricRegistry metricRegistry) throws Exception { + MetricRegistryImpl metricRegistry) throws Exception { this( config, haServices, @@ -132,7 +132,7 @@ public MiniClusterJobDispatcher( HighAvailabilityServices haServices, BlobServer blobServer, HeartbeatServices heartbeatServices, - MetricRegistry metricRegistry, + MetricRegistryImpl metricRegistry, int numJobManagers, RpcService[] rpcServices) throws Exception { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/StandaloneMiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/StandaloneMiniCluster.java index cbefe5ac40bcb..90fb115f97363 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/StandaloneMiniCluster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/StandaloneMiniCluster.java @@ -18,9 +18,6 @@ package org.apache.flink.runtime.minicluster; -import akka.actor.ActorRef; -import akka.actor.ActorSystem; -import akka.pattern.Patterns; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.runtime.akka.AkkaUtils; @@ -31,18 +28,25 @@ import org.apache.flink.runtime.jobmanager.JobManager; import org.apache.flink.runtime.jobmanager.MemoryArchivist; import org.apache.flink.runtime.messages.TaskManagerMessages; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; +import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; import org.apache.flink.runtime.taskmanager.TaskManager; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.Preconditions; -import scala.Option; -import scala.concurrent.Await; -import scala.concurrent.Future; -import scala.concurrent.duration.FiniteDuration; + +import akka.actor.ActorRef; +import akka.actor.ActorSystem; +import akka.pattern.Patterns; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; +import scala.Option; +import scala.concurrent.Await; +import scala.concurrent.Future; +import scala.concurrent.duration.FiniteDuration; + /** * Mini cluster to run the old JobManager code without embedded high availability services. This * class has been implemented because the normal {@link FlinkMiniCluster} has been changed to use @@ -63,6 +67,8 @@ public class StandaloneMiniCluster { private final HighAvailabilityServices highAvailabilityServices; + private final MetricRegistryImpl metricRegistry; + private final FiniteDuration timeout; private final int port; @@ -86,21 +92,28 @@ public StandaloneMiniCluster(Configuration configuration) throws Exception { Executors.directExecutor(), HighAvailabilityServicesUtils.AddressResolution.TRY_ADDRESS_RESOLUTION); + metricRegistry = new MetricRegistryImpl( + MetricRegistryConfiguration.fromConfiguration(configuration)); + JobManager.startJobManagerActors( configuration, actorSystem, scheduledExecutorService, scheduledExecutorService, highAvailabilityServices, + metricRegistry, Option.empty(), JobManager.class, MemoryArchivist.class); + final ResourceID taskManagerResourceId = ResourceID.generate(); + ActorRef taskManager = TaskManager.startTaskManagerComponentsAndActor( configuration, - ResourceID.generate(), + taskManagerResourceId, actorSystem, highAvailabilityServices, + metricRegistry, LOCAL_HOSTNAME, Option.empty(), true, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java index 1d4d4f3a0760b..98b80c636a492 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java @@ -42,7 +42,7 @@ import org.apache.flink.runtime.leaderelection.LeaderContender; import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.metrics.dump.MetricQueryService; import org.apache.flink.runtime.registration.RegistrationResponse; import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException; @@ -118,7 +118,7 @@ public abstract class ResourceManager private final HeartbeatManager jobManagerHeartbeatManager; /** Registry to use for metrics. */ - private final MetricRegistry metricRegistry; + private final MetricRegistryImpl metricRegistry; /** Fatal error handler. */ private final FatalErrorHandler fatalErrorHandler; @@ -140,7 +140,7 @@ public ResourceManager( HighAvailabilityServices highAvailabilityServices, HeartbeatServices heartbeatServices, SlotManager slotManager, - MetricRegistry metricRegistry, + MetricRegistryImpl metricRegistry, JobLeaderIdService jobLeaderIdService, FatalErrorHandler fatalErrorHandler) { @@ -498,8 +498,8 @@ public CompletableFuture requestResourceOverview(Time timeout) } @Override - public CompletableFuture>> requestTaskManagerMetricQueryServicePaths(Time timeout) { - final ArrayList> metricQueryServicePaths = new ArrayList<>(taskExecutors.size()); + public CompletableFuture>> requestTaskManagerMetricQueryServicePaths(Time timeout) { + final ArrayList> metricQueryServicePaths = new ArrayList<>(taskExecutors.size()); for (Map.Entry> workerRegistrationEntry : taskExecutors.entrySet()) { final ResourceID tmResourceId = workerRegistrationEntry.getKey(); @@ -508,7 +508,7 @@ public CompletableFuture>> requestTaskMana final String tmMetricQueryServicePath = taskManagerAddress.substring(0, taskManagerAddress.lastIndexOf('/') + 1) + MetricQueryService.METRIC_QUERY_SERVICE_NAME + '_' + tmResourceId.getResourceIdString(); - metricQueryServicePaths.add(Tuple2.of(workerRegistration.getInstanceID(), tmMetricQueryServicePath)); + metricQueryServicePaths.add(Tuple2.of(tmResourceId, tmMetricQueryServicePath)); } return CompletableFuture.completedFuture(metricQueryServicePaths); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java index 9eacb4b9558a3..cc2766b4d6ff4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java @@ -176,5 +176,5 @@ void notifySlotAvailable( * @param timeout for the asynchronous operation * @return Future containing the collection of instance ids and the corresponding metric query service path */ - CompletableFuture>> requestTaskManagerMetricQueryServicePaths(@RpcTimeout Time timeout); + CompletableFuture>> requestTaskManagerMetricQueryServicePaths(@RpcTimeout Time timeout); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRunner.java index caa3ba0a6c913..361bdd4332053 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRunner.java @@ -22,7 +22,7 @@ import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.util.FlinkException; @@ -55,7 +55,7 @@ public ResourceManagerRunner( final RpcService rpcService, final HighAvailabilityServices highAvailabilityServices, final HeartbeatServices heartbeatServices, - final MetricRegistry metricRegistry) throws Exception { + final MetricRegistryImpl metricRegistry) throws Exception { Preconditions.checkNotNull(resourceId); Preconditions.checkNotNull(configuration); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java index 624f31ddc8b8e..d2b1205694eb9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java @@ -23,7 +23,7 @@ import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; import org.apache.flink.runtime.rpc.FatalErrorHandler; @@ -45,7 +45,7 @@ public StandaloneResourceManager( HighAvailabilityServices highAvailabilityServices, HeartbeatServices heartbeatServices, SlotManager slotManager, - MetricRegistry metricRegistry, + MetricRegistryImpl metricRegistry, JobLeaderIdService jobLeaderIdService, FatalErrorHandler fatalErrorHandler) { super( diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/TaskManagerLogHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/TaskManagerLogHandler.java index cf5bfcb83b3a0..4d6ccd5b2125b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/TaskManagerLogHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/TaskManagerLogHandler.java @@ -31,8 +31,8 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.blob.TransientBlobCache; import org.apache.flink.runtime.blob.TransientBlobKey; +import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.instance.Instance; -import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.jobmaster.JobManagerGateway; import org.apache.flink.runtime.rest.handler.RedirectHandler; import org.apache.flink.runtime.rest.handler.WebHandler; @@ -166,13 +166,13 @@ protected void respondAsLeader(final ChannelHandlerContext ctx, final Routed rou //fetch TaskManager logs if no other process is currently doing it if (lastRequestPending.putIfAbsent(taskManagerID, true) == null) { try { - InstanceID instanceID = new InstanceID(StringUtils.hexStringToByte(taskManagerID)); - CompletableFuture> taskManagerFuture = jobManagerGateway.requestTaskManagerInstance(instanceID, timeout); + ResourceID resourceId = new ResourceID(new String(StringUtils.hexStringToByte(taskManagerID))); + CompletableFuture> taskManagerFuture = jobManagerGateway.requestTaskManagerInstance(resourceId, timeout); CompletableFuture blobKeyFuture = taskManagerFuture.thenCompose( (Optional optTMInstance) -> { Instance taskManagerInstance = optTMInstance.orElseThrow( - () -> new CompletionException(new FlinkException("Could not find instance with " + instanceID + '.'))); + () -> new CompletionException(new FlinkException("Could not find instance with " + resourceId + '.'))); switch (fileMode) { case LOG: return taskManagerInstance.getTaskManagerGateway().requestTaskManagerLog(timeout); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/TaskManagersHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/TaskManagersHandler.java index ad2ee1b6e99f3..84c6e418c885f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/TaskManagersHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/TaskManagersHandler.java @@ -19,19 +19,20 @@ package org.apache.flink.runtime.rest.handler.legacy; import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.instance.Instance; -import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.jobmaster.JobManagerGateway; import org.apache.flink.runtime.rest.handler.legacy.metrics.MetricFetcher; import org.apache.flink.runtime.rest.handler.legacy.metrics.MetricStore; import org.apache.flink.util.FlinkException; -import org.apache.flink.util.StringUtils; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; import java.io.IOException; import java.io.StringWriter; +import java.io.UnsupportedEncodingException; +import java.net.URLDecoder; import java.util.Collection; import java.util.Collections; import java.util.Map; @@ -74,8 +75,16 @@ public CompletableFuture handleJsonRequest(Map pathParam // return them in an array. This avoids unnecessary code complexity. // If only one task manager is requested, we only fetch one task manager metrics. if (pathParams.containsKey(TASK_MANAGER_ID_KEY)) { - InstanceID instanceID = new InstanceID(StringUtils.hexStringToByte(pathParams.get(TASK_MANAGER_ID_KEY))); - CompletableFuture> tmInstanceFuture = jobManagerGateway.requestTaskManagerInstance(instanceID, timeout); + final String unescapedString; + + try { + unescapedString = URLDecoder.decode(pathParams.get(TASK_MANAGER_ID_KEY), "UTF-8"); + } catch (UnsupportedEncodingException e) { + return FutureUtils.completedExceptionally(new FlinkException("Could not decode task manager id: " + pathParams.get(TASK_MANAGER_ID_KEY) + '.', e)); + } + + ResourceID resourceId = new ResourceID(unescapedString); + CompletableFuture> tmInstanceFuture = jobManagerGateway.requestTaskManagerInstance(resourceId, timeout); return tmInstanceFuture.thenApplyAsync( (Optional optTaskManager) -> { @@ -116,7 +125,7 @@ private String writeTaskManagersJson(Collection instances, Map instances, Map>> taskManagerQueryServicePathsFuture = leaderGateway + CompletableFuture>> taskManagerQueryServicePathsFuture = leaderGateway .requestTaskManagerMetricQueryServicePaths(timeout); taskManagerQueryServicePathsFuture.whenCompleteAsync( - (Collection> queryServicePaths, Throwable throwable) -> { + (Collection> queryServicePaths, Throwable throwable) -> { if (throwable != null) { LOG.warn("Requesting TaskManager's path for query services failed.", throwable); } else { List taskManagersToRetain = queryServicePaths .stream() .map( - (Tuple2 tuple) -> { + (Tuple2 tuple) -> { retrieveAndQueryMetrics(tuple.f1); - return tuple.f0.toString(); + return tuple.f0.getResourceIdString(); } ).collect(Collectors.toList()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java index cd67705fc37cb..a95611170c731 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java @@ -55,7 +55,7 @@ import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener; import org.apache.flink.runtime.memory.MemoryManager; import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup; import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.registration.RegistrationConnectionListener; @@ -135,7 +135,7 @@ public class TaskExecutor extends RpcEndpoint implements TaskExecutorGateway { private final NetworkEnvironment networkEnvironment; /** The metric registry in the task manager */ - private final MetricRegistry metricRegistry; + private final MetricRegistryImpl metricRegistry; /** The heartbeat manager for job manager in the task manager */ private final HeartbeatManager jobManagerHeartbeatManager; @@ -179,7 +179,7 @@ public TaskExecutor( NetworkEnvironment networkEnvironment, HighAvailabilityServices haServices, HeartbeatServices heartbeatServices, - MetricRegistry metricRegistry, + MetricRegistryImpl metricRegistry, TaskManagerMetricGroup taskManagerMetricGroup, BroadcastVariableManager broadcastVariableManager, FileCache fileCache, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java index 782ab079e960f..5a69bb1c9a25c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java @@ -29,22 +29,19 @@ import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; -import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.akka.AkkaRpcService; import org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils; import org.apache.flink.runtime.security.SecurityConfiguration; import org.apache.flink.runtime.security.SecurityUtils; -import org.apache.flink.runtime.taskexecutor.utils.TaskExecutorMetricsInitializer; import org.apache.flink.runtime.util.EnvironmentInformation; import org.apache.flink.runtime.util.ExecutorThreadFactory; import org.apache.flink.runtime.util.Hardware; import org.apache.flink.runtime.util.JvmShutdownSafeguard; import org.apache.flink.runtime.util.LeaderRetrievalUtils; - import org.apache.flink.runtime.util.SignalHandler; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.Preconditions; @@ -86,7 +83,7 @@ public class TaskManagerRunner implements FatalErrorHandler { private final HighAvailabilityServices highAvailabilityServices; - private final MetricRegistry metricRegistry; + private final MetricRegistryImpl metricRegistry; /** Executor used to run future callbacks */ private final ExecutorService executor; @@ -112,7 +109,7 @@ public TaskManagerRunner(Configuration configuration, ResourceID resourceId) thr HeartbeatServices heartbeatServices = HeartbeatServices.fromConfiguration(configuration); - metricRegistry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(configuration)); + metricRegistry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(configuration)); final ActorSystem actorSystem = ((AkkaRpcService) rpcService).getActorSystem(); metricRegistry.startQueryService(actorSystem, resourceId); @@ -250,7 +247,7 @@ public static TaskExecutor startTaskManager( RpcService rpcService, HighAvailabilityServices highAvailabilityServices, HeartbeatServices heartbeatServices, - MetricRegistry metricRegistry, + MetricRegistryImpl metricRegistry, boolean localCommunicationOnly, FatalErrorHandler fatalErrorHandler) throws Exception { @@ -269,18 +266,11 @@ public static TaskExecutor startTaskManager( TaskManagerServices taskManagerServices = TaskManagerServices.fromConfiguration( taskManagerServicesConfiguration, - resourceID); + resourceID, + metricRegistry); TaskManagerConfiguration taskManagerConfiguration = TaskManagerConfiguration.fromConfiguration(configuration); - TaskManagerMetricGroup taskManagerMetricGroup = new TaskManagerMetricGroup( - metricRegistry, - taskManagerServices.getTaskManagerLocation().getHostname(), - resourceID.toString()); - - // Initialize the TM metrics - TaskExecutorMetricsInitializer.instantiateStatusMetrics(taskManagerMetricGroup, taskManagerServices.getNetworkEnvironment()); - return new TaskExecutor( rpcService, taskManagerConfiguration, @@ -291,7 +281,7 @@ public static TaskExecutor startTaskManager( highAvailabilityServices, heartbeatServices, metricRegistry, - taskManagerMetricGroup, + taskManagerServices.getTaskManagerMetricGroup(), taskManagerServices.getBroadcastVariableManager(), taskManagerServices.getFileCache(), taskManagerServices.getTaskSlotTable(), diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java index 2baf644b61a8c..85e62c6d7cb45 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java @@ -38,6 +38,7 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; import org.apache.flink.runtime.memory.MemoryManager; import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup; import org.apache.flink.runtime.query.KvStateClientProxy; import org.apache.flink.runtime.query.KvStateRegistry; @@ -62,7 +63,7 @@ /** * Container for {@link TaskExecutor} services such as the {@link MemoryManager}, {@link IOManager}, - * {@link NetworkEnvironment} and the {@link MetricRegistry}. + * {@link NetworkEnvironment} and the {@link MetricRegistryImpl}. */ public class TaskManagerServices { private static final Logger LOG = LoggerFactory.getLogger(TaskManagerServices.class); @@ -72,7 +73,6 @@ public class TaskManagerServices { private final MemoryManager memoryManager; private final IOManager ioManager; private final NetworkEnvironment networkEnvironment; - private final MetricRegistry metricRegistry; private final TaskManagerMetricGroup taskManagerMetricGroup; private final BroadcastVariableManager broadcastVariableManager; private final FileCache fileCache; @@ -85,7 +85,6 @@ private TaskManagerServices( MemoryManager memoryManager, IOManager ioManager, NetworkEnvironment networkEnvironment, - MetricRegistry metricRegistry, TaskManagerMetricGroup taskManagerMetricGroup, BroadcastVariableManager broadcastVariableManager, FileCache fileCache, @@ -97,7 +96,6 @@ private TaskManagerServices( this.memoryManager = Preconditions.checkNotNull(memoryManager); this.ioManager = Preconditions.checkNotNull(ioManager); this.networkEnvironment = Preconditions.checkNotNull(networkEnvironment); - this.metricRegistry = Preconditions.checkNotNull(metricRegistry); this.taskManagerMetricGroup = Preconditions.checkNotNull(taskManagerMetricGroup); this.broadcastVariableManager = Preconditions.checkNotNull(broadcastVariableManager); this.fileCache = Preconditions.checkNotNull(fileCache); @@ -126,10 +124,6 @@ public TaskManagerLocation getTaskManagerLocation() { return taskManagerLocation; } - public MetricRegistry getMetricRegistry() { - return metricRegistry; - } - public TaskManagerMetricGroup getTaskManagerMetricGroup() { return taskManagerMetricGroup; } @@ -163,12 +157,14 @@ public JobLeaderService getJobLeaderService() { * * @param resourceID resource ID of the task manager * @param taskManagerServicesConfiguration task manager configuration + * @param metricRegistry to register the TaskManagerMetricGroup * @return task manager components * @throws Exception */ public static TaskManagerServices fromConfiguration( TaskManagerServicesConfiguration taskManagerServicesConfiguration, - ResourceID resourceID) throws Exception { + ResourceID resourceID, + MetricRegistry metricRegistry) throws Exception { // pre-start checks checkTempDirs(taskManagerServicesConfiguration.getTmpDirPaths()); @@ -187,9 +183,6 @@ public static TaskManagerServices fromConfiguration( // start the I/O manager, it will create some temp directories. final IOManager ioManager = new IOManagerAsync(taskManagerServicesConfiguration.getTmpDirPaths()); - final MetricRegistry metricRegistry = new MetricRegistry( - taskManagerServicesConfiguration.getMetricRegistryConfiguration()); - final TaskManagerMetricGroup taskManagerMetricGroup = new TaskManagerMetricGroup( metricRegistry, taskManagerLocation.getHostname(), @@ -223,7 +216,6 @@ public static TaskManagerServices fromConfiguration( memoryManager, ioManager, network, - metricRegistry, taskManagerMetricGroup, broadcastVariableManager, fileCache, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java index bfd37bc8e9bf3..990fb2259829e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java @@ -28,7 +28,6 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.network.netty.NettyConfig; import org.apache.flink.runtime.memory.MemoryManager; -import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration; import org.apache.flink.util.MathUtils; import org.apache.flink.util.NetUtils; @@ -72,8 +71,6 @@ public class TaskManagerServicesConfiguration { private final float memoryFraction; - private final MetricRegistryConfiguration metricRegistryConfiguration; - private final long timerServiceShutdownTimeout; public TaskManagerServicesConfiguration( @@ -85,7 +82,6 @@ public TaskManagerServicesConfiguration( long configuredMemory, boolean preAllocateMemory, float memoryFraction, - MetricRegistryConfiguration metricRegistryConfiguration, long timerServiceShutdownTimeout) { this.taskManagerAddress = checkNotNull(taskManagerAddress); @@ -98,8 +94,6 @@ public TaskManagerServicesConfiguration( this.preAllocateMemory = preAllocateMemory; this.memoryFraction = memoryFraction; - this.metricRegistryConfiguration = checkNotNull(metricRegistryConfiguration); - checkArgument(timerServiceShutdownTimeout >= 0L, "The timer " + "service shutdown timeout must be greater or equal to 0."); this.timerServiceShutdownTimeout = timerServiceShutdownTimeout; @@ -148,10 +142,6 @@ public boolean isPreAllocateMemory() { return preAllocateMemory; } - public MetricRegistryConfiguration getMetricRegistryConfiguration() { - return metricRegistryConfiguration; - } - public long getTimerServiceShutdownTimeout() { return timerServiceShutdownTimeout; } @@ -211,8 +201,6 @@ public static TaskManagerServicesConfiguration fromConfiguration( TaskManagerOptions.MANAGED_MEMORY_FRACTION.key(), "MemoryManager fraction of the free memory must be between 0.0 and 1.0"); - final MetricRegistryConfiguration metricRegistryConfiguration = MetricRegistryConfiguration.fromConfiguration(configuration); - long timerServiceShutdownTimeout = AkkaUtils.getTimeout(configuration).toMillis(); return new TaskManagerServicesConfiguration( @@ -224,7 +212,6 @@ public static TaskManagerServicesConfiguration fromConfiguration( configuredMemory, preAllocateMemory, memoryFraction, - metricRegistryConfiguration, timerServiceShutdownTimeout); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/RestfulGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/RestfulGateway.java index d871b067d0d0f..331e96b0b3c99 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/RestfulGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/RestfulGateway.java @@ -21,8 +21,8 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; -import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.messages.FlinkJobNotFoundException; import org.apache.flink.runtime.messages.webmonitor.ClusterOverview; import org.apache.flink.runtime.messages.webmonitor.MultipleJobsDetails; @@ -95,5 +95,5 @@ CompletableFuture requestJobDetails( * @param timeout for the asynchronous operation * @return Future containing the collection of instance ids and the corresponding metric query service path */ - CompletableFuture>> requestTaskManagerMetricQueryServicePaths(@RpcTimeout Time timeout); + CompletableFuture>> requestTaskManagerMetricQueryServicePaths(@RpcTimeout Time timeout); } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/clusterframework/ContaineredJobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/clusterframework/ContaineredJobManager.scala index 74ef1de33c65c..1c573c098385f 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/clusterframework/ContaineredJobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/clusterframework/ContaineredJobManager.scala @@ -35,7 +35,8 @@ import org.apache.flink.runtime.jobmanager.{JobManager, SubmittedJobGraphStore} import org.apache.flink.runtime.leaderelection.LeaderElectionService import org.apache.flink.runtime.messages.Acknowledge import org.apache.flink.runtime.messages.JobManagerMessages.{CurrentJobStatus, JobNotFound, RequestJobStatus} -import org.apache.flink.runtime.metrics.{MetricRegistry => FlinkMetricRegistry} +import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup +import org.apache.flink.runtime.metrics.{MetricRegistryImpl => FlinkMetricRegistry} import scala.concurrent.duration._ import scala.language.postfixOps @@ -74,7 +75,7 @@ abstract class ContaineredJobManager( submittedJobGraphs : SubmittedJobGraphStore, checkpointRecoveryFactory : CheckpointRecoveryFactory, jobRecoveryTimeout: FiniteDuration, - metricsRegistry: Option[FlinkMetricRegistry], + jobManagerMetricGroup: JobManagerMetricGroup, optRestAddress: Option[String]) extends JobManager( flinkConfiguration, @@ -91,7 +92,7 @@ abstract class ContaineredJobManager( submittedJobGraphs, checkpointRecoveryFactory, jobRecoveryTimeout, - metricsRegistry, + jobManagerMetricGroup, optRestAddress) { val jobPollingInterval: FiniteDuration diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala index 043504646d622..d40a0fd86e6b1 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala @@ -34,7 +34,6 @@ import org.apache.flink.api.common.time.Time import org.apache.flink.configuration._ import org.apache.flink.core.fs.{FileSystem, Path} import org.apache.flink.core.io.InputSplitAssigner -import org.apache.flink.metrics.groups.UnregisteredMetricsGroup import org.apache.flink.metrics.{Gauge, MetricGroup} import org.apache.flink.runtime.accumulators.AccumulatorSnapshot import org.apache.flink.runtime.akka.{AkkaUtils, ListeningBehaviour} @@ -50,10 +49,10 @@ import org.apache.flink.runtime.concurrent.{FutureUtils, ScheduledExecutorServic import org.apache.flink.runtime.execution.SuppressRestartsException import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders.ResolveOrder import org.apache.flink.runtime.execution.librarycache.{BlobLibraryCacheManager, LibraryCacheManager} -import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory import org.apache.flink.runtime.executiongraph._ -import org.apache.flink.runtime.highavailability.{HighAvailabilityServices, HighAvailabilityServicesUtils} +import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils.AddressResolution +import org.apache.flink.runtime.highavailability.{HighAvailabilityServices, HighAvailabilityServicesUtils} import org.apache.flink.runtime.instance.{AkkaActorGateway, InstanceID, InstanceManager} import org.apache.flink.runtime.jobgraph.{JobGraph, JobStatus} import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore.SubmittedJobGraphListener @@ -66,20 +65,18 @@ import org.apache.flink.runtime.messages.ExecutionGraphMessages.JobStatusChanged import org.apache.flink.runtime.messages.JobManagerMessages._ import org.apache.flink.runtime.messages.Messages.Disconnect import org.apache.flink.runtime.messages.RegistrationMessages._ -import org.apache.flink.runtime.messages.{Acknowledge, StackTrace} import org.apache.flink.runtime.messages.TaskManagerMessages.Heartbeat import org.apache.flink.runtime.messages.TaskMessages.UpdateTaskExecutionState import org.apache.flink.runtime.messages.accumulators._ import org.apache.flink.runtime.messages.checkpoint.{AbstractCheckpointMessage, AcknowledgeCheckpoint, DeclineCheckpoint} import org.apache.flink.runtime.messages.webmonitor.{InfoMessage, _} +import org.apache.flink.runtime.messages.{Acknowledge, StackTrace} import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup -import org.apache.flink.runtime.metrics.{MetricRegistryConfiguration, MetricRegistry => FlinkMetricRegistry} import org.apache.flink.runtime.metrics.util.MetricUtils -import org.apache.flink.runtime.net.SSLUtils +import org.apache.flink.runtime.metrics.{MetricRegistryConfiguration, MetricRegistryImpl, MetricRegistry => FlinkMetricRegistry} import org.apache.flink.runtime.process.ProcessReaper import org.apache.flink.runtime.query.KvStateMessage.{LookupKvStateLocation, NotifyKvStateRegistered, NotifyKvStateUnregistered} import org.apache.flink.runtime.query.{KvStateMessage, UnknownKvStateLocation} -import org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils import org.apache.flink.runtime.security.{SecurityConfiguration, SecurityUtils} import org.apache.flink.runtime.taskexecutor.TaskExecutor import org.apache.flink.runtime.taskmanager.TaskManager @@ -137,7 +134,7 @@ class JobManager( protected val submittedJobGraphs : SubmittedJobGraphStore, protected val checkpointRecoveryFactory : CheckpointRecoveryFactory, protected val jobRecoveryTimeout: FiniteDuration, - protected val metricsRegistry: Option[FlinkMetricRegistry], + protected val jobManagerMetricGroup: JobManagerMetricGroup, protected val optRestAddress: Option[String]) extends FlinkActor with LeaderSessionMessageFilter // mixin oder is important, we want filtering after logging @@ -154,16 +151,6 @@ class JobManager( var leaderSessionID: Option[UUID] = None - protected val jobManagerMetricGroup : Option[JobManagerMetricGroup] = metricsRegistry match { - case Some(registry) => - val host = flinkConfiguration.getString(JobManagerOptions.ADDRESS) - Option(new JobManagerMetricGroup( - registry, NetUtils.unresolvedHostToNormalizedString(host))) - case None => - log.warn("Could not instantiate JobManager metrics.") - None - } - /** Futures which have to be completed before terminating the job manager */ var futuresToComplete: Option[Seq[Future[Unit]]] = None @@ -205,12 +192,7 @@ class JobManager( throw new RuntimeException("Could not start the submitted job graphs service.", e) } - jobManagerMetricGroup match { - case Some(group) => - instantiateMetrics(group) - case None => - log.warn("Could not instantiate JobManager metric group.") - } + instantiateMetrics(jobManagerMetricGroup) } override def postStop(): Unit = { @@ -250,6 +232,8 @@ class JobManager( archive ! decorateMessage(PoisonPill) } + jobManagerMetricGroup.close() + instanceManager.shutdown() scheduler.shutdown() libraryCacheManager.shutdown() @@ -260,13 +244,6 @@ class JobManager( case e: IOException => log.error("Could not properly shutdown the blob server.", e) } - // failsafe shutdown of the metrics registry - try { - metricsRegistry.foreach(_.shutdown()) - } catch { - case t: Exception => log.error("MetricRegistry did not shutdown properly.", t) - } - log.debug(s"Job manager ${self.path} is completely stopped.") } @@ -1073,9 +1050,9 @@ class JobManager( ) ) - case RequestTaskManagerInstance(instanceID) => + case RequestTaskManagerInstance(resourceId) => sender ! decorateMessage( - TaskManagerInstance(Option(instanceManager.getRegisteredInstanceById(instanceID))) + TaskManagerInstance(Option(instanceManager.getRegisteredInstance(resourceId))) ) case Heartbeat(instanceID, accumulators) => @@ -1283,15 +1260,7 @@ class JobManager( log.info(s"Using restart strategy $restartStrategy for $jobId.") - val jobMetrics = jobManagerMetricGroup match { - case Some(group) => - group.addJob(jobGraph) match { - case (jobGroup:Any) => jobGroup - case null => new UnregisteredMetricsGroup() - } - case None => - new UnregisteredMetricsGroup() - } + val jobMetrics = jobManagerMetricGroup.addJob(jobGraph) val numSlots = scheduler.getTotalNumberOfSlots() @@ -1791,7 +1760,7 @@ class JobManager( libraryCacheManager.unregisterJob(jobID) blobServer.cleanupJob(jobID) - jobManagerMetricGroup.foreach(_.removeJob(jobID)) + jobManagerMetricGroup.removeJob(jobID) futureOption } @@ -2042,7 +2011,12 @@ object JobManager { val highAvailabilityServices = HighAvailabilityServicesUtils.createHighAvailabilityServices( configuration, ioExecutor, - AddressResolution.NO_ADDRESS_RESOLUTION); + AddressResolution.NO_ADDRESS_RESOLUTION) + + val metricRegistry = new MetricRegistryImpl( + MetricRegistryConfiguration.fromConfiguration(configuration)) + + metricRegistry.startQueryService(jobManagerSystem, null) val (_, _, webMonitorOption, _) = try { startJobManagerActors( @@ -2053,6 +2027,7 @@ object JobManager { futureExecutor, ioExecutor, highAvailabilityServices, + metricRegistry, classOf[JobManager], classOf[MemoryArchivist], Option(classOf[StandaloneResourceManager]) @@ -2085,6 +2060,13 @@ object JobManager { LOG.warn("Could not properly stop the high availability services.", t) } + try { + metricRegistry.shutdown() + } catch { + case t: Throwable => + LOG.warn("Could not properly shut down the metric registry.", t) + } + FlinkExecutors.gracefulShutdown( timeout.toMillis, TimeUnit.MILLISECONDS, @@ -2191,6 +2173,7 @@ object JobManager { futureExecutor: ScheduledExecutorService, ioExecutor: Executor, highAvailabilityServices: HighAvailabilityServices, + metricRegistry: FlinkMetricRegistry, jobManagerClass: Class[_ <: JobManager], archiveClass: Class[_ <: MemoryArchivist], resourceManagerClass: Option[Class[_ <: FlinkResourceManager[_]]]) @@ -2231,6 +2214,7 @@ object JobManager { futureExecutor, ioExecutor, highAvailabilityServices, + metricRegistry, webMonitor.map(_.getRestAddress), jobManagerClass, archiveClass) @@ -2250,11 +2234,14 @@ object JobManager { if (executionMode == JobManagerMode.LOCAL) { LOG.info("Starting embedded TaskManager for JobManager's LOCAL execution mode") + val resourceId = ResourceID.generate() + val taskManagerActor = TaskManager.startTaskManagerComponentsAndActor( configuration, - ResourceID.generate(), + resourceId, jobManagerSystem, highAvailabilityServices, + metricRegistry, externalHostname, Some(TaskExecutor.TASK_MANAGER_NAME), localTaskManagerCommunication = true, @@ -2433,7 +2420,8 @@ object JobManager { configuration: Configuration, futureExecutor: ScheduledExecutorService, ioExecutor: Executor, - blobStore: BlobStore) : + blobStore: BlobStore, + metricRegistry: FlinkMetricRegistry) : (InstanceManager, FlinkScheduler, BlobServer, @@ -2443,7 +2431,7 @@ object JobManager { Int, // number of archived jobs Option[Path], // archive path FiniteDuration, // timeout for job recovery - Option[FlinkMetricRegistry] + JobManagerMetricGroup ) = { val timeout: FiniteDuration = AkkaUtils.getTimeout(configuration) @@ -2525,12 +2513,9 @@ object JobManager { } } - val metricRegistry = try { - Option(new FlinkMetricRegistry(MetricRegistryConfiguration.fromConfiguration(configuration))) - } catch { - case _: Exception => - None - } + val jobManagerMetricGroup = new JobManagerMetricGroup( + metricRegistry, + configuration.getString(JobManagerOptions.ADDRESS)) (instanceManager, scheduler, @@ -2541,7 +2526,7 @@ object JobManager { archiveCount, archivePath, jobRecoveryTimeout, - metricRegistry) + jobManagerMetricGroup) } /** @@ -2564,6 +2549,7 @@ object JobManager { futureExecutor: ScheduledExecutorService, ioExecutor: Executor, highAvailabilityServices: HighAvailabilityServices, + metricRegistry: FlinkMetricRegistry, optRestAddress: Option[String], jobManagerClass: Class[_ <: JobManager], archiveClass: Class[_ <: MemoryArchivist]) @@ -2575,6 +2561,7 @@ object JobManager { futureExecutor, ioExecutor, highAvailabilityServices, + metricRegistry, optRestAddress, Some(JobMaster.JOB_MANAGER_NAME), Some(JobMaster.ARCHIVE_NAME), @@ -2606,6 +2593,7 @@ object JobManager { futureExecutor: ScheduledExecutorService, ioExecutor: Executor, highAvailabilityServices: HighAvailabilityServices, + metricRegistry: FlinkMetricRegistry, optRestAddress: Option[String], jobManagerActorName: Option[String], archiveActorName: Option[String], @@ -2622,11 +2610,12 @@ object JobManager { archiveCount, archivePath, jobRecoveryTimeout, - metricsRegistry) = createJobManagerComponents( + jobManagerMetricGroup) = createJobManagerComponents( configuration, futureExecutor, ioExecutor, - highAvailabilityServices.createBlobStore()) + highAvailabilityServices.createBlobStore(), + metricRegistry) val archiveProps = getArchiveProps(archiveClass, archiveCount, archivePath) @@ -2653,7 +2642,7 @@ object JobManager { highAvailabilityServices.getSubmittedJobGraphStore(), highAvailabilityServices.getCheckpointRecoveryFactory(), jobRecoveryTimeout, - metricsRegistry, + jobManagerMetricGroup, optRestAddress) val jobManager: ActorRef = jobManagerActorName match { @@ -2661,12 +2650,6 @@ object JobManager { case None => actorSystem.actorOf(jobManagerProps) } - metricsRegistry match { - case Some(registry) => - registry.startQueryService(actorSystem, null) - case None => - } - (jobManager, archive) } @@ -2693,7 +2676,7 @@ object JobManager { submittedJobGraphStore: SubmittedJobGraphStore, checkpointRecoveryFactory: CheckpointRecoveryFactory, jobRecoveryTimeout: FiniteDuration, - metricsRegistry: Option[FlinkMetricRegistry], + jobManagerMetricGroup: JobManagerMetricGroup, optRestAddress: Option[String]): Props = { Props( @@ -2712,7 +2695,7 @@ object JobManager { submittedJobGraphStore, checkpointRecoveryFactory, jobRecoveryTimeout, - metricsRegistry, + jobManagerMetricGroup, optRestAddress) } } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala index 3e896cab99a27..5c19c7aa1b587 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala @@ -24,8 +24,9 @@ import java.util.UUID import akka.actor.ActorRef import org.apache.flink.api.common.JobID import org.apache.flink.runtime.akka.ListeningBehaviour -import org.apache.flink.runtime.blob.{PermanentBlobKey} +import org.apache.flink.runtime.blob.PermanentBlobKey import org.apache.flink.runtime.client.{JobStatusMessage, SerializedJobExecutionResult} +import org.apache.flink.runtime.clusterframework.types.ResourceID import org.apache.flink.runtime.executiongraph.{AccessExecutionGraph, ExecutionAttemptID, ExecutionGraph} import org.apache.flink.runtime.instance.{Instance, InstanceID} import org.apache.flink.runtime.io.network.partition.ResultPartitionID @@ -419,9 +420,9 @@ object JobManagerMessages { /** * Requests the [[Instance]] object of the task manager with the given instance ID * - * @param instanceID Instance ID of the task manager + * @param resourceId identifying the TaskManager which shall be retrieved */ - case class RequestTaskManagerInstance(instanceID: InstanceID) + case class RequestTaskManagerInstance(resourceId: ResourceID) /** * Returns the [[Instance]] object of the requested task manager. This is in response to diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala index c152f4abc01ed..689d98f6b4470 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala @@ -40,6 +40,7 @@ import org.apache.flink.runtime.jobgraph.JobGraph import org.apache.flink.runtime.jobmanager.HighAvailabilityMode import org.apache.flink.runtime.leaderretrieval.{LeaderRetrievalListener, LeaderRetrievalService} import org.apache.flink.runtime.messages.TaskManagerMessages.NotifyWhenRegisteredAtJobManager +import org.apache.flink.runtime.metrics.{MetricRegistryConfiguration, MetricRegistryImpl} import org.apache.flink.runtime.util.{ExecutorThreadFactory, Hardware} import org.apache.flink.runtime.webmonitor.retriever.impl.{AkkaJobManagerRetriever, AkkaQueryServiceRetriever} import org.apache.flink.runtime.webmonitor.{WebMonitor, WebMonitorUtils} @@ -121,6 +122,9 @@ abstract class FlinkMiniCluster( Hardware.getNumberCPUCores(), new ExecutorThreadFactory("mini-cluster-io")) + protected val metricRegistry = new MetricRegistryImpl( + MetricRegistryConfiguration.fromConfiguration(originalConfiguration)) + def this(configuration: Configuration, useSingleActorSystem: Boolean) { this( configuration, @@ -325,6 +329,10 @@ abstract class FlinkMiniCluster( lazy val singleActorSystem = startJobManagerActorSystem(0) + if (originalConfiguration.getBoolean(ConfigConstants.LOCAL_START_WEBSERVER, false)) { + metricRegistry.startQueryService(singleActorSystem, null) + } + val (jmActorSystems, jmActors) = (for(i <- 0 until numJobManagers) yield { val actorSystem = if(useSingleActorSystem) { diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala index e22230e37f1d9..e9bdb2a07eb0d 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala @@ -19,6 +19,7 @@ package org.apache.flink.runtime.minicluster import java.net.InetAddress +import java.util.UUID import java.util.concurrent.{Executor, ScheduledExecutorService} import akka.actor.{ActorRef, ActorSystem, Props} @@ -46,7 +47,8 @@ import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService import org.apache.flink.runtime.memory.MemoryManager import org.apache.flink.runtime.messages.JobManagerMessages import org.apache.flink.runtime.messages.JobManagerMessages.{RunningJobsStatus, StoppingFailure, StoppingResponse} -import org.apache.flink.runtime.metrics.MetricRegistry +import org.apache.flink.runtime.metrics.groups.{JobManagerMetricGroup, TaskManagerMetricGroup} +import org.apache.flink.runtime.metrics.{MetricRegistryConfiguration, MetricRegistryImpl} import org.apache.flink.runtime.taskexecutor.{TaskExecutor, TaskManagerConfiguration, TaskManagerServices, TaskManagerServicesConfiguration} import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerLocation} import org.apache.flink.runtime.util.EnvironmentInformation @@ -83,6 +85,12 @@ class LocalFlinkMiniCluster( def this(userConfiguration: Configuration) = this(userConfiguration, true) + override def startInternalShutdown() { + metricRegistry.shutdown() + + super.startInternalShutdown() + } + // -------------------------------------------------------------------------- override def generateConfiguration(userConfiguration: Configuration): Configuration = { @@ -137,23 +145,20 @@ class LocalFlinkMiniCluster( } val (instanceManager, - scheduler, - blobServer, - libraryCacheManager, - restartStrategyFactory, - timeout, - archiveCount, - archivePath, - jobRecoveryTimeout, - metricsRegistry) = JobManager.createJobManagerComponents( + scheduler, + blobServer, + libraryCacheManager, + restartStrategyFactory, + timeout, + archiveCount, + archivePath, + jobRecoveryTimeout, + jobManagerMetricGroup) = JobManager.createJobManagerComponents( config, futureExecutor, ioExecutor, - highAvailabilityServices.createBlobStore()) - - if (config.getBoolean(ConfigConstants.LOCAL_START_WEBSERVER, false)) { - metricsRegistry.get.startQueryService(system, null) - } + highAvailabilityServices.createBlobStore(), + metricRegistry) val archive = system.actorOf( getArchiveProps( @@ -180,7 +185,7 @@ class LocalFlinkMiniCluster( highAvailabilityServices.getSubmittedJobGraphStore(), highAvailabilityServices.getCheckpointRecoveryFactory(), jobRecoveryTimeout, - metricsRegistry, + jobManagerMetricGroup, optRestAddress), jobManagerName) } @@ -248,9 +253,8 @@ class LocalFlinkMiniCluster( val taskManagerServices = TaskManagerServices.fromConfiguration( taskManagerServicesConfiguration, - resourceID) - - val metricRegistry = taskManagerServices.getMetricRegistry() + resourceID, + metricRegistry) val props = getTaskManagerProps( taskManagerClass, @@ -260,7 +264,7 @@ class LocalFlinkMiniCluster( taskManagerServices.getMemoryManager(), taskManagerServices.getIOManager(), taskManagerServices.getNetworkEnvironment, - metricRegistry) + taskManagerServices.getTaskManagerMetricGroup) if (config.getBoolean(ConfigConstants.LOCAL_START_WEBSERVER, false)) { metricRegistry.startQueryService(system, resourceID) @@ -296,7 +300,7 @@ class LocalFlinkMiniCluster( submittedJobGraphStore: SubmittedJobGraphStore, checkpointRecoveryFactory: CheckpointRecoveryFactory, jobRecoveryTimeout: FiniteDuration, - metricsRegistry: Option[MetricRegistry], + jobManagerMetricGroup: JobManagerMetricGroup, optRestAddress: Option[String]) : Props = { @@ -316,7 +320,7 @@ class LocalFlinkMiniCluster( submittedJobGraphStore, checkpointRecoveryFactory, jobRecoveryTimeout, - metricsRegistry, + jobManagerMetricGroup, optRestAddress) } @@ -328,7 +332,7 @@ class LocalFlinkMiniCluster( memoryManager: MemoryManager, ioManager: IOManager, networkEnvironment: NetworkEnvironment, - metricsRegistry: MetricRegistry): Props = { + taskManagerMetricGroup: TaskManagerMetricGroup): Props = { TaskManager.getTaskManagerProps( taskManagerClass, @@ -339,7 +343,7 @@ class LocalFlinkMiniCluster( ioManager, networkEnvironment, highAvailabilityServices, - metricsRegistry) + taskManagerMetricGroup) } def getResourceManagerProps( diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala index cc01a8dc1585b..f209dacf429f5 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala @@ -22,7 +22,7 @@ import java.io.{File, FileInputStream, IOException} import java.lang.management.ManagementFactory import java.net.{BindException, InetAddress, InetSocketAddress, ServerSocket} import java.util -import java.util.concurrent.{Callable, TimeUnit} +import java.util.concurrent.{Callable, TimeUnit, TimeoutException} import java.util.{Collections, UUID} import _root_.akka.actor._ @@ -63,8 +63,7 @@ import org.apache.flink.runtime.messages.TaskMessages._ import org.apache.flink.runtime.messages.checkpoint.{AbstractCheckpointMessage, NotifyCheckpointComplete, TriggerCheckpoint} import org.apache.flink.runtime.messages.{Acknowledge, StackTraceSampleResponse} import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup -import org.apache.flink.runtime.metrics.util.MetricUtils -import org.apache.flink.runtime.metrics.{MetricRegistry => FlinkMetricRegistry} +import org.apache.flink.runtime.metrics.{MetricRegistryConfiguration, MetricRegistryImpl, MetricRegistry => FlinkMetricRegistry} import org.apache.flink.runtime.process.ProcessReaper import org.apache.flink.runtime.security.{SecurityConfiguration, SecurityUtils} import org.apache.flink.runtime.taskexecutor.{TaskExecutor, TaskManagerConfiguration, TaskManagerServices, TaskManagerServicesConfiguration} @@ -127,7 +126,7 @@ class TaskManager( protected val network: NetworkEnvironment, protected val numberOfSlots: Int, protected val highAvailabilityServices: HighAvailabilityServices, - protected val metricsRegistry: FlinkMetricRegistry) + protected val taskManagerMetricGroup: TaskManagerMetricGroup) extends FlinkActor with LeaderSessionMessageFilter // Mixin order is important: We want to filter after logging with LogMessages // Mixin order is important: first we want to support message logging @@ -154,8 +153,6 @@ class TaskManager( getJobManagerLeaderRetriever( HighAvailabilityServices.DEFAULT_JOB_ID) - private var taskManagerMetricGroup : TaskManagerMetricGroup = _ - /** Actors which want to be notified once this task manager has been * registered at the job manager */ private val waitForRegistration = scala.collection.mutable.Set[ActorRef]() @@ -258,13 +255,8 @@ class TaskManager( } catch { case t: Exception => log.error("FileCache did not shutdown properly.", t) } - - // failsafe shutdown of the metrics registry - try { - metricsRegistry.shutdown() - } catch { - case t: Exception => log.error("MetricRegistry did not shutdown properly.", t) - } + + taskManagerMetricGroup.close() log.info(s"Task manager ${self.path} is completely shut down.") } @@ -980,12 +972,6 @@ class TaskManager( throw new RuntimeException(message, e) } - taskManagerMetricGroup = - new TaskManagerMetricGroup(metricsRegistry, location.getHostname, id.toString) - - MetricUtils.instantiateStatusMetrics(taskManagerMetricGroup) - MetricUtils.instantiateNetworkMetrics(taskManagerMetricGroup, network) - // watch job manager to detect when it dies context.watch(jobManager) @@ -1832,15 +1818,22 @@ object TaskManager { actorSystemPort, LOG.logger) + val metricRegistry = new MetricRegistryImpl( + MetricRegistryConfiguration.fromConfiguration(configuration)) + + metricRegistry.startQueryService(taskManagerSystem, resourceID) + // start all the TaskManager services (network stack, library cache, ...) // and the TaskManager actor try { + LOG.info("Starting TaskManager actor") val taskManager = startTaskManagerComponentsAndActor( configuration, resourceID, taskManagerSystem, highAvailabilityServices, + metricRegistry, taskManagerHostname, Some(TaskExecutor.TASK_MANAGER_NAME), localTaskManagerCommunication = false, @@ -1893,6 +1886,9 @@ object TaskManager { } throw t } + + // shut down the metric query service + metricRegistry.shutdown() } /** @@ -1984,6 +1980,7 @@ object TaskManager { resourceID: ResourceID, actorSystem: ActorSystem, highAvailabilityServices: HighAvailabilityServices, + metricRegistry: FlinkMetricRegistry, taskManagerHostname: String, taskManagerActorName: Option[String], localTaskManagerCommunication: Boolean, @@ -1999,9 +1996,8 @@ object TaskManager { val taskManagerServices = TaskManagerServices.fromConfiguration( taskManagerServicesConfiguration, - resourceID) - - val metricRegistry = taskManagerServices.getMetricRegistry() + resourceID, + metricRegistry) // create the actor properties (which define the actor constructor parameters) val tmProps = getTaskManagerProps( @@ -2013,9 +2009,7 @@ object TaskManager { taskManagerServices.getIOManager(), taskManagerServices.getNetworkEnvironment(), highAvailabilityServices, - metricRegistry) - - metricRegistry.startQueryService(actorSystem, resourceID) + taskManagerServices.getTaskManagerMetricGroup) taskManagerActorName match { case Some(actorName) => actorSystem.actorOf(tmProps, actorName) @@ -2032,7 +2026,7 @@ object TaskManager { ioManager: IOManager, networkEnvironment: NetworkEnvironment, highAvailabilityServices: HighAvailabilityServices, - metricsRegistry: FlinkMetricRegistry + taskManagerMetricGroup: TaskManagerMetricGroup ): Props = { Props( taskManagerClass, @@ -2044,7 +2038,7 @@ object TaskManager { networkEnvironment, taskManagerConfig.getNumberSlots(), highAvailabilityServices, - metricsRegistry) + taskManagerMetricGroup) } // -------------------------------------------------------------------------- diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java index 1a3ca70ae08e5..d8e65a6c8ac50 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java @@ -45,7 +45,7 @@ import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.messages.JobManagerMessages; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.registration.RegistrationResponse; import org.apache.flink.runtime.resourcemanager.JobLeaderIdService; import org.apache.flink.runtime.resourcemanager.ResourceManagerConfiguration; @@ -502,7 +502,7 @@ public void testHeartbeatTimeoutWithTaskExecutor() throws Exception { final ScheduledExecutor scheduledExecutor = mock(ScheduledExecutor.class); final HeartbeatServices heartbeatServices = new TestingHeartbeatServices(heartbeatInterval, heartbeatTimeout, scheduledExecutor); - final MetricRegistry metricRegistry = mock(MetricRegistry.class); + final MetricRegistryImpl metricRegistry = mock(MetricRegistryImpl.class); final JobLeaderIdService jobLeaderIdService = mock(JobLeaderIdService.class); final TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler(); final SlotManager slotManager = new SlotManager( @@ -601,7 +601,7 @@ public void testHeartbeatTimeoutWithJobManager() throws Exception { final ScheduledExecutor scheduledExecutor = mock(ScheduledExecutor.class); final HeartbeatServices heartbeatServices = new TestingHeartbeatServices(heartbeatInterval, heartbeatTimeout, scheduledExecutor); - final MetricRegistry metricRegistry = mock(MetricRegistry.class); + final MetricRegistryImpl metricRegistry = mock(MetricRegistryImpl.class); final JobLeaderIdService jobLeaderIdService = new JobLeaderIdService( highAvailabilityServices, rpcService.getScheduledExecutor(), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java index d1ca75777baee..8558145770472 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java @@ -34,7 +34,7 @@ import org.apache.flink.runtime.jobmaster.JobManagerServices; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; @@ -114,7 +114,7 @@ public void testJobSubmission() throws Exception { mock(ResourceManagerGateway.class), mock(BlobServer.class), heartbeatServices, - mock(MetricRegistry.class), + mock(MetricRegistryImpl.class), fatalErrorHandler, jobManagerRunner, jobId); @@ -174,7 +174,7 @@ public void confirmLeaderSessionID(UUID leaderSessionId) { mock(ResourceManagerGateway.class), mock(BlobServer.class), heartbeatServices, - mock(MetricRegistry.class), + mock(MetricRegistryImpl.class), fatalErrorHandler, mock(JobManagerRunner.class), jobId); @@ -209,7 +209,7 @@ protected TestingDispatcher( ResourceManagerGateway resourceManagerGateway, BlobServer blobServer, HeartbeatServices heartbeatServices, - MetricRegistry metricRegistry, + MetricRegistryImpl metricRegistry, FatalErrorHandler fatalErrorHandler, JobManagerRunner jobManagerRunner, JobID expectedJobId) throws Exception { @@ -238,7 +238,7 @@ protected JobManagerRunner createJobManagerRunner( HighAvailabilityServices highAvailabilityServices, HeartbeatServices heartbeatServices, JobManagerServices jobManagerServices, - MetricRegistry metricRegistry, + MetricRegistryImpl metricRegistry, OnCompletionActions onCompleteActions, FatalErrorHandler fatalErrorHandler) throws Exception { assertEquals(expectedJobId, jobGraph.getJobID()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java index 7df26fc299660..d843da2785e05 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java @@ -62,7 +62,9 @@ import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService; import org.apache.flink.runtime.messages.JobManagerMessages; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; +import org.apache.flink.runtime.metrics.NoOpMetricRegistry; +import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; import org.apache.flink.runtime.state.OperatorStateHandle; import org.apache.flink.runtime.state.memory.ByteStreamStateHandle; import org.apache.flink.runtime.taskmanager.TaskManager; @@ -206,7 +208,7 @@ public void testJobRecoveryWhenLosingLeadership() throws Exception { mySubmittedJobGraphStore, checkpointStateFactory, jobRecoveryTimeout, - Option.empty(), + Option.empty(), Option.empty()); jobManager = system.actorOf(jobManagerProps); @@ -217,6 +219,7 @@ public void testJobRecoveryWhenLosingLeadership() throws Exception { ResourceID.generate(), system, testingHighAvailabilityServices, + new NoOpMetricRegistry(), "localhost", Option.apply("taskmanager"), true, @@ -380,7 +383,7 @@ public void testFailingJobRecovery() throws Exception { submittedJobGraphStore, mock(CheckpointRecoveryFactory.class), jobRecoveryTimeout, - Option.apply(null), + Option.apply(null), recoveredJobs).withDispatcher(CallingThreadDispatcher.Id()); jobManager = system.actorOf(jobManagerProps); @@ -418,7 +421,7 @@ public TestingFailingHAJobManager( SubmittedJobGraphStore submittedJobGraphs, CheckpointRecoveryFactory checkpointRecoveryFactory, FiniteDuration jobRecoveryTimeout, - Option metricsRegistry, + JobManagerMetricGroup jobManagerMetricGroup, Collection recoveredJobs) { super( flinkConfiguration, @@ -435,7 +438,7 @@ public TestingFailingHAJobManager( submittedJobGraphs, checkpointRecoveryFactory, jobRecoveryTimeout, - metricsRegistry, + jobManagerMetricGroup, Option.empty()); this.recoveredJobs = recoveredJobs; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java index bd7f11fd31eb7..a697aae2a693a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java @@ -75,6 +75,7 @@ import org.apache.flink.runtime.messages.JobManagerMessages.TriggerSavepoint; import org.apache.flink.runtime.messages.JobManagerMessages.TriggerSavepointSuccess; import org.apache.flink.runtime.messages.RegistrationMessages; +import org.apache.flink.runtime.metrics.NoOpMetricRegistry; import org.apache.flink.runtime.query.KvStateLocation; import org.apache.flink.runtime.query.KvStateMessage.LookupKvStateLocation; import org.apache.flink.runtime.query.KvStateMessage.NotifyKvStateRegistered; @@ -624,6 +625,7 @@ public void testKvStateMessages() throws Exception { TestingUtils.defaultExecutor(), TestingUtils.defaultExecutor(), highAvailabilityServices, + new NoOpMetricRegistry(), Option.empty(), TestingJobManager.class, MemoryArchivist.class)._1(); @@ -645,6 +647,7 @@ public void testKvStateMessages() throws Exception { ResourceID.generate(), system, highAvailabilityServices, + new NoOpMetricRegistry(), "localhost", scala.Option.empty(), true, @@ -841,6 +844,7 @@ public void testCancelWithSavepoint() throws Exception { TestingUtils.defaultExecutor(), TestingUtils.defaultExecutor(), highAvailabilityServices, + new NoOpMetricRegistry(), Option.empty(), Option.apply("jm"), Option.apply("arch"), @@ -859,6 +863,7 @@ public void testCancelWithSavepoint() throws Exception { ResourceID.generate(), actorSystem, highAvailabilityServices, + new NoOpMetricRegistry(), "localhost", Option.apply("tm"), true, @@ -1051,6 +1056,7 @@ public void testCancelWithSavepointNoDirectoriesConfigured() throws Exception { TestingUtils.defaultExecutor(), TestingUtils.defaultExecutor(), highAvailabilityServices, + new NoOpMetricRegistry(), Option.empty(), Option.apply("jm"), Option.apply("arch"), @@ -1069,6 +1075,7 @@ public void testCancelWithSavepointNoDirectoriesConfigured() throws Exception { ResourceID.generate(), actorSystem, highAvailabilityServices, + new NoOpMetricRegistry(), "localhost", Option.apply("tm"), true, @@ -1164,6 +1171,7 @@ public void testSavepointWithDeactivatedPeriodicCheckpointing() throws Exception TestingUtils.defaultExecutor(), TestingUtils.defaultExecutor(), highAvailabilityServices, + new NoOpMetricRegistry(), Option.empty(), Option.apply("jm"), Option.apply("arch"), @@ -1182,6 +1190,7 @@ public void testSavepointWithDeactivatedPeriodicCheckpointing() throws Exception ResourceID.generate(), actorSystem, highAvailabilityServices, + new NoOpMetricRegistry(), "localhost", Option.apply("tm"), true, @@ -1275,6 +1284,7 @@ public void testSavepointRestoreSettings() throws Exception { TestingUtils.defaultExecutor(), TestingUtils.defaultExecutor(), highAvailabilityServices, + new NoOpMetricRegistry(), Option.empty(), Option.apply("jm"), Option.apply("arch"), @@ -1296,6 +1306,7 @@ public void testSavepointRestoreSettings() throws Exception { ResourceID.generate(), actorSystem, highAvailabilityServices, + new NoOpMetricRegistry(), "localhost", Option.apply("tm"), true, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java index 340a735a23b22..cc93f18951934 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java @@ -37,6 +37,7 @@ import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.messages.JobManagerMessages; +import org.apache.flink.runtime.metrics.NoOpMetricRegistry; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.testtasks.NoOpInvokable; import org.apache.flink.runtime.util.LeaderRetrievalUtils; @@ -94,6 +95,7 @@ public static void setupJobManager() { TestingUtils.defaultExecutor(), TestingUtils.defaultExecutor(), highAvailabilityServices, + new NoOpMetricRegistry(), Option.empty(), JobManager.class, MemoryArchivist.class)._1(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerMockTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerMockTest.java index b4f50fbb01668..083d6e9c59f82 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerMockTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerMockTest.java @@ -33,7 +33,7 @@ import org.apache.flink.runtime.jobmanager.OnCompletionActions; import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore; import org.apache.flink.runtime.leaderelection.LeaderElectionService; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; @@ -115,7 +115,7 @@ public void setUp() throws Exception { haServices, heartbeatServices, JobManagerServices.fromConfiguration(new Configuration(), mock(BlobServer.class)), - new MetricRegistry(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()), + new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()), jobCompletion, jobCompletion)); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/JobManagerLeaderElectionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/JobManagerLeaderElectionTest.java index c3b57fa366d99..e4ceb403ff007 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/JobManagerLeaderElectionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/JobManagerLeaderElectionTest.java @@ -28,7 +28,6 @@ import org.apache.curator.framework.CuratorFramework; import org.apache.curator.test.TestingServer; import org.apache.flink.configuration.BlobServerOptions; -import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.blob.BlobServer; @@ -43,7 +42,7 @@ import org.apache.flink.runtime.jobmanager.StandaloneSubmittedJobGraphStore; import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore; import org.apache.flink.runtime.jobmanager.scheduler.Scheduler; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.testingUtils.TestingJobManager; import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages; import org.apache.flink.runtime.testingUtils.TestingUtils; @@ -199,7 +198,7 @@ private Props createJobManagerProps(Configuration configuration) throws Exceptio submittedJobGraphStore, checkpointRecoveryFactory, AkkaUtils.getDefaultTimeoutAsFiniteDuration(), - Option.empty(), + Option.empty(), Option.empty()); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/MetricRegistryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/MetricRegistryImplTest.java similarity index 93% rename from flink-runtime/src/test/java/org/apache/flink/runtime/metrics/MetricRegistryTest.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/metrics/MetricRegistryImplTest.java index 284b86a77e851..b0b20b2ddb3f9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/MetricRegistryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/MetricRegistryImplTest.java @@ -52,15 +52,15 @@ import static org.junit.Assert.fail; /** - * Tests for the {@link MetricRegistry}. + * Tests for the {@link MetricRegistryImpl}. */ -public class MetricRegistryTest extends TestLogger { +public class MetricRegistryImplTest extends TestLogger { private static final char GLOBAL_DEFAULT_DELIMITER = '.'; @Test public void testIsShutdown() { - MetricRegistry metricRegistry = new MetricRegistry(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); + MetricRegistryImpl metricRegistry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); Assert.assertFalse(metricRegistry.isShutdown()); @@ -79,7 +79,7 @@ public void testReporterInstantiation() { config.setString(MetricOptions.REPORTERS_LIST, "test"); config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter1.class.getName()); - MetricRegistry metricRegistry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(config)); + MetricRegistryImpl metricRegistry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(config)); assertTrue(metricRegistry.getReporters().size() == 1); @@ -112,7 +112,7 @@ public void testMultipleReporterInstantiation() { config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test2." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter12.class.getName()); config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test3." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter13.class.getName()); - MetricRegistry metricRegistry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(config)); + MetricRegistryImpl metricRegistry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(config)); assertTrue(metricRegistry.getReporters().size() == 3); @@ -171,7 +171,7 @@ public void testReporterArgumentForwarding() { config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test.arg1", "hello"); config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test.arg2", "world"); - new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(config)).shutdown(); + new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(config)).shutdown(); Assert.assertEquals("hello", TestReporter2.mc.getString("arg1", null)); Assert.assertEquals("world", TestReporter2.mc.getString("arg2", null)); @@ -202,7 +202,7 @@ public void testReporterScheduling() throws InterruptedException { config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test.arg1", "hello"); config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test." + ConfigConstants.METRICS_REPORTER_INTERVAL_SUFFIX, "50 MILLISECONDS"); - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(config)); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(config)); long start = System.currentTimeMillis(); @@ -251,7 +251,7 @@ public void testReporterNotifications() { config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test1." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter6.class.getName()); config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test2." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter7.class.getName()); - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(config)); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(config)); TaskManagerMetricGroup root = new TaskManagerMetricGroup(registry, "host", "id"); root.counter("rootCounter"); @@ -345,7 +345,7 @@ public void testConfigurableDelimiter() { config.setString(MetricOptions.SCOPE_DELIMITER, "_"); config.setString(MetricOptions.SCOPE_NAMING_TM, "A.B.C.D.E"); - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(config)); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(config)); TaskManagerMetricGroup tmGroup = new TaskManagerMetricGroup(registry, "host", "id"); assertEquals("A_B_C_D_E_name", tmGroup.getMetricIdentifier("name")); @@ -364,7 +364,7 @@ public void testConfigurableDelimiterForReporters() { config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test3." + ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER, "AA"); config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test3." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter.class.getName()); - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(config)); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(config)); assertEquals(GLOBAL_DEFAULT_DELIMITER, registry.getDelimiter()); assertEquals('_', registry.getDelimiter(0)); @@ -389,7 +389,7 @@ public void testConfigurableDelimiterForReportersInGroup() { config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test4." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter8.class.getName()); config.setString(MetricOptions.SCOPE_NAMING_TM, "A.B"); - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(config)); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(config)); List reporters = registry.getReporters(); ((TestReporter8) reporters.get(0)).expectedDelimiter = '_'; //test1 reporter ((TestReporter8) reporters.get(1)).expectedDelimiter = '-'; //test2 reporter @@ -411,7 +411,7 @@ public void testConfigurableDelimiterForReportersInGroup() { public void testQueryActorShutdown() throws Exception { final FiniteDuration timeout = new FiniteDuration(10L, TimeUnit.SECONDS); - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); final ActorSystem actorSystem = AkkaUtils.createDefaultActorSystem(); @@ -463,7 +463,7 @@ public void testExceptionIsolation() throws Exception { config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test1." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, FailingReporter.class.getName()); config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test2." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter7.class.getName()); - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(config)); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(config)); Counter metric = new SimpleCounter(); registry.register(metric, "counter", new MetricGroupTest.DummyAbstractMetricGroup(registry)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/NoOpMetricRegistry.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/NoOpMetricRegistry.java new file mode 100644 index 0000000000000..1140e3d3ee747 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/NoOpMetricRegistry.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.metrics; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Metric; +import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup; +import org.apache.flink.runtime.metrics.scope.ScopeFormats; + +/** + * Metric registry which does nothing and is intended for testing purposes. + */ +public class NoOpMetricRegistry implements MetricRegistry { + + final char delimiter = ','; + + final ScopeFormats scopeFormats = ScopeFormats.fromConfig(new Configuration()); + + @Override + public char getDelimiter() { + return delimiter; + } + + @Override + public char getDelimiter(int index) { + return delimiter; + } + + @Override + public int getNumberReporters() { + return 0; + } + + @Override + public void register(Metric metric, String metricName, AbstractMetricGroup group) {} + + @Override + public void unregister(Metric metric, String metricName, AbstractMetricGroup group) {} + + @Override + public ScopeFormats getScopeFormats() { + return scopeFormats; + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/TaskManagerMetricsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/TaskManagerMetricsTest.java index cea09281c914d..31304e54a951a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/TaskManagerMetricsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/TaskManagerMetricsTest.java @@ -61,6 +61,9 @@ public void testMetricRegistryLifeCycle() throws Exception { HighAvailabilityServices highAvailabilityServices = new EmbeddedHaServices(TestingUtils.defaultExecutor()); + final MetricRegistryImpl metricRegistry = new MetricRegistryImpl( + MetricRegistryConfiguration.fromConfiguration(new Configuration())); + try { actorSystem = AkkaUtils.createLocalActorSystem(new Configuration()); @@ -73,6 +76,7 @@ public void testMetricRegistryLifeCycle() throws Exception { TestingUtils.defaultExecutor(), TestingUtils.defaultExecutor(), highAvailabilityServices, + new NoOpMetricRegistry(), Option.empty(), JobManager.class, MemoryArchivist.class)._1(); @@ -89,9 +93,9 @@ public void testMetricRegistryLifeCycle() throws Exception { TaskManagerConfiguration taskManagerConfiguration = TaskManagerConfiguration.fromConfiguration(config); TaskManagerServices taskManagerServices = TaskManagerServices.fromConfiguration( - taskManagerServicesConfiguration, tmResourceID); - - final MetricRegistry tmRegistry = taskManagerServices.getMetricRegistry(); + taskManagerServicesConfiguration, + tmResourceID, + metricRegistry); // create the task manager final Props tmProps = TaskManager.getTaskManagerProps( @@ -103,7 +107,7 @@ public void testMetricRegistryLifeCycle() throws Exception { taskManagerServices.getIOManager(), taskManagerServices.getNetworkEnvironment(), highAvailabilityServices, - tmRegistry); + taskManagerServices.getTaskManagerMetricGroup()); final ActorRef taskManager = actorSystem.actorOf(tmProps); @@ -135,7 +139,7 @@ protected void run() { }}; // verify that the registry was not shutdown due to the disconnect - Assert.assertFalse(tmRegistry.isShutdown()); + Assert.assertFalse(metricRegistry.isShutdown()); // shut down the actors and the actor system actorSystem.shutdown(); @@ -148,6 +152,8 @@ protected void run() { if (highAvailabilityServices != null) { highAvailabilityServices.closeAndCleanupAllData(); } + + metricRegistry.shutdown(); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/dump/MetricQueryServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/dump/MetricQueryServiceTest.java index 5c33ad6ae138f..55ba3a9dd2900 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/dump/MetricQueryServiceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/dump/MetricQueryServiceTest.java @@ -25,8 +25,8 @@ import org.apache.flink.metrics.Meter; import org.apache.flink.metrics.SimpleCounter; import org.apache.flink.runtime.akka.AkkaUtils; -import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup; import org.apache.flink.runtime.metrics.util.TestingHistogram; import org.apache.flink.util.TestLogger; @@ -82,7 +82,7 @@ public long getCount() { } }; - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); final TaskManagerMetricGroup tm = new TaskManagerMetricGroup(registry, "host", "id"); MetricQueryService.notifyOfAddedMetric(serviceActor, c, "counter", tm); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroupTest.java index 648ee4760003c..8d91b818b8a13 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroupTest.java @@ -25,8 +25,8 @@ import org.apache.flink.metrics.Metric; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.metrics.reporter.MetricReporter; -import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.metrics.dump.QueryScopeInfo; import org.apache.flink.runtime.metrics.util.TestReporter; @@ -45,7 +45,7 @@ public class AbstractMetricGroupTest { */ @Test public void testGetAllVariables() { - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); AbstractMetricGroup group = new AbstractMetricGroup>(registry, new String[0], null) { @Override @@ -90,7 +90,7 @@ public void testScopeCachingForMultipleReporters() throws Exception { config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test2." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter2.class.getName()); config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test2." + ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER, "!"); - MetricRegistry testRegistry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(config)); + MetricRegistryImpl testRegistry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(config)); try { MetricGroup tmGroup = new TaskManagerMetricGroup(testRegistry, "host", "id"); tmGroup.counter("1"); @@ -180,7 +180,7 @@ public String filterCharacters(String input) { public void testScopeGenerationWithoutReporters() { Configuration config = new Configuration(); config.setString(MetricOptions.SCOPE_NAMING_TM, "A.B.C.D"); - MetricRegistry testRegistry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(config)); + MetricRegistryImpl testRegistry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(config)); try { TaskManagerMetricGroup group = new TaskManagerMetricGroup(testRegistry, "host", "id"); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/JobManagerGroupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/JobManagerGroupTest.java index 03341a63ad95d..05a72acb4d1e4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/JobManagerGroupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/JobManagerGroupTest.java @@ -22,8 +22,8 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.MetricOptions; import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.metrics.dump.QueryScopeInfo; import org.apache.flink.runtime.metrics.util.DummyCharacterFilter; import org.apache.flink.util.TestLogger; @@ -45,7 +45,7 @@ public class JobManagerGroupTest extends TestLogger { @Test public void addAndRemoveJobs() { - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); final JobManagerMetricGroup group = new JobManagerMetricGroup(registry, "localhost"); final JobID jid1 = new JobID(); @@ -77,7 +77,7 @@ public void addAndRemoveJobs() { @Test public void testCloseClosesAll() { - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); final JobManagerMetricGroup group = new JobManagerMetricGroup(registry, "localhost"); final JobID jid1 = new JobID(); @@ -103,7 +103,7 @@ public void testCloseClosesAll() { @Test public void testGenerateScopeDefault() { - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); JobManagerMetricGroup group = new JobManagerMetricGroup(registry, "localhost"); assertArrayEquals(new String[]{"localhost", "jobmanager"}, group.getScopeComponents()); @@ -116,7 +116,7 @@ public void testGenerateScopeDefault() { public void testGenerateScopeCustom() { Configuration cfg = new Configuration(); cfg.setString(MetricOptions.SCOPE_NAMING_JM, "constant..foo."); - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(cfg)); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(cfg)); JobManagerMetricGroup group = new JobManagerMetricGroup(registry, "host"); @@ -128,7 +128,7 @@ public void testGenerateScopeCustom() { @Test public void testCreateQueryServiceMetricInfo() { - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); JobManagerMetricGroup jm = new JobManagerMetricGroup(registry, "host"); QueryScopeInfo.JobManagerQueryScopeInfo info = jm.createQueryServiceMetricInfo(new DummyCharacterFilter()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/JobManagerJobGroupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/JobManagerJobGroupTest.java index d734dfdddc9ca..4373f80c37d9b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/JobManagerJobGroupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/JobManagerJobGroupTest.java @@ -21,8 +21,8 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.MetricOptions; -import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.metrics.dump.QueryScopeInfo; import org.apache.flink.runtime.metrics.util.DummyCharacterFilter; import org.apache.flink.util.TestLogger; @@ -39,7 +39,7 @@ public class JobManagerJobGroupTest extends TestLogger { @Test public void testGenerateScopeDefault() { - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); JobManagerMetricGroup tmGroup = new JobManagerMetricGroup(registry, "theHostName"); JobMetricGroup jmGroup = new JobManagerJobMetricGroup(registry, tmGroup, new JobID(), "myJobName"); @@ -60,7 +60,7 @@ public void testGenerateScopeCustom() { Configuration cfg = new Configuration(); cfg.setString(MetricOptions.SCOPE_NAMING_JM, "abc"); cfg.setString(MetricOptions.SCOPE_NAMING_JM_JOB, "some-constant."); - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(cfg)); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(cfg)); JobID jid = new JobID(); @@ -83,7 +83,7 @@ public void testGenerateScopeCustomWildcard() { Configuration cfg = new Configuration(); cfg.setString(MetricOptions.SCOPE_NAMING_JM, "peter"); cfg.setString(MetricOptions.SCOPE_NAMING_JM_JOB, "*.some-constant."); - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(cfg)); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(cfg)); JobID jid = new JobID(); @@ -104,7 +104,7 @@ public void testGenerateScopeCustomWildcard() { @Test public void testCreateQueryServiceMetricInfo() { JobID jid = new JobID(); - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); JobManagerMetricGroup jm = new JobManagerMetricGroup(registry, "host"); JobManagerJobMetricGroup jmj = new JobManagerJobMetricGroup(registry, jm, jid, "jobname"); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/MetricGroupRegistrationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/MetricGroupRegistrationTest.java index 56ce5fa34d1be..324bb733fcf62 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/MetricGroupRegistrationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/MetricGroupRegistrationTest.java @@ -27,9 +27,10 @@ import org.apache.flink.metrics.HistogramStatistics; import org.apache.flink.metrics.Metric; import org.apache.flink.metrics.MetricGroup; -import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.metrics.util.TestReporter; +import org.apache.flink.util.TestLogger; import org.junit.Assert; import org.junit.Test; @@ -39,7 +40,7 @@ /** * Tests for the registration of groups and metrics on a {@link MetricGroup}. */ -public class MetricGroupRegistrationTest { +public class MetricGroupRegistrationTest extends TestLogger { /** * Verifies that group methods instantiate the correct metric with the given name. */ @@ -49,7 +50,7 @@ public void testMetricInstantiation() { config.setString(MetricOptions.REPORTERS_LIST, "test"); config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter1.class.getName()); - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(config)); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(config)); MetricGroup root = new TaskManagerMetricGroup(registry, "host", "id"); @@ -111,7 +112,7 @@ public void notifyOfAddedMetric(Metric metric, String metricName, MetricGroup gr public void testDuplicateGroupName() { Configuration config = new Configuration(); - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(config)); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(config)); MetricGroup root = new TaskManagerMetricGroup(registry, "host", "id"); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/MetricGroupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/MetricGroupTest.java index 633dbedca8caf..94760e614cb8a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/MetricGroupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/MetricGroupTest.java @@ -26,6 +26,7 @@ import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.metrics.dump.QueryScopeInfo; import org.apache.flink.runtime.metrics.util.DummyCharacterFilter; import org.apache.flink.util.AbstractID; @@ -48,13 +49,13 @@ public class MetricGroupTest extends TestLogger { private static final MetricRegistryConfiguration defaultMetricRegistryConfiguration = MetricRegistryConfiguration.defaultMetricRegistryConfiguration(); - private MetricRegistry registry; + private MetricRegistryImpl registry; - private final MetricRegistry exceptionOnRegister = new ExceptionOnRegisterRegistry(); + private final MetricRegistryImpl exceptionOnRegister = new ExceptionOnRegisterRegistry(); @Before public void createRegistry() { - this.registry = new MetricRegistry(defaultMetricRegistryConfiguration); + this.registry = new MetricRegistryImpl(defaultMetricRegistryConfiguration); } @After @@ -134,7 +135,7 @@ public void testCreateQueryServiceMetricInfo() { JobID jid = new JobID(); JobVertexID vid = new JobVertexID(); AbstractID eid = new AbstractID(); - MetricRegistry registry = new MetricRegistry(defaultMetricRegistryConfiguration); + MetricRegistryImpl registry = new MetricRegistryImpl(defaultMetricRegistryConfiguration); TaskManagerMetricGroup tm = new TaskManagerMetricGroup(registry, "host", "id"); TaskManagerJobMetricGroup job = new TaskManagerJobMetricGroup(registry, tm, jid, "jobname"); TaskMetricGroup task = new TaskMetricGroup(registry, job, vid, eid, "taskName", 4, 5); @@ -156,7 +157,7 @@ public void testCreateQueryServiceMetricInfo() { // ------------------------------------------------------------------------ - private static class ExceptionOnRegisterRegistry extends MetricRegistry { + private static class ExceptionOnRegisterRegistry extends MetricRegistryImpl { public ExceptionOnRegisterRegistry() { super(defaultMetricRegistryConfiguration); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/OperatorGroupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/OperatorGroupTest.java index 4363a9d7ad14d..820b73efc0010 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/OperatorGroupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/OperatorGroupTest.java @@ -24,8 +24,8 @@ import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.OperatorID; -import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.metrics.dump.QueryScopeInfo; import org.apache.flink.runtime.metrics.scope.ScopeFormat; import org.apache.flink.runtime.metrics.util.DummyCharacterFilter; @@ -47,7 +47,7 @@ public class OperatorGroupTest extends TestLogger { @Test public void testGenerateScopeDefault() { - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); TaskManagerMetricGroup tmGroup = new TaskManagerMetricGroup(registry, "theHostName", "test-tm-id"); TaskManagerJobMetricGroup jmGroup = new TaskManagerJobMetricGroup(registry, tmGroup, new JobID(), "myJobName"); @@ -70,7 +70,7 @@ public void testGenerateScopeDefault() { public void testGenerateScopeCustom() { Configuration cfg = new Configuration(); cfg.setString(MetricOptions.SCOPE_NAMING_OPERATOR, "...."); - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(cfg)); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(cfg)); try { String tmID = "test-tm-id"; JobID jid = new JobID(); @@ -97,7 +97,7 @@ public void testGenerateScopeCustom() { @Test public void testIOMetricGroupInstantiation() { - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); TaskManagerMetricGroup tmGroup = new TaskManagerMetricGroup(registry, "theHostName", "test-tm-id"); TaskManagerJobMetricGroup jmGroup = new TaskManagerJobMetricGroup(registry, tmGroup, new JobID(), "myJobName"); @@ -114,7 +114,7 @@ public void testIOMetricGroupInstantiation() { @Test public void testVariables() { - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); JobID jid = new JobID(); JobVertexID tid = new JobVertexID(); @@ -156,7 +156,7 @@ public void testCreateQueryServiceMetricInfo() { JobVertexID vid = new JobVertexID(); AbstractID eid = new AbstractID(); OperatorID oid = new OperatorID(); - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); TaskManagerMetricGroup tm = new TaskManagerMetricGroup(registry, "host", "id"); TaskManagerJobMetricGroup job = new TaskManagerJobMetricGroup(registry, tm, jid, "jobname"); TaskMetricGroup task = new TaskMetricGroup(registry, job, vid, eid, "taskName", 4, 5); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/TaskManagerGroupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/TaskManagerGroupTest.java index bd853032e75ce..3272f73765a2b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/TaskManagerGroupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/TaskManagerGroupTest.java @@ -23,8 +23,8 @@ import org.apache.flink.configuration.MetricOptions; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.jobgraph.JobVertexID; -import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.metrics.dump.QueryScopeInfo; import org.apache.flink.runtime.metrics.util.DummyCharacterFilter; import org.apache.flink.util.AbstractID; @@ -50,7 +50,7 @@ public class TaskManagerGroupTest extends TestLogger { @Test public void addAndRemoveJobs() throws IOException { - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); final TaskManagerMetricGroup group = new TaskManagerMetricGroup( registry, "localhost", new AbstractID().toString()); @@ -112,7 +112,7 @@ public void addAndRemoveJobs() throws IOException { @Test public void testCloseClosesAll() throws IOException { - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); final TaskManagerMetricGroup group = new TaskManagerMetricGroup( registry, "localhost", new AbstractID().toString()); @@ -152,7 +152,7 @@ public void testCloseClosesAll() throws IOException { @Test public void testGenerateScopeDefault() { - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); TaskManagerMetricGroup group = new TaskManagerMetricGroup(registry, "localhost", "id"); assertArrayEquals(new String[]{"localhost", "taskmanager", "id"}, group.getScopeComponents()); @@ -164,7 +164,7 @@ public void testGenerateScopeDefault() { public void testGenerateScopeCustom() { Configuration cfg = new Configuration(); cfg.setString(MetricOptions.SCOPE_NAMING_TM, "constant..foo."); - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(cfg)); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(cfg)); TaskManagerMetricGroup group = new TaskManagerMetricGroup(registry, "host", "id"); assertArrayEquals(new String[]{"constant", "host", "foo", "host"}, group.getScopeComponents()); @@ -174,7 +174,7 @@ public void testGenerateScopeCustom() { @Test public void testCreateQueryServiceMetricInfo() { - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); TaskManagerMetricGroup tm = new TaskManagerMetricGroup(registry, "host", "id"); QueryScopeInfo.TaskManagerQueryScopeInfo info = tm.createQueryServiceMetricInfo(new DummyCharacterFilter()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/TaskManagerJobGroupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/TaskManagerJobGroupTest.java index fc0ce5c55e741..b6be31c2dbfb3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/TaskManagerJobGroupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/TaskManagerJobGroupTest.java @@ -21,8 +21,8 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.MetricOptions; -import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.metrics.dump.QueryScopeInfo; import org.apache.flink.runtime.metrics.util.DummyCharacterFilter; import org.apache.flink.util.TestLogger; @@ -39,7 +39,7 @@ public class TaskManagerJobGroupTest extends TestLogger { @Test public void testGenerateScopeDefault() { - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); TaskManagerMetricGroup tmGroup = new TaskManagerMetricGroup(registry, "theHostName", "test-tm-id"); JobMetricGroup jmGroup = new TaskManagerJobMetricGroup(registry, tmGroup, new JobID(), "myJobName"); @@ -59,7 +59,7 @@ public void testGenerateScopeCustom() { Configuration cfg = new Configuration(); cfg.setString(MetricOptions.SCOPE_NAMING_TM, "abc"); cfg.setString(MetricOptions.SCOPE_NAMING_TM_JOB, "some-constant."); - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(cfg)); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(cfg)); JobID jid = new JobID(); @@ -81,7 +81,7 @@ public void testGenerateScopeCustomWildcard() { Configuration cfg = new Configuration(); cfg.setString(MetricOptions.SCOPE_NAMING_TM, "peter."); cfg.setString(MetricOptions.SCOPE_NAMING_TM_JOB, "*.some-constant."); - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(cfg)); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(cfg)); JobID jid = new JobID(); @@ -101,7 +101,7 @@ public void testGenerateScopeCustomWildcard() { @Test public void testCreateQueryServiceMetricInfo() { JobID jid = new JobID(); - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); TaskManagerMetricGroup tm = new TaskManagerMetricGroup(registry, "host", "id"); TaskManagerJobMetricGroup job = new TaskManagerJobMetricGroup(registry, tm, jid, "jobname"); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/TaskMetricGroupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/TaskMetricGroupTest.java index 43cbbf1b7eb09..be7407e867aab 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/TaskMetricGroupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/TaskMetricGroupTest.java @@ -23,8 +23,8 @@ import org.apache.flink.configuration.MetricOptions; import org.apache.flink.metrics.Metric; import org.apache.flink.runtime.jobgraph.JobVertexID; -import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.metrics.dump.QueryScopeInfo; import org.apache.flink.runtime.metrics.scope.ScopeFormat; import org.apache.flink.runtime.metrics.util.DummyCharacterFilter; @@ -49,7 +49,7 @@ public class TaskMetricGroupTest extends TestLogger { @Test public void testGenerateScopeDefault() { - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); JobVertexID vertexId = new JobVertexID(); AbstractID executionId = new AbstractID(); @@ -73,7 +73,7 @@ public void testGenerateScopeCustom() { cfg.setString(MetricOptions.SCOPE_NAMING_TM, "abc"); cfg.setString(MetricOptions.SCOPE_NAMING_TM_JOB, "def"); cfg.setString(MetricOptions.SCOPE_NAMING_TASK, "..."); - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(cfg)); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(cfg)); JobID jid = new JobID(); JobVertexID vertexId = new JobVertexID(); @@ -98,7 +98,7 @@ public void testGenerateScopeCustom() { public void testGenerateScopeWilcard() { Configuration cfg = new Configuration(); cfg.setString(MetricOptions.SCOPE_NAMING_TASK, "*.."); - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(cfg)); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(cfg)); AbstractID executionId = new AbstractID(); @@ -123,7 +123,7 @@ public void testCreateQueryServiceMetricInfo() { JobID jid = new JobID(); JobVertexID vid = new JobVertexID(); AbstractID eid = new AbstractID(); - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); TaskManagerMetricGroup tm = new TaskManagerMetricGroup(registry, "host", "id"); TaskManagerJobMetricGroup job = new TaskManagerJobMetricGroup(registry, tm, jid, "jobname"); TaskMetricGroup task = new TaskMetricGroup(registry, job, vid, eid, "taskName", 4, 5); @@ -157,7 +157,7 @@ public void testTaskMetricGroupCleanup() { public void testOperatorNameTruncation() { Configuration cfg = new Configuration(); cfg.setString(MetricOptions.SCOPE_NAMING_OPERATOR, ScopeFormat.SCOPE_OPERATOR_NAME); - MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(cfg)); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(cfg)); TaskManagerMetricGroup tm = new TaskManagerMetricGroup(registry, "host", "id"); TaskManagerJobMetricGroup job = new TaskManagerJobMetricGroup(registry, tm, new JobID(), "jobname"); TaskMetricGroup taskMetricGroup = new TaskMetricGroup(registry, job, new JobVertexID(), new AbstractID(), "task", 0, 0); @@ -170,7 +170,7 @@ public void testOperatorNameTruncation() { Assert.assertEquals(originalName.substring(0, TaskMetricGroup.METRICS_OPERATOR_NAME_MAX_LENGTH), storedName); } - private static class CountingMetricRegistry extends MetricRegistry { + private static class CountingMetricRegistry extends MetricRegistryImpl { private int counter = 0; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnregisteredTaskMetricsGroup.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnregisteredTaskMetricsGroup.java index 2ca19c106a1c4..3d2981577e9e9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnregisteredTaskMetricsGroup.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnregisteredTaskMetricsGroup.java @@ -29,7 +29,7 @@ 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.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.jobgraph.JobVertexID; @@ -37,7 +37,7 @@ public class UnregisteredTaskMetricsGroup extends TaskMetricGroup { - private static final MetricRegistry EMPTY_REGISTRY = new MetricRegistry(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); + private static final MetricRegistryImpl EMPTY_REGISTRY = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); public UnregisteredTaskMetricsGroup() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java index d0dd973687c86..306d4d45803ad 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java @@ -24,7 +24,7 @@ import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerConfiguration; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.TestingRpcService; @@ -78,7 +78,7 @@ public void confirmLeaderSessionID(UUID leaderId) { highAvailabilityServices, rpcService.getScheduledExecutor()); - MetricRegistry metricRegistry = mock(MetricRegistry.class); + MetricRegistryImpl metricRegistry = mock(MetricRegistryImpl.class); TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java index 73c5b5c72e4ad..1b6324c92d92f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java @@ -32,7 +32,7 @@ import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; import org.apache.flink.runtime.rpc.FatalErrorHandler; @@ -270,7 +270,7 @@ private ResourceManager createAndStartResourceManager( ResourceManagerConfiguration resourceManagerConfiguration = new ResourceManagerConfiguration( Time.seconds(5L), Time.seconds(5L)); - MetricRegistry metricRegistry = mock(MetricRegistry.class); + MetricRegistryImpl metricRegistry = mock(MetricRegistryImpl.class); JobLeaderIdService jobLeaderIdService = new JobLeaderIdService( highAvailabilityServices, rpcService.getScheduledExecutor(), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java index 0206adebc744a..147d180e6b990 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java @@ -25,7 +25,7 @@ import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.TestingRpcService; @@ -182,7 +182,7 @@ private StandaloneResourceManager createAndStartResourceManager(LeaderElectionSe TestingUtils.infiniteTime(), TestingUtils.infiniteTime()); - MetricRegistry metricRegistry = mock(MetricRegistry.class); + MetricRegistryImpl metricRegistry = mock(MetricRegistryImpl.class); JobLeaderIdService jobLeaderIdService = new JobLeaderIdService( highAvailabilityServices, rpcService.getScheduledExecutor(), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/TaskManagerLogHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/TaskManagerLogHandlerTest.java index 8e76674f6aedb..b600cbe09ee20 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/TaskManagerLogHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/TaskManagerLogHandlerTest.java @@ -105,7 +105,7 @@ public void testLogFetchingFailure() throws Exception { JobManagerGateway jobManagerGateway = mock(JobManagerGateway.class); when(jobManagerGateway.requestBlobServerPort(any(Time.class))).thenReturn(CompletableFuture.completedFuture(1337)); when(jobManagerGateway.getHostname()).thenReturn("localhost"); - when(jobManagerGateway.requestTaskManagerInstance(any(InstanceID.class), any(Time.class))).thenReturn( + when(jobManagerGateway.requestTaskManagerInstance(any(ResourceID.class), any(Time.class))).thenReturn( CompletableFuture.completedFuture(Optional.of(taskManager))); GatewayRetriever retriever = mock(GatewayRetriever.class); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/MetricFetcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/MetricFetcherTest.java index ce98f31dda475..7b10db6736a68 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/MetricFetcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/MetricFetcherTest.java @@ -89,7 +89,7 @@ public void testUpdate() throws Exception { when(jobManagerGateway.requestMetricQueryServicePaths(any(Time.class))).thenReturn( CompletableFuture.completedFuture(Collections.singleton(jmMetricQueryServicePath))); when(jobManagerGateway.requestTaskManagerMetricQueryServicePaths(any(Time.class))).thenReturn( - CompletableFuture.completedFuture(Collections.singleton(Tuple2.of(tmID, tmMetricQueryServicePath)))); + CompletableFuture.completedFuture(Collections.singleton(Tuple2.of(tmRID, tmMetricQueryServicePath)))); GatewayRetriever retriever = mock(AkkaJobManagerRetriever.class); when(retriever.getNow()) @@ -99,7 +99,7 @@ public void testUpdate() throws Exception { MetricQueryServiceGateway jmQueryService = mock(MetricQueryServiceGateway.class); MetricQueryServiceGateway tmQueryService = mock(MetricQueryServiceGateway.class); - MetricDumpSerialization.MetricSerializationResult requestMetricsAnswer = createRequestDumpAnswer(tmID, jobID); + MetricDumpSerialization.MetricSerializationResult requestMetricsAnswer = createRequestDumpAnswer(tmRID, jobID); when(jmQueryService.queryMetrics(any(Time.class))) .thenReturn(CompletableFuture.completedFuture(new MetricDumpSerialization.MetricSerializationResult(new byte[0], 0, 0, 0, 0))); @@ -133,14 +133,14 @@ public void testUpdate() throws Exception { assertEquals("0.99", store.getJobManagerMetricStore().getMetric("abc.hist_p99")); assertEquals("0.999", store.getJobManagerMetricStore().getMetric("abc.hist_p999")); - assertEquals("x", store.getTaskManagerMetricStore(tmID.toString()).metrics.get("abc.gauge")); + assertEquals("x", store.getTaskManagerMetricStore(tmRID.toString()).metrics.get("abc.gauge")); assertEquals("5.0", store.getJobMetricStore(jobID.toString()).metrics.get("abc.jc")); assertEquals("2", store.getTaskMetricStore(jobID.toString(), "taskid").metrics.get("2.abc.tc")); assertEquals("1", store.getTaskMetricStore(jobID.toString(), "taskid").metrics.get("2.opname.abc.oc")); } } - private static MetricDumpSerialization.MetricSerializationResult createRequestDumpAnswer(InstanceID tmID, JobID jobID) { + private static MetricDumpSerialization.MetricSerializationResult createRequestDumpAnswer(ResourceID tmRID, JobID jobID) { Map> counters = new HashMap<>(); Map, Tuple2> gauges = new HashMap<>(); Map> histograms = new HashMap<>(); @@ -178,7 +178,7 @@ public long getCount() { public String getValue() { return "x"; } - }, new Tuple2<>(new QueryScopeInfo.TaskManagerQueryScopeInfo(tmID.toString(), "abc"), "gauge")); + }, new Tuple2<>(new QueryScopeInfo.TaskManagerQueryScopeInfo(tmRID.toString(), "abc"), "gauge")); histograms.put(new TestingHistogram(), new Tuple2<>(new QueryScopeInfo.JobManagerQueryScopeInfo("abc"), "hist")); MetricDumpSerialization.MetricDumpSerializer serializer = new MetricDumpSerialization.MetricDumpSerializer(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java index e448cccbcaaf7..7c6b7ddc68c42 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java @@ -39,7 +39,7 @@ import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService; import org.apache.flink.runtime.memory.MemoryManager; import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup; import org.apache.flink.runtime.registration.RegistrationResponse; import org.apache.flink.runtime.resourcemanager.JobLeaderIdService; @@ -112,7 +112,7 @@ public void testSlotAllocation() throws Exception { testingHAServices, rpcService.getScheduledExecutor(), Time.minutes(5L)); - MetricRegistry metricRegistry = mock(MetricRegistry.class); + MetricRegistryImpl metricRegistry = mock(MetricRegistryImpl.class); HeartbeatServices heartbeatServices = mock(HeartbeatServices.class, RETURNS_MOCKS); final TaskManagerConfiguration taskManagerConfiguration = TaskManagerConfiguration.fromConfiguration(configuration); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java index e10623807f0e9..fcd6e4e45c5f9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java @@ -61,7 +61,7 @@ import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.memory.MemoryManager; import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.metrics.groups.TaskIOMetricGroup; import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup; import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; @@ -205,7 +205,7 @@ public HeartbeatManagerImpl answer(InvocationOnMock invocation) thro mock(NetworkEnvironment.class), haServices, heartbeatServices, - mock(MetricRegistry.class), + mock(MetricRegistryImpl.class), mock(TaskManagerMetricGroup.class), mock(BroadcastVariableManager.class), mock(FileCache.class), @@ -311,7 +311,7 @@ public HeartbeatManagerImpl answer(InvocationOnMock invocation mock(NetworkEnvironment.class), haServices, heartbeatServices, - mock(MetricRegistry.class), + mock(MetricRegistryImpl.class), mock(TaskManagerMetricGroup.class), mock(BroadcastVariableManager.class), mock(FileCache.class), @@ -429,7 +429,7 @@ public HeartbeatManagerImpl answer(InvocationOnMock invocation mock(NetworkEnvironment.class), haServices, heartbeatServices, - mock(MetricRegistry.class), + mock(MetricRegistryImpl.class), mock(TaskManagerMetricGroup.class), mock(BroadcastVariableManager.class), mock(FileCache.class), @@ -522,7 +522,7 @@ public void testImmediatelyRegistersIfLeaderIsKnown() throws Exception { mock(NetworkEnvironment.class), haServices, mock(HeartbeatServices.class, RETURNS_MOCKS), - mock(MetricRegistry.class), + mock(MetricRegistryImpl.class), mock(TaskManagerMetricGroup.class), mock(BroadcastVariableManager.class), mock(FileCache.class), @@ -605,7 +605,7 @@ public void testTriggerRegistrationOnLeaderChange() throws Exception { mock(NetworkEnvironment.class), haServices, mock(HeartbeatServices.class, RETURNS_MOCKS), - mock(MetricRegistry.class), + mock(MetricRegistryImpl.class), mock(TaskManagerMetricGroup.class), mock(BroadcastVariableManager.class), mock(FileCache.class), @@ -747,7 +747,7 @@ public void testTaskSubmission() throws Exception { networkEnvironment, haServices, mock(HeartbeatServices.class, RETURNS_MOCKS), - mock(MetricRegistry.class), + mock(MetricRegistryImpl.class), taskManagerMetricGroup, mock(BroadcastVariableManager.class), mock(FileCache.class), @@ -863,7 +863,7 @@ public void testJobLeaderDetection() throws Exception { mock(NetworkEnvironment.class), haServices, mock(HeartbeatServices.class, RETURNS_MOCKS), - mock(MetricRegistry.class), + mock(MetricRegistryImpl.class), mock(TaskManagerMetricGroup.class), mock(BroadcastVariableManager.class), mock(FileCache.class), @@ -981,7 +981,7 @@ public void testSlotAcceptance() throws Exception { mock(NetworkEnvironment.class), haServices, mock(HeartbeatServices.class, RETURNS_MOCKS), - mock(MetricRegistry.class), + mock(MetricRegistryImpl.class), mock(TaskManagerMetricGroup.class), mock(BroadcastVariableManager.class), mock(FileCache.class), @@ -1074,7 +1074,7 @@ public void testRejectAllocationRequestsForOutOfSyncSlots() throws Exception { mock(NetworkEnvironment.class), haServices, mock(HeartbeatServices.class, RETURNS_MOCKS), - mock(MetricRegistry.class), + mock(MetricRegistryImpl.class), mock(TaskManagerMetricGroup.class), mock(BroadcastVariableManager.class), mock(FileCache.class), @@ -1248,7 +1248,7 @@ public void testSubmitTaskBeforeAcceptSlot() throws Exception { networkMock, haServices, mock(HeartbeatServices.class, RETURNS_MOCKS), - mock(MetricRegistry.class), + mock(MetricRegistryImpl.class), taskManagerMetricGroup, mock(BroadcastVariableManager.class), mock(FileCache.class), @@ -1371,7 +1371,7 @@ public void testFilterOutDuplicateJobMasterRegistrations() throws Exception { mock(NetworkEnvironment.class), haServicesMock, heartbeatServicesMock, - mock(MetricRegistry.class), + mock(MetricRegistryImpl.class), mock(TaskManagerMetricGroup.class), mock(BroadcastVariableManager.class), mock(FileCache.class), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesTest.java index bf90634ea2a81..a8358a1b634e8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesTest.java @@ -21,9 +21,9 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.core.memory.MemoryType; -import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration; import org.apache.flink.runtime.util.EnvironmentInformation; + import org.junit.Test; import org.junit.runner.RunWith; import org.powermock.api.mockito.PowerMockito; @@ -259,7 +259,6 @@ private static TaskManagerServicesConfiguration getTmConfig( managedMemory, false, managedMemoryFraction, - mock(MetricRegistryConfiguration.class), 0); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java index 10b74c3a24acf..8249fcaa9ca76 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java @@ -49,8 +49,9 @@ import org.apache.flink.runtime.jobmanager.MemoryArchivist; import org.apache.flink.runtime.memory.MemoryManager; import org.apache.flink.runtime.messages.TaskManagerMessages; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; +import org.apache.flink.runtime.metrics.NoOpMetricRegistry; import org.apache.flink.runtime.query.KvStateRegistry; import org.apache.flink.runtime.taskexecutor.TaskManagerConfiguration; import org.apache.flink.runtime.testingUtils.TestingUtils; @@ -62,7 +63,6 @@ import scala.concurrent.duration.FiniteDuration; import java.net.InetAddress; -import java.util.Arrays; import java.util.concurrent.TimeUnit; public class TaskManagerComponentsStartupShutdownTest extends TestLogger { @@ -99,6 +99,7 @@ public void testComponentsStartupShutdown() throws Exception { TestingUtils.defaultExecutor(), TestingUtils.defaultExecutor(), highAvailabilityServices, + new NoOpMetricRegistry(), Option.empty(), JobManager.class, MemoryArchivist.class)._1(); @@ -168,7 +169,7 @@ public void testComponentsStartupShutdown() throws Exception { network, numberOfSlots, highAvailabilityServices, - new MetricRegistry(metricRegistryConfiguration)); + new MetricRegistryImpl(metricRegistryConfiguration)); taskManager = actorSystem.actorOf(tmProps); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerProcessReapingTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerProcessReapingTestBase.java index 22d49d2eda1db..7429ec5e80c5d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerProcessReapingTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerProcessReapingTestBase.java @@ -18,12 +18,9 @@ package org.apache.flink.runtime.taskmanager; -import akka.actor.ActorRef; -import akka.actor.ActorSystem; -import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.clusterframework.FlinkResourceManager; import org.apache.flink.runtime.clusterframework.standalone.StandaloneResourceManager; @@ -32,17 +29,16 @@ import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; import org.apache.flink.runtime.jobmanager.JobManager; import org.apache.flink.runtime.jobmanager.MemoryArchivist; +import org.apache.flink.runtime.metrics.NoOpMetricRegistry; import org.apache.flink.runtime.taskexecutor.TaskExecutor; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.util.NetUtils; import org.apache.flink.util.TestLogger; -import org.junit.Test; -import scala.Option; -import scala.Some; -import scala.Tuple2; -import scala.concurrent.duration.FiniteDuration; +import akka.actor.ActorRef; +import akka.actor.ActorSystem; +import org.junit.Test; import java.io.File; import java.io.IOException; @@ -50,6 +46,11 @@ import java.io.StringWriter; import java.util.concurrent.TimeUnit; +import scala.Option; +import scala.Some; +import scala.Tuple2; +import scala.concurrent.duration.FiniteDuration; + import static org.apache.flink.runtime.testutils.CommonTestUtils.getCurrentClasspath; import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath; import static org.apache.flink.runtime.testutils.CommonTestUtils.isProcessAlive; @@ -121,6 +122,7 @@ public void testReapProcessOnFailure() throws Exception { TestingUtils.defaultExecutor(), TestingUtils.defaultExecutor(), highAvailabilityServices, + new NoOpMetricRegistry(), Option.empty(), JobManager.class, MemoryArchivist.class)._1; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerStartupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerStartupTest.java index 67accdb366ad4..2e6c580fb00a3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerStartupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerStartupTest.java @@ -29,6 +29,7 @@ import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedHaServices; +import org.apache.flink.runtime.metrics.NoOpMetricRegistry; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.util.StartupUtils; import org.apache.flink.util.NetUtils; @@ -249,6 +250,7 @@ public void testStartupWhenNetworkStackFailsToInitialize() throws Exception { ResourceID.generate(), null, highAvailabilityServices, + new NoOpMetricRegistry(), "localhost", Option.empty(), false, diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerRegistrationTest.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerRegistrationTest.scala index f575867600221..887c4f5d37709 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerRegistrationTest.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerRegistrationTest.scala @@ -33,6 +33,7 @@ import org.apache.flink.runtime.instance._ import org.apache.flink.runtime.jobmanager.JobManagerRegistrationTest.PlainForwardingActor import org.apache.flink.runtime.messages.JobManagerMessages.LeaderSessionMessage import org.apache.flink.runtime.messages.RegistrationMessages.{AcknowledgeRegistration, AlreadyRegistered, RegisterTaskManager} +import org.apache.flink.runtime.metrics.{MetricRegistryImpl, MetricRegistryConfiguration} import org.apache.flink.runtime.taskmanager.TaskManagerLocation import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.NotifyWhenLeader import org.apache.flink.runtime.testingUtils.{TestingJobManager, TestingUtils} @@ -60,6 +61,10 @@ ImplicitSender with WordSpecLike with Matchers with BeforeAndAfterAll with Befor var highAvailabilityServices: HighAvailabilityServices = _ + val metricRegistry: MetricRegistryImpl = new MetricRegistryImpl( + MetricRegistryConfiguration.fromConfiguration(new Configuration()) + ) + val timeout = FiniteDuration(30, TimeUnit.SECONDS) override def afterAll(): Unit = { @@ -87,7 +92,7 @@ ImplicitSender with WordSpecLike with Matchers with BeforeAndAfterAll with Befor var tm2Option: Option[ActorRef] = None try { - val jm = startTestingJobManager(_system, highAvailabilityServices) + val jm = startTestingJobManager(_system, highAvailabilityServices, metricRegistry) jmOption = Some(jm) val rm = startTestingResourceManager(_system, highAvailabilityServices) @@ -169,7 +174,7 @@ ImplicitSender with WordSpecLike with Matchers with BeforeAndAfterAll with Befor try { val probe = TestProbe() - val jm = startTestingJobManager(_system, highAvailabilityServices) + val jm = startTestingJobManager(_system, highAvailabilityServices, metricRegistry) jmOption = Some(jm) val rm = startTestingResourceManager(_system, highAvailabilityServices) rmOption = Some(rm) @@ -242,7 +247,8 @@ ImplicitSender with WordSpecLike with Matchers with BeforeAndAfterAll with Befor private def startTestingJobManager( system: ActorSystem, - highAvailabilityServices: HighAvailabilityServices): ActorGateway = { + highAvailabilityServices: HighAvailabilityServices, + metricRegistry: MetricRegistryImpl): ActorGateway = { val config = new Configuration() @@ -250,7 +256,8 @@ ImplicitSender with WordSpecLike with Matchers with BeforeAndAfterAll with Befor config, executor, executor, - highAvailabilityServices.createBlobStore()) + highAvailabilityServices.createBlobStore(), + metricRegistry) // Start the JobManager without a MetricRegistry so that we don't start the MetricQueryService. // The problem of the MetricQueryService is that it starts an actor with a fixed name. Thus, @@ -273,7 +280,7 @@ ImplicitSender with WordSpecLike with Matchers with BeforeAndAfterAll with Befor highAvailabilityServices.getSubmittedJobGraphStore(), highAvailabilityServices.getCheckpointRecoveryFactory(), components._9, - None, + components._10, None) _system.actorOf(props) diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala index 2e884a0b84544..2b91cd4ab8cfd 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala @@ -42,7 +42,8 @@ import org.apache.flink.runtime.jobmanager.{JobManager, MemoryArchivist, Submitt import org.apache.flink.runtime.leaderelection.LeaderElectionService import org.apache.flink.runtime.messages.JobManagerMessages import org.apache.flink.runtime.messages.JobManagerMessages._ -import org.apache.flink.runtime.metrics.MetricRegistry +import org.apache.flink.runtime.metrics.MetricRegistryImpl +import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster import org.apache.flink.runtime.taskmanager.TaskManager import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages._ @@ -120,7 +121,7 @@ class TestingCluster( submittedJobGraphStore: SubmittedJobGraphStore, checkpointRecoveryFactory: CheckpointRecoveryFactory, jobRecoveryTimeout: FiniteDuration, - metricsRegistry: Option[MetricRegistry], + jobManagerMetricGroup: JobManagerMetricGroup, optRestAddress: Option[String]): Props = { val props = super.getJobManagerProps( @@ -139,7 +140,7 @@ class TestingCluster( submittedJobGraphStore, checkpointRecoveryFactory, jobRecoveryTimeout, - metricsRegistry, + jobManagerMetricGroup, optRestAddress) if (synchronousDispatcher) { diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala index 2170a8c4ba949..13e2b75972839 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala @@ -30,7 +30,7 @@ import org.apache.flink.runtime.instance.InstanceManager import org.apache.flink.runtime.jobmanager.scheduler.Scheduler import org.apache.flink.runtime.jobmanager.{JobManager, SubmittedJobGraphStore} import org.apache.flink.runtime.leaderelection.LeaderElectionService -import org.apache.flink.runtime.metrics.MetricRegistry +import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup import scala.concurrent.duration._ import scala.language.postfixOps @@ -53,7 +53,7 @@ class TestingJobManager( submittedJobGraphs : SubmittedJobGraphStore, checkpointRecoveryFactory : CheckpointRecoveryFactory, jobRecoveryTimeout : FiniteDuration, - metricRegistry : Option[MetricRegistry], + jobManagerMetricGroup : JobManagerMetricGroup, optRestAddress: Option[String]) extends JobManager( flinkConfiguration, @@ -70,6 +70,6 @@ class TestingJobManager( submittedJobGraphs, checkpointRecoveryFactory, jobRecoveryTimeout, - metricRegistry, + jobManagerMetricGroup, optRestAddress) with TestingJobManagerLike {} diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala index 1db0a8528ff5a..da753ae03cae7 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala @@ -23,7 +23,7 @@ import org.apache.flink.runtime.highavailability.HighAvailabilityServices import org.apache.flink.runtime.io.disk.iomanager.IOManager import org.apache.flink.runtime.io.network.NetworkEnvironment import org.apache.flink.runtime.memory.MemoryManager -import org.apache.flink.runtime.metrics.MetricRegistry +import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup import org.apache.flink.runtime.taskexecutor.TaskManagerConfiguration import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerLocation} @@ -40,7 +40,7 @@ class TestingTaskManager( network: NetworkEnvironment, numberOfSlots: Int, highAvailabilityServices: HighAvailabilityServices, - metricRegistry : MetricRegistry) + taskManagerMetricGroup : TaskManagerMetricGroup) extends TaskManager( config, resourceID, @@ -50,7 +50,7 @@ class TestingTaskManager( network, numberOfSlots, highAvailabilityServices, - metricRegistry) + taskManagerMetricGroup) with TestingTaskManagerLike { def this( @@ -61,7 +61,7 @@ class TestingTaskManager( network: NetworkEnvironment, numberOfSlots: Int, highAvailabilityServices: HighAvailabilityServices, - metricRegistry : MetricRegistry) { + taskManagerMetricGroup : TaskManagerMetricGroup) { this( config, ResourceID.generate(), @@ -71,6 +71,6 @@ class TestingTaskManager( network, numberOfSlots, highAvailabilityServices, - metricRegistry) + taskManagerMetricGroup) } } diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala index f78af9f953112..2de6f9e59b771 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala @@ -18,9 +18,10 @@ package org.apache.flink.runtime.testingUtils +import java.net.InetAddress import java.util -import java.util.{Collections, UUID} import java.util.concurrent._ +import java.util.{Collections, UUID} import akka.actor.{ActorRef, ActorSystem, Kill, Props} import akka.pattern.{Patterns, ask} @@ -38,13 +39,15 @@ import org.apache.flink.runtime.jobmanager.{JobManager, MemoryArchivist} import org.apache.flink.runtime.jobmaster.JobMaster import org.apache.flink.runtime.leaderretrieval.StandaloneLeaderRetrievalService import org.apache.flink.runtime.messages.TaskManagerMessages.{NotifyWhenRegisteredAtJobManager, RegisteredAtJobManager} +import org.apache.flink.runtime.metrics.{MetricRegistryImpl, MetricRegistryConfiguration} +import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup +import org.apache.flink.runtime.taskexecutor.{TaskManagerServices, TaskManagerServicesConfiguration} import org.apache.flink.runtime.taskmanager.TaskManager import org.apache.flink.runtime.testutils.TestingResourceManager import org.apache.flink.runtime.util.LeaderRetrievalUtils import org.apache.flink.runtime.{FlinkActor, LeaderSessionMessageFilter, LogMessages} -import scala.concurrent.duration.TimeUnit -import scala.concurrent.duration._ +import scala.concurrent.duration.{TimeUnit, _} import scala.concurrent.{Await, ExecutionContext, ExecutionContextExecutor} import scala.language.postfixOps @@ -266,11 +269,17 @@ object TestingUtils { resultingConfiguration.addAll(configuration) + val metricRegistry = new MetricRegistryImpl( + MetricRegistryConfiguration.fromConfiguration(configuration)) + + val taskManagerResourceId = ResourceID.generate() + val taskManager = TaskManager.startTaskManagerComponentsAndActor( resultingConfiguration, - ResourceID.generate(), + taskManagerResourceId, actorSystem, highAvailabilityServices, + metricRegistry, "localhost", None, useLocalCommunication, @@ -471,12 +480,16 @@ object TestingUtils { HighAvailabilityOptions.HA_MODE, ConfigConstants.DEFAULT_HA_MODE) + val metricRegistry = new MetricRegistryImpl( + MetricRegistryConfiguration.fromConfiguration(configuration)) + val (actor, _) = JobManager.startJobManagerActors( configuration, actorSystem, futureExecutor, ioExecutor, highAvailabilityServices, + metricRegistry, None, Some(prefix + JobMaster.JOB_MANAGER_NAME), Some(prefix + JobMaster.ARCHIVE_NAME), diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java index a78c528ba4e12..94aed2a439f27 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java @@ -30,6 +30,7 @@ import org.apache.flink.runtime.jobmanager.JobManager; import org.apache.flink.runtime.jobmanager.MemoryArchivist; import org.apache.flink.runtime.messages.JobManagerMessages; +import org.apache.flink.runtime.metrics.NoOpMetricRegistry; import org.apache.flink.runtime.taskmanager.TaskManager; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.testutils.CommonTestUtils; @@ -145,6 +146,7 @@ public void testTaskManagerProcessFailure() throws Exception { TestingUtils.defaultExecutor(), TestingUtils.defaultExecutor(), highAvailabilityServices, + new NoOpMetricRegistry(), Option.empty(), JobManager.class, MemoryArchivist.class)._1(); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHACheckpointRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHACheckpointRecoveryITCase.java index 51868af88a3e7..7c53d523a9eb0 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHACheckpointRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHACheckpointRecoveryITCase.java @@ -41,6 +41,7 @@ import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.leaderelection.TestingListener; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; +import org.apache.flink.runtime.metrics.NoOpMetricRegistry; import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.apache.flink.runtime.state.AbstractStateBackend; import org.apache.flink.runtime.state.CheckpointListener; @@ -217,6 +218,7 @@ public void testCheckpointRecoveryFailure() throws Exception { ResourceID.generate(), taskManagerSystem, highAvailabilityServices, + new NoOpMetricRegistry(), "localhost", Option.empty(), false, diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAJobGraphRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAJobGraphRecoveryITCase.java index b575dcad17250..ee37d6d7bd409 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAJobGraphRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAJobGraphRecoveryITCase.java @@ -39,6 +39,7 @@ import org.apache.flink.runtime.messages.JobManagerMessages; import org.apache.flink.runtime.messages.JobManagerMessages.LeaderSessionMessage; import org.apache.flink.runtime.messages.JobManagerMessages.SubmitJob; +import org.apache.flink.runtime.metrics.NoOpMetricRegistry; import org.apache.flink.runtime.taskmanager.TaskManager; import org.apache.flink.runtime.testingUtils.TestingCluster; import org.apache.flink.runtime.testingUtils.TestingUtils; @@ -203,6 +204,7 @@ public void testClientNonDetachedListeningBehaviour() throws Exception { ResourceID.generate(), taskManagerSystem, highAvailabilityServices, + new NoOpMetricRegistry(), "localhost", Option.empty(), false, diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureBatchRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureBatchRecoveryITCase.java index 2820dd22696e8..8e97e9d17cf0e 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureBatchRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureBatchRecoveryITCase.java @@ -37,6 +37,7 @@ import org.apache.flink.runtime.instance.AkkaActorGateway; import org.apache.flink.runtime.leaderelection.TestingListener; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; +import org.apache.flink.runtime.metrics.NoOpMetricRegistry; import org.apache.flink.runtime.taskmanager.TaskManager; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.testutils.CommonTestUtils; @@ -281,6 +282,7 @@ public void testJobManagerProcessFailure() throws Exception { ResourceID.generate(), tmActorSystem[i], highAvailabilityServices, + new NoOpMetricRegistry(), "localhost", Option.empty(), false, diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java index c70c2d5c44712..ecd0beac5727c 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java @@ -34,6 +34,7 @@ import org.apache.flink.runtime.jobmanager.JobManager; import org.apache.flink.runtime.jobmanager.MemoryArchivist; import org.apache.flink.runtime.messages.JobManagerMessages; +import org.apache.flink.runtime.metrics.NoOpMetricRegistry; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.util.NetUtils; @@ -120,6 +121,7 @@ public void testCancelingOnProcessFailure() throws Exception { TestingUtils.defaultExecutor(), TestingUtils.defaultExecutor(), highAvailabilityServices, + new NoOpMetricRegistry(), Option.empty(), JobManager.class, MemoryArchivist.class)._1(); diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java index e5131afa87357..7488b629fa375 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java @@ -34,6 +34,7 @@ import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.runtime.jobmanager.JobManager; import org.apache.flink.runtime.messages.JobManagerMessages; +import org.apache.flink.runtime.metrics.NoOpMetricRegistry; import org.apache.flink.runtime.state.memory.MemoryStateBackend; import org.apache.flink.runtime.taskmanager.TaskManager; import org.apache.flink.runtime.testingUtils.TestingJobManager; @@ -112,6 +113,7 @@ public static void setupCluster() throws Exception { TestingUtils.defaultExecutor(), TestingUtils.defaultExecutor(), highAvailabilityServices, + new NoOpMetricRegistry(), Option.empty(), Option.apply("jm"), Option.apply("arch"), @@ -133,6 +135,7 @@ public static void setupCluster() throws Exception { ResourceID.generate(), actorSystem, highAvailabilityServices, + new NoOpMetricRegistry(), "localhost", Option.apply("tm"), true, diff --git a/flink-yarn-tests/src/test/scala/org/apache/flink/yarn/TestingYarnJobManager.scala b/flink-yarn-tests/src/test/scala/org/apache/flink/yarn/TestingYarnJobManager.scala index 298cd52f3cb19..d0084b6cf8c08 100644 --- a/flink-yarn-tests/src/test/scala/org/apache/flink/yarn/TestingYarnJobManager.scala +++ b/flink-yarn-tests/src/test/scala/org/apache/flink/yarn/TestingYarnJobManager.scala @@ -30,7 +30,7 @@ import org.apache.flink.runtime.instance.InstanceManager import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore import org.apache.flink.runtime.jobmanager.scheduler.Scheduler import org.apache.flink.runtime.leaderelection.LeaderElectionService -import org.apache.flink.runtime.metrics.MetricRegistry +import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup import org.apache.flink.runtime.testingUtils.TestingJobManagerLike import scala.concurrent.duration.FiniteDuration @@ -68,7 +68,7 @@ class TestingYarnJobManager( submittedJobGraphs : SubmittedJobGraphStore, checkpointRecoveryFactory : CheckpointRecoveryFactory, jobRecoveryTimeout: FiniteDuration, - metricRegistry : Option[MetricRegistry], + jobManagerMetricGroup : JobManagerMetricGroup, optRestAddress: Option[String]) extends YarnJobManager( flinkConfiguration, @@ -85,6 +85,6 @@ class TestingYarnJobManager( submittedJobGraphs, checkpointRecoveryFactory, jobRecoveryTimeout, - metricRegistry, + jobManagerMetricGroup, optRestAddress) with TestingJobManagerLike {} diff --git a/flink-yarn-tests/src/test/scala/org/apache/flink/yarn/TestingYarnTaskManager.scala b/flink-yarn-tests/src/test/scala/org/apache/flink/yarn/TestingYarnTaskManager.scala index a03f3655e993d..228eaaa0a74b0 100644 --- a/flink-yarn-tests/src/test/scala/org/apache/flink/yarn/TestingYarnTaskManager.scala +++ b/flink-yarn-tests/src/test/scala/org/apache/flink/yarn/TestingYarnTaskManager.scala @@ -23,7 +23,7 @@ import org.apache.flink.runtime.highavailability.HighAvailabilityServices import org.apache.flink.runtime.io.disk.iomanager.IOManager import org.apache.flink.runtime.io.network.NetworkEnvironment import org.apache.flink.runtime.memory.MemoryManager -import org.apache.flink.runtime.metrics.MetricRegistry +import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup import org.apache.flink.runtime.taskexecutor.TaskManagerConfiguration import org.apache.flink.runtime.taskmanager.TaskManagerLocation import org.apache.flink.runtime.testingUtils.TestingTaskManagerLike @@ -52,7 +52,7 @@ class TestingYarnTaskManager( network: NetworkEnvironment, numberOfSlots: Int, highAvailabilityServices: HighAvailabilityServices, - metricRegistry : MetricRegistry) + taskManagerMetricGroup : TaskManagerMetricGroup) extends YarnTaskManager( config, resourceID, @@ -62,7 +62,7 @@ class TestingYarnTaskManager( network, numberOfSlots, highAvailabilityServices, - metricRegistry) + taskManagerMetricGroup) with TestingTaskManagerLike { object YarnTaskManager { diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java index c101b759d0676..95ba154a16da3 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java @@ -36,6 +36,8 @@ import org.apache.flink.runtime.jobmanager.JobManager; import org.apache.flink.runtime.jobmanager.MemoryArchivist; import org.apache.flink.runtime.jobmaster.JobMaster; +import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.process.ProcessReaper; import org.apache.flink.runtime.security.SecurityConfiguration; import org.apache.flink.runtime.security.SecurityUtils; @@ -219,6 +221,7 @@ protected int runApplicationMaster(Configuration config) { ActorSystem actorSystem = null; WebMonitor webMonitor = null; HighAvailabilityServices highAvailabilityServices = null; + MetricRegistryImpl metricRegistry = null; int numberProcessors = Hardware.getNumberCPUCores(); @@ -357,6 +360,11 @@ protected int runApplicationMaster(Configuration config) { new ScheduledExecutorServiceAdapter(futureExecutor), LOG); + metricRegistry = new MetricRegistryImpl( + MetricRegistryConfiguration.fromConfiguration(config)); + + metricRegistry.startQueryService(actorSystem, null); + // 2: the JobManager LOG.debug("Starting JobManager actor"); @@ -367,6 +375,7 @@ protected int runApplicationMaster(Configuration config) { futureExecutor, ioExecutor, highAvailabilityServices, + metricRegistry, webMonitor == null ? Option.empty() : Option.apply(webMonitor.getRestAddress()), new Some<>(JobMaster.JOB_MANAGER_NAME), Option.empty(), @@ -455,6 +464,10 @@ protected int runApplicationMaster(Configuration config) { } } + if (metricRegistry != null) { + metricRegistry.shutdown(); + } + org.apache.flink.runtime.concurrent.Executors.gracefulShutdown( AkkaUtils.getTimeout(config).toMillis(), TimeUnit.MILLISECONDS, diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java index b32d25caa7ed7..6feb287a5da9f 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java @@ -28,7 +28,7 @@ import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.resourcemanager.JobLeaderIdService; import org.apache.flink.runtime.resourcemanager.ResourceManager; import org.apache.flink.runtime.resourcemanager.ResourceManagerConfiguration; @@ -116,7 +116,7 @@ public YarnResourceManager( HighAvailabilityServices highAvailabilityServices, HeartbeatServices heartbeatServices, SlotManager slotManager, - MetricRegistry metricRegistry, + MetricRegistryImpl metricRegistry, JobLeaderIdService jobLeaderIdService, FatalErrorHandler fatalErrorHandler) { super( diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnJobClusterEntrypoint.java b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnJobClusterEntrypoint.java index e1efb54f706a5..439fdf32af56c 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnJobClusterEntrypoint.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnJobClusterEntrypoint.java @@ -24,7 +24,7 @@ import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.resourcemanager.ResourceManager; import org.apache.flink.runtime.resourcemanager.ResourceManagerConfiguration; import org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServices; @@ -78,7 +78,7 @@ protected ResourceManager createResourceManager( RpcService rpcService, HighAvailabilityServices highAvailabilityServices, HeartbeatServices heartbeatServices, - MetricRegistry metricRegistry, + MetricRegistryImpl metricRegistry, FatalErrorHandler fatalErrorHandler) throws Exception { final ResourceManagerConfiguration rmConfiguration = ResourceManagerConfiguration.fromConfiguration(configuration); final ResourceManagerRuntimeServicesConfiguration rmServicesConfiguration = ResourceManagerRuntimeServicesConfiguration.fromConfiguration(configuration); diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnSessionClusterEntrypoint.java b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnSessionClusterEntrypoint.java index 042644b6f4e7c..a13f62c4b42ae 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnSessionClusterEntrypoint.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnSessionClusterEntrypoint.java @@ -23,7 +23,7 @@ import org.apache.flink.runtime.entrypoint.SessionClusterEntrypoint; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.resourcemanager.ResourceManager; import org.apache.flink.runtime.resourcemanager.ResourceManagerConfiguration; import org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServices; @@ -68,7 +68,7 @@ protected ResourceManager createResourceManager( RpcService rpcService, HighAvailabilityServices highAvailabilityServices, HeartbeatServices heartbeatServices, - MetricRegistry metricRegistry, + MetricRegistryImpl metricRegistry, FatalErrorHandler fatalErrorHandler) throws Exception { final ResourceManagerConfiguration rmConfiguration = ResourceManagerConfiguration.fromConfiguration(configuration); final ResourceManagerRuntimeServicesConfiguration rmServicesConfiguration = ResourceManagerRuntimeServicesConfiguration.fromConfiguration(configuration); diff --git a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnJobManager.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnJobManager.scala index 590916035201f..6b439bda9b9f8 100644 --- a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnJobManager.scala +++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnJobManager.scala @@ -34,7 +34,8 @@ import org.apache.flink.runtime.instance.InstanceManager import org.apache.flink.runtime.jobmanager.scheduler.{Scheduler => FlinkScheduler} import org.apache.flink.runtime.jobmanager.{JobManager, SubmittedJobGraphStore} import org.apache.flink.runtime.leaderelection.LeaderElectionService -import org.apache.flink.runtime.metrics.MetricRegistry +import org.apache.flink.runtime.metrics.MetricRegistryImpl +import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup import org.apache.flink.yarn.configuration.YarnConfigOptions import scala.concurrent.duration._ @@ -72,7 +73,7 @@ class YarnJobManager( submittedJobGraphs : SubmittedJobGraphStore, checkpointRecoveryFactory : CheckpointRecoveryFactory, jobRecoveryTimeout: FiniteDuration, - metricsRegistry: Option[MetricRegistry], + jobManagerMetricGroup: JobManagerMetricGroup, optRestAddress: Option[String]) extends ContaineredJobManager( flinkConfiguration, @@ -89,7 +90,7 @@ class YarnJobManager( submittedJobGraphs, checkpointRecoveryFactory, jobRecoveryTimeout, - metricsRegistry, + jobManagerMetricGroup, optRestAddress) { val DEFAULT_YARN_HEARTBEAT_DELAY: FiniteDuration = 5 seconds diff --git a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala index e37ff6f37d641..615466db0e8be 100644 --- a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala +++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala @@ -23,7 +23,7 @@ import org.apache.flink.runtime.highavailability.HighAvailabilityServices import org.apache.flink.runtime.io.disk.iomanager.IOManager import org.apache.flink.runtime.io.network.NetworkEnvironment import org.apache.flink.runtime.memory.MemoryManager -import org.apache.flink.runtime.metrics.MetricRegistry +import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup import org.apache.flink.runtime.taskexecutor.TaskManagerConfiguration import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerLocation} @@ -39,7 +39,7 @@ class YarnTaskManager( network: NetworkEnvironment, numberOfSlots: Int, highAvailabilityServices: HighAvailabilityServices, - metricRegistry : MetricRegistry) + taskManagerMetricGroup: TaskManagerMetricGroup) extends TaskManager( config, resourceID, @@ -49,7 +49,7 @@ class YarnTaskManager( network, numberOfSlots, highAvailabilityServices, - metricRegistry) { + taskManagerMetricGroup) { override def handleMessage: Receive = { super.handleMessage