zentol commented on a change in pull request #14526: URL: https://github.com/apache/flink/pull/14526#discussion_r551323072
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/TimeGauge.java ########## @@ -0,0 +1,80 @@ +/* + * 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. + */ + +// ---------------------------------------------------------------------------- +// 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.groups; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.View; + +/** + * {@link TimeGauge} encapsulate logic of measuring time rates (like idleTime ms/s). In particularly + * it takes care of the case, when single measurement (as defined by {@link #markStart()} and {@link + * #markEnd()}), crosses the boundary of single {@link #update()} interval. Thanks to the {@link + * #markStart()} {@link TimeGauge} can correctly calculate next value in {@link #update()}, even if + * this measurement hasn't not ended ({@link #markEnd()}). Review comment: It is not obvious from this description why this is important. Overall I would wager that it would be difficult to accurately state what this thing does based on the javadocs. Maybe something alone the lines of "...measures how much time is spent in a given state, with entry into that state being signaled by #markStart()" could help with that. Then it also makes it obvious why it is important to account for the case that markEnd has not been called yet. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/TimeGauge.java ########## @@ -0,0 +1,80 @@ +/* + * 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. + */ + +// ---------------------------------------------------------------------------- +// 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.groups; Review comment: move to the metrics package? ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/TimeGauge.java ########## @@ -0,0 +1,80 @@ +/* + * 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. + */ + +// ---------------------------------------------------------------------------- +// This class is largely adapted from "com.google.common.base.Preconditions", Review comment: Wrong class? Surely there is no timer-like class in `Preconditions`. ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/TimeGaugeTest.java ########## @@ -0,0 +1,80 @@ +/* + * 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. + */ + +// ---------------------------------------------------------------------------- +// 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.groups; + +import org.apache.flink.metrics.View; + +import org.junit.Test; + +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThat; + +/** Tests for {@link TimeGauge}. */ +public class TimeGaugeTest { + private static final long SLEEP = 10; + + @Test + public void testBasicUsage() throws Exception { + TimeGauge gauge = new TimeGauge(); + + gauge.update(); + assertThat(gauge.getValue(), is(0L)); + + gauge.markStart(); + Thread.sleep(SLEEP); Review comment: You could make testing easier by making the time measurements be based on a `Clock`. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/TimeGauge.java ########## @@ -0,0 +1,80 @@ +/* + * 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. + */ + +// ---------------------------------------------------------------------------- +// 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.groups; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.View; + +/** + * {@link TimeGauge} encapsulate logic of measuring time rates (like idleTime ms/s). In particularly Review comment: This question may come a bit late, but why are we exposing this as msPerSecond instead of a percentage? I understand they are equivalent (1000 == 100%), but I'd think a percentage would be easier to understand? ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/TimeGauge.java ########## @@ -0,0 +1,80 @@ +/* + * 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. + */ + +// ---------------------------------------------------------------------------- +// 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.groups; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.View; + +/** + * {@link TimeGauge} encapsulate logic of measuring time rates (like idleTime ms/s). In particularly + * it takes care of the case, when single measurement (as defined by {@link #markStart()} and {@link + * #markEnd()}), crosses the boundary of single {@link #update()} interval. Thanks to the {@link + * #markStart()} {@link TimeGauge} can correctly calculate next value in {@link #update()}, even if + * this measurement hasn't not ended ({@link #markEnd()}). + */ +public class TimeGauge implements Gauge<Long>, View { + private long previousCount; + private long currentCount; + private long lastDifference; + private long currentMeasurementStart; + + public synchronized void markStart() { + if (currentMeasurementStart == 0) { + currentMeasurementStart = System.currentTimeMillis(); + } + } + + public synchronized void markEnd() { + if (currentMeasurementStart != 0) { + currentCount += System.currentTimeMillis() - currentMeasurementStart; + currentMeasurementStart = 0; + } + } + + @Override + public synchronized void update() { + if (currentMeasurementStart != 0) { + long now = System.currentTimeMillis(); + currentCount += now - currentMeasurementStart; + currentMeasurementStart = now; + } + lastDifference = (currentCount - previousCount) / UPDATE_INTERVAL_SECONDS; + lastDifference = Math.max(Math.min(lastDifference, 1000), 0); + previousCount = currentCount; Review comment: Couldn't we simplify this be setting `currentCount` to 0? ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: [email protected]
