[ https://issues.apache.org/jira/browse/FLINK-4361?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15506714#comment-15506714 ]
ASF GitHub Bot commented on FLINK-4361: --------------------------------------- Github user tillrohrmann commented on a diff in the pull request: https://github.com/apache/flink/pull/2472#discussion_r79622945 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/Future.java --- @@ -0,0 +1,156 @@ +/* + * 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.concurrent; + +import java.util.concurrent.CancellationException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +/** + * Flink's basic future abstraction. A future represents an asynchronous operation whose result + * will be contained in this instance upon completion. + * + * @param <T> type of the future's result + */ +public interface Future<T> { + + /** + * Checks if the future has been completed. A future is completed, if the result has been + * delivered. + * + * @return true if the future is completed; otherwise false + */ + boolean isDone(); --- End diff -- That's because I sticked to Java 8's `CompletableFuture` implementation where it is named the same. I'm not super happy with the naming either. But I also see the benefit of sticking to Java 8's `CompletableFuture` interface. This will allow us to easily replace it once we switch to Java 8. I think we have to decide whether we want to stick to Java 8's `CompletableFuture` or not. In the latter case we can rename other methods as well. > Introduce Flink's own future abstraction > ---------------------------------------- > > Key: FLINK-4361 > URL: https://issues.apache.org/jira/browse/FLINK-4361 > Project: Flink > Issue Type: Sub-task > Components: Distributed Coordination > Reporter: Stephan Ewen > Assignee: Till Rohrmann > > In order to keep the abstraction Scala Independent, we should not rely on > Scala Futures -- This message was sent by Atlassian JIRA (v6.3.4#6332)