forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-19174][metrics] Fix idle and backpressured time accuracy on lo…
…ng sleeps In particularly, if task is idling forever, as there are no new records incomming previous version would report idleTime as 0% and busyTime as 100% which is incorrect. In this version, idleTime metric is aware that idling period has started and can take that into account when updating it's value.
- Loading branch information
Showing
8 changed files
with
205 additions
and
30 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
82 changes: 82 additions & 0 deletions
82
flink-runtime/src/main/java/org/apache/flink/runtime/metrics/TimerGauge.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,82 @@ | ||
/* | ||
* 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:https://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.metrics.Gauge; | ||
import org.apache.flink.metrics.View; | ||
import org.apache.flink.util.clock.Clock; | ||
import org.apache.flink.util.clock.SystemClock; | ||
|
||
/** | ||
* {@link TimerGauge} measures how much time is spent in a given state, with entry into that state | ||
* being signaled by {@link #markStart()}. Measuring is stopped by {@link #markEnd()}. This class in | ||
* particularly takes care of the case, when {@link #update()} is called when some measurement | ||
* started but has not yet finished. For example even if next {@link #markEnd()} call is expected to | ||
* happen in a couple of hours, the returned value will account for this ongoing measurement. | ||
*/ | ||
public class TimerGauge implements Gauge<Long>, View { | ||
private final Clock clock; | ||
|
||
private long previousCount; | ||
private long currentCount; | ||
private long currentMeasurementStart; | ||
|
||
public TimerGauge() { | ||
this(SystemClock.getInstance()); | ||
} | ||
|
||
public TimerGauge(Clock clock) { | ||
this.clock = clock; | ||
} | ||
|
||
public synchronized void markStart() { | ||
if (currentMeasurementStart == 0) { | ||
currentMeasurementStart = clock.absoluteTimeMillis(); | ||
} | ||
} | ||
|
||
public synchronized void markEnd() { | ||
if (currentMeasurementStart != 0) { | ||
currentCount += clock.absoluteTimeMillis() - currentMeasurementStart; | ||
currentMeasurementStart = 0; | ||
} | ||
} | ||
|
||
@Override | ||
public synchronized void update() { | ||
if (currentMeasurementStart != 0) { | ||
long now = clock.absoluteTimeMillis(); | ||
currentCount += now - currentMeasurementStart; | ||
currentMeasurementStart = now; | ||
} | ||
previousCount = Math.max(Math.min(currentCount / UPDATE_INTERVAL_SECONDS, 1000), 0); | ||
currentCount = 0; | ||
} | ||
|
||
@Override | ||
public synchronized Long getValue() { | ||
return previousCount; | ||
} | ||
|
||
@VisibleForTesting | ||
public synchronized long getCount() { | ||
return currentCount; | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
86 changes: 86 additions & 0 deletions
86
flink-runtime/src/test/java/org/apache/flink/runtime/metrics/TimerGaugeTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,86 @@ | ||
/* | ||
* 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:https://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. | ||
*/ | ||
|
||
// ---------------------------------------------------------------------------- | ||
// This class is largely adapted from "com.google.common.base.Preconditions", | ||
// which is part of the "Guava" library. | ||
// | ||
// Because of frequent issues with dependency conflicts, this class was | ||
// added to the Flink code base to reduce dependency on Guava. | ||
// ---------------------------------------------------------------------------- | ||
|
||
package org.apache.flink.runtime.metrics; | ||
|
||
import org.apache.flink.metrics.View; | ||
import org.apache.flink.util.clock.ManualClock; | ||
|
||
import org.junit.Test; | ||
|
||
import java.util.concurrent.TimeUnit; | ||
|
||
import static org.hamcrest.Matchers.greaterThanOrEqualTo; | ||
import static org.hamcrest.Matchers.is; | ||
import static org.junit.Assert.assertThat; | ||
|
||
/** Tests for {@link TimerGauge}. */ | ||
public class TimerGaugeTest { | ||
private static final long SLEEP = 10; | ||
|
||
@Test | ||
public void testBasicUsage() { | ||
ManualClock clock = new ManualClock(42_000_000); | ||
TimerGauge gauge = new TimerGauge(clock); | ||
|
||
gauge.update(); | ||
assertThat(gauge.getValue(), is(0L)); | ||
|
||
gauge.markStart(); | ||
clock.advanceTime(SLEEP, TimeUnit.MILLISECONDS); | ||
gauge.markEnd(); | ||
gauge.update(); | ||
|
||
assertThat(gauge.getValue(), greaterThanOrEqualTo(SLEEP / View.UPDATE_INTERVAL_SECONDS)); | ||
} | ||
|
||
@Test | ||
public void testUpdateWithoutMarkingEnd() { | ||
ManualClock clock = new ManualClock(42_000_000); | ||
TimerGauge gauge = new TimerGauge(clock); | ||
|
||
gauge.markStart(); | ||
clock.advanceTime(SLEEP, TimeUnit.MILLISECONDS); | ||
gauge.update(); | ||
|
||
assertThat(gauge.getValue(), greaterThanOrEqualTo(SLEEP / View.UPDATE_INTERVAL_SECONDS)); | ||
} | ||
|
||
@Test | ||
public void testGetWithoutUpdate() { | ||
ManualClock clock = new ManualClock(42_000_000); | ||
TimerGauge gauge = new TimerGauge(clock); | ||
|
||
gauge.markStart(); | ||
clock.advanceTime(SLEEP, TimeUnit.MILLISECONDS); | ||
|
||
assertThat(gauge.getValue(), is(0L)); | ||
|
||
gauge.markEnd(); | ||
|
||
assertThat(gauge.getValue(), is(0L)); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.