GJL commented on a change in pull request #6785: [FLINK-10309][rest] Before shutting down cluster, wait for asynchronous operations URL: https://github.com/apache/flink/pull/6785#discussion_r221953953
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/async/CompletedOperationCache.java ########## @@ -0,0 +1,202 @@ +/* + * 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.rest.handler.async; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.types.Either; +import org.apache.flink.util.AutoCloseableAsync; +import org.apache.flink.util.Preconditions; + +import org.apache.flink.shaded.guava18.com.google.common.base.Ticker; +import org.apache.flink.shaded.guava18.com.google.common.cache.Cache; +import org.apache.flink.shaded.guava18.com.google.common.cache.CacheBuilder; +import org.apache.flink.shaded.guava18.com.google.common.cache.RemovalListener; + +import javax.annotation.Nullable; +import javax.annotation.concurrent.ThreadSafe; + +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + +/** + * Cache to manage ongoing operations. + * + * <p>The cache allows to register ongoing operations by calling + * {@link #registerOngoingOperation(K, CompletableFuture)}, where the + * {@code CompletableFuture} contains the operation result. Completed operations will be + * removed from the cache automatically after a fixed timeout. + */ +@ThreadSafe +class CompletedOperationCache<K extends OperationKey, R> implements AutoCloseableAsync { + + private static final long COMPLETED_OPERATION_RESULT_CACHE_DURATION_SECONDS = 300L; + + /** + * In-progress asynchronous operations. + */ + private final Map<K, ResultAccessTracker<R>> registeredOperationTriggers = new ConcurrentHashMap<>(); + + /** + * Caches the result of completed operations. + */ + private final Cache<K, ResultAccessTracker<R>> completedOperations; + + CompletedOperationCache() { + this(Ticker.systemTicker()); + } + + @VisibleForTesting + CompletedOperationCache(final Ticker ticker) { + completedOperations = CacheBuilder.newBuilder() + .expireAfterWrite(COMPLETED_OPERATION_RESULT_CACHE_DURATION_SECONDS, TimeUnit.SECONDS) + .removalListener((RemovalListener<K, ResultAccessTracker<R>>) removalNotification -> { + if (removalNotification.wasEvicted()) { + Preconditions.checkState(removalNotification.getValue() != null); + removalNotification.getValue().markAccessed(); Review comment: Exactly, we might evict a result while we wait for it to be accessed. I added a comment and a log statement if items get evicted. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected] With regards, Apache Git Services
