pnowojski commented on code in PR #25167:
URL: https://github.com/apache/flink/pull/25167#discussion_r1719583700


##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/util/PausableRelativeClock.java:
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.streaming.api.operators.util;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.runtime.metrics.TimerGauge;
+import org.apache.flink.util.clock.Clock;
+import org.apache.flink.util.clock.RelativeClock;
+
+import javax.annotation.concurrent.ThreadSafe;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A {@link RelativeClock} whose time progress with respect to the wall clock 
can be paused and
+ * un-paused. It can be paused multiple times. If it is paused N times, it has 
to be un-paused also
+ * N times to resume progress.
+ */
+@Internal
+@ThreadSafe
+public class PausableRelativeClock implements RelativeClock, 
TimerGauge.StartStopListener {
+    private final Clock baseClock;
+
+    private long accumulativeBlockedNanoTime;
+    private long currentBlockedNanoTimeStart;
+    /** How many times this clock has been paused. */
+    private long pausedCounter;
+
+    public PausableRelativeClock(Clock baseClock) {
+        this.baseClock = baseClock;
+    }
+
+    @Override
+    public long relativeTimeMillis() {
+        return relativeTimeNanos() / 1_000_000;
+    }
+
+    @Override
+    public long relativeTimeNanos() {
+        return baseClock.relativeTimeNanos() - getBlockedTime();
+    }
+
+    public synchronized void pause() {
+        if (pausedCounter == 0) {
+            currentBlockedNanoTimeStart = baseClock.relativeTimeNanos();
+        }
+        pausedCounter++;
+    }
+
+    public synchronized void unPause() {
+        checkState(pausedCounter >= 1);
+        pausedCounter--;
+        if (pausedCounter == 0) {
+            accumulativeBlockedNanoTime +=
+                    baseClock.relativeTimeNanos() - 
currentBlockedNanoTimeStart;
+        }
+    }
+
+    private synchronized long getBlockedTime() {
+        return accumulativeBlockedNanoTime + getCurrentBlockedNanoTime();
+    }
+
+    private long getCurrentBlockedNanoTime() {
+        return pausedCounter == 0 ? 0 : baseClock.relativeTimeNanos() - 
currentBlockedNanoTimeStart;
+    }
+

Review Comment:
   I've tried to inline it, and it was even more difficult for me to understand 
what's going on. Can you look at the updated version as the compromise? I've 
inlined `getCurrentBlockedNanoTime`, as indeed that wasn't helping much, 
re-ordered the methods, added some comments, and removed second `nanoTime` call.
   
   PTAL at the most recent fixup: 
https://github.com/apache/flink/pull/25167/commits/e2a1cd4f4e191bb0d85c848f7fd1c02f556a8211



-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to