afedulov commented on a change in pull request #15054:
URL: https://github.com/apache/flink/pull/15054#discussion_r602915782



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/stats/TaskStatsRequestCoordinator.java
##########
@@ -0,0 +1,323 @@
+/*
+ * 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.webmonitor.stats;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.guava18.com.google.common.collect.Maps;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.util.ArrayDeque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/**
+ * Encapsulates the common functionality for requesting statistics from 
individual tasks and
+ * combining their responses.
+ *
+ * @param <T> Type of the statistics to be gathered.
+ * @param <V> Type of the combined response.
+ */
+public class TaskStatsRequestCoordinator<T, V> {
+    protected final Logger log = LoggerFactory.getLogger(getClass());
+
+    protected final int numGhostSampleIds = 10;
+
+    protected final Object lock = new Object();
+
+    /** Executor used to run the futures. */
+    protected final Executor executor;
+
+    /** Request time out of a triggered task stats request. */
+    protected final Time requestTimeout;
+
+    /** In progress samples. */
+    @GuardedBy("lock")
+    protected final Map<Integer, PendingStatsRequest<T, V>> pendingRequests = 
new HashMap<>();
+
+    /** A list of recent request IDs to identify late messages vs. invalid 
ones. */
+    protected final ArrayDeque<Integer> recentPendingRequestIds =
+            new ArrayDeque<>(numGhostSampleIds);
+
+    /** Sample ID counter. */
+    @GuardedBy("lock")
+    protected int requestIdCounter;
+
+    /** Flag indicating whether the coordinator is still running. */
+    @GuardedBy("lock")
+    protected boolean isShutDown;
+
+    /**
+     * Creates a new coordinator for the cluster.
+     *
+     * @param executor Used to execute the futures.
+     * @param requestTimeout Request time out of a triggered task stats 
request.
+     */
+    public TaskStatsRequestCoordinator(Executor executor, long requestTimeout) 
{
+        checkArgument(requestTimeout >= 0L, "The request timeout must be 
non-negative.");
+        this.executor = Preconditions.checkNotNull(executor);
+        this.requestTimeout = Time.milliseconds(requestTimeout);
+    }
+
+    /**
+     * Handles the failed stats response by canceling the corresponding 
unfinished pending request.
+     *
+     * @param requestId ID of the request to cancel.
+     * @param cause Cause of the cancelling (can be <code>null</code>).
+     */
+    public void handleFailedResponse(int requestId, @Nullable Throwable cause) 
{
+        synchronized (lock) {
+            if (isShutDown) {
+                return;
+            }
+
+            PendingStatsRequest<T, V> pendingRequest = 
pendingRequests.remove(requestId);
+            if (pendingRequest != null) {
+                if (cause != null) {
+                    log.info("Cancelling request " + requestId, cause);
+                } else {
+                    log.info("Cancelling request {}", requestId);
+                }
+
+                pendingRequest.discard(cause);
+                rememberRecentRequestId(requestId);
+            }
+        }
+    }
+
+    /**
+     * Shuts down the coordinator.
+     *
+     * <p>After shut down, no further operations are executed.
+     */
+    public void shutDown() {
+        synchronized (lock) {
+            if (!isShutDown) {
+                log.info("Shutting down task stats request coordinator.");
+
+                for (PendingStatsRequest<T, V> pending : 
pendingRequests.values()) {
+                    pending.discard(new RuntimeException("Shut down"));
+                }
+
+                pendingRequests.clear();
+                recentPendingRequestIds.clear();
+
+                isShutDown = true;
+            }
+        }
+    }
+
+    /**
+     * Handles the successfully returned task stats response by collecting the 
corresponding subtask
+     * samples.
+     *
+     * @param requestId ID of the request.
+     * @param executionId ID of the sampled task.
+     * @param result Result of stats request returned by an individual task.
+     * @throws IllegalStateException If unknown request ID and not recently 
finished or cancelled
+     *     sample.
+     */
+    public void handleSuccessfulResponse(int requestId, ExecutionAttemptID 
executionId, T result) {
+
+        synchronized (lock) {
+            if (isShutDown) {
+                return;
+            }
+
+            if (log.isDebugEnabled()) {
+                log.debug("Collecting stats sample {} of task {}", requestId, 
executionId);
+            }
+
+            PendingStatsRequest<T, V> pending = pendingRequests.get(requestId);
+
+            if (pending != null) {
+                pending.collectTaskStats(executionId, result);
+
+                // Publish the sample
+                if (pending.isComplete()) {
+                    pendingRequests.remove(requestId);
+                    rememberRecentRequestId(requestId);
+
+                    pending.completePromiseAndDiscard();
+                }
+            } else if (recentPendingRequestIds.contains(requestId)) {
+                if (log.isDebugEnabled()) {
+                    log.debug("Received late stats sample {} of task {}", 
requestId, executionId);
+                }
+            } else {
+                if (log.isDebugEnabled()) {
+                    log.debug(String.format("Unknown request ID %d.", 
requestId));
+                }
+            }
+        }
+    }
+
+    private void rememberRecentRequestId(int sampleId) {
+        if (recentPendingRequestIds.size() >= numGhostSampleIds) {
+            recentPendingRequestIds.removeFirst();
+        }
+        recentPendingRequestIds.addLast(sampleId);
+    }
+
+    @VisibleForTesting
+    public int getNumberOfPendingRequests() {
+        synchronized (lock) {
+            return pendingRequests.size();
+        }
+    }
+
+    // ------------------------------------------------------------------------
+
+    /**
+     * A pending task stats request, which collects samples from individual 
tasks and completes the
+     * response future upon gathering all of of them.
+     *
+     * <p>Has to be accessed in lock scope.

Review comment:
       added the annotation.




-- 
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]


Reply via email to