dmvk commented on a change in pull request #17735:
URL: https://github.com/apache/flink/pull/17735#discussion_r748187956



##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerTest.java
##########
@@ -486,6 +492,101 @@ public void testNumRestartsMetric() throws Exception {
         assertThat(numRestartsMetric.getValue(), is(1));
     }
 
+    @Test
+    public void testStatusMetrics() throws Exception {
+        final CompletableFuture<UpTimeGauge> upTimeMetricFuture = new 
CompletableFuture<>();
+        final CompletableFuture<DownTimeGauge> downTimeMetricFuture = new 
CompletableFuture<>();
+        final CompletableFuture<RestartTimeGauge> restartTimeMetricFuture =
+                new CompletableFuture<>();
+        final MetricRegistry metricRegistry =
+                TestingMetricRegistry.builder()
+                        .setRegisterConsumer(
+                                (metric, name, group) -> {
+                                    switch (name) {
+                                        case UpTimeGauge.METRIC_NAME:
+                                            
upTimeMetricFuture.complete((UpTimeGauge) metric);
+                                            break;
+                                        case DownTimeGauge.METRIC_NAME:
+                                            
downTimeMetricFuture.complete((DownTimeGauge) metric);
+                                            break;
+                                        case RestartTimeGauge.METRIC_NAME:
+                                            restartTimeMetricFuture.complete(
+                                                    (RestartTimeGauge) metric);
+                                            break;
+                                    }
+                                })
+                        .build();
+
+        final JobGraph jobGraph = createJobGraph();
+
+        final DefaultDeclarativeSlotPool declarativeSlotPool =
+                createDeclarativeSlotPool(jobGraph.getJobID());
+
+        final Configuration configuration = new Configuration();
+        configuration.set(JobManagerOptions.MIN_PARALLELISM_INCREASE, 1);
+        configuration.set(JobManagerOptions.RESOURCE_WAIT_TIMEOUT, 
Duration.ofMillis(10L));
+
+        final AdaptiveScheduler scheduler =
+                new AdaptiveSchedulerBuilder(jobGraph, 
singleThreadMainThreadExecutor)
+                        .setJobMasterConfiguration(configuration)
+                        .setJobManagerJobMetricGroup(
+                                
JobManagerMetricGroup.createJobManagerMetricGroup(
+                                                metricRegistry, "localhost")
+                                        .addJob(new JobID(), "jobName"))
+                        .setDeclarativeSlotPool(declarativeSlotPool)
+                        .build();
+
+        final UpTimeGauge upTimeGauge = upTimeMetricFuture.get();
+        final DownTimeGauge downTimeGauge = downTimeMetricFuture.get();
+        final RestartTimeGauge restartTimeGauge = 
restartTimeMetricFuture.get();
+
+        final SubmissionBufferingTaskManagerGateway taskManagerGateway =
+                new SubmissionBufferingTaskManagerGateway(1 + PARALLELISM);
+
+        taskManagerGateway.setCancelConsumer(createCancelConsumer(scheduler));
+
+        singleThreadMainThreadExecutor.execute(
+                () -> {
+                    scheduler.startScheduling();
+
+                    offerSlots(
+                            declarativeSlotPool,
+                            createSlotOffersForResourceRequirements(
+                                    
ResourceCounter.withResource(ResourceProfile.UNKNOWN, 1)),
+                            taskManagerGateway);
+                });
+
+        // wait for the first task submission
+        taskManagerGateway.waitForSubmissions(1, Duration.ofSeconds(5));
+
+        // sleep a bit to ensure uptime is > 0
+        Thread.sleep(10L);
+
+        assertThat(upTimeGauge.getValue(), greaterThan(0L));
+        assertThat(downTimeGauge.getValue(), is(0L));
+        assertThat(restartTimeGauge.getValue(), is(0L));
+
+        singleThreadMainThreadExecutor.execute(
+                () -> {
+                    // offer more slots, which will cause a restart in order 
to scale up
+                    offerSlots(
+                            declarativeSlotPool,
+                            createSlotOffersForResourceRequirements(
+                                    
ResourceCounter.withResource(ResourceProfile.UNKNOWN, 1)),
+                            taskManagerGateway);
+                });
+
+        // wait for the second task submissions
+        taskManagerGateway.waitForSubmissions(2, Duration.ofSeconds(5));
+
+        // sleep a bit to ensure uptime is > 0

Review comment:
       What do you think about passing a `Clock` instance to Adaptive scheduler 
instead of relying on `System.currentTimeMillis()` to simplify the test?

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/JobStatusStore.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.scheduler;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.JobStatus;
+import org.apache.flink.runtime.executiongraph.JobStatusListener;
+import org.apache.flink.runtime.executiongraph.JobStatusProvider;
+
+/** Listens for and exposes the current job state and state timestamps. */
+public class JobStatusStore implements JobStatusListener, JobStatusProvider {

Review comment:
       What do you think about using this for storing timestamps inside the 
execution graph as well? It seems to be duplicating the logic (even though it's 
pretty simple).

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveScheduler.java
##########
@@ -392,10 +401,16 @@ private void newResourcesAvailable(Collection<? extends 
PhysicalSlot> physicalSl
                 "newResourcesAvailable");
     }
 
-    private void registerMetrics() {
+    private void registerMetrics(JobStatusProvider jobStatusProvider) {

Review comment:
       Can we make the `SchedulerBase#registerJobMetrics` reusable instead? It 
seems to be doing the same thing (execution graph also implements 
`JobStatusProvider`).




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to