From da8fae2e13ebe848f4c8e31c5bad69c464b03957 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 3 Aug 2016 19:31:34 +0200 Subject: [PATCH 01/50] [FLINK-4346] [rpc] Add new RPC abstraction --- flink-runtime/pom.xml | 5 + .../flink/runtime/rpc/MainThreadExecutor.java | 54 +++ .../apache/flink/runtime/rpc/RpcEndpoint.java | 182 ++++++++++ .../apache/flink/runtime/rpc/RpcGateway.java | 25 ++ .../apache/flink/runtime/rpc/RpcMethod.java | 35 ++ .../apache/flink/runtime/rpc/RpcService.java | 74 ++++ .../apache/flink/runtime/rpc/RpcTimeout.java | 34 ++ .../flink/runtime/rpc/akka/AkkaGateway.java | 29 ++ .../runtime/rpc/akka/AkkaRpcService.java | 145 ++++++++ .../flink/runtime/rpc/akka/BaseAkkaActor.java | 50 +++ .../runtime/rpc/akka/BaseAkkaGateway.java | 41 +++ .../akka/jobmaster/JobMasterAkkaActor.java | 58 ++++ .../akka/jobmaster/JobMasterAkkaGateway.java | 57 +++ .../rpc/akka/messages/CallableMessage.java | 33 ++ .../runtime/rpc/akka/messages/CancelTask.java | 36 ++ .../rpc/akka/messages/ExecuteTask.java | 36 ++ .../messages/RegisterAtResourceManager.java | 36 ++ .../rpc/akka/messages/RegisterJobMaster.java | 36 ++ .../rpc/akka/messages/RequestSlot.java | 37 ++ .../rpc/akka/messages/RunnableMessage.java | 31 ++ .../messages/UpdateTaskExecutionState.java | 37 ++ .../ResourceManagerAkkaActor.java | 65 ++++ .../ResourceManagerAkkaGateway.java | 67 ++++ .../taskexecutor/TaskExecutorAkkaActor.java | 77 +++++ .../taskexecutor/TaskExecutorAkkaGateway.java | 59 ++++ .../runtime/rpc/jobmaster/JobMaster.java | 249 +++++++++++++ .../rpc/jobmaster/JobMasterGateway.java | 45 +++ .../JobMasterRegistration.java | 35 ++ .../resourcemanager/RegistrationResponse.java | 43 +++ .../rpc/resourcemanager/ResourceManager.java | 94 +++++ .../ResourceManagerGateway.java | 58 ++++ .../rpc/resourcemanager/SlotAssignment.java | 25 ++ .../rpc/resourcemanager/SlotRequest.java | 25 ++ .../rpc/taskexecutor/TaskExecutor.java | 82 +++++ .../rpc/taskexecutor/TaskExecutorGateway.java | 48 +++ .../runtime/rpc/RpcCompletenessTest.java | 327 ++++++++++++++++++ .../runtime/rpc/akka/AkkaRpcServiceTest.java | 81 +++++ .../rpc/taskexecutor/TaskExecutorTest.java | 92 +++++ .../runtime/util/DirectExecutorService.java | 234 +++++++++++++ flink-tests/pom.xml | 1 - pom.xml | 7 + 41 files changed, 2784 insertions(+), 1 deletion(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcGateway.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcMethod.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcTimeout.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaGateway.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/BaseAkkaActor.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/BaseAkkaGateway.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/jobmaster/JobMasterAkkaActor.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/jobmaster/JobMasterAkkaGateway.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CallableMessage.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CancelTask.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/ExecuteTask.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RegisterAtResourceManager.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RegisterJobMaster.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RequestSlot.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunnableMessage.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/UpdateTaskExecutionState.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/resourcemanager/ResourceManagerAkkaActor.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/resourcemanager/ResourceManagerAkkaGateway.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/taskexecutor/TaskExecutorAkkaActor.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/taskexecutor/TaskExecutorAkkaGateway.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMasterGateway.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/JobMasterRegistration.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/RegistrationResponse.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerGateway.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotAssignment.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotRequest.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorGateway.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/util/DirectExecutorService.java diff --git a/flink-runtime/pom.xml b/flink-runtime/pom.xml index 5fea8fb86b3c7..09c6fd0ebe391 100644 --- a/flink-runtime/pom.xml +++ b/flink-runtime/pom.xml @@ -189,6 +189,11 @@ under the License. akka-testkit_${scala.binary.version} + + org.reflections + reflections + + diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java new file mode 100644 index 0000000000000..e06711e27b6e6 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java @@ -0,0 +1,54 @@ +/* + * 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.rpc; + +import akka.util.Timeout; +import scala.concurrent.Future; + +import java.util.concurrent.Callable; +import java.util.concurrent.TimeoutException; + +/** + * Interface to execute {@link Runnable} and {@link Callable} in the main thread of the underlying + * rpc server. + * + * This interface is intended to be implemented by the self gateway in a {@link RpcEndpoint} + * implementation which allows to dispatch local procedures to the main thread of the underlying + * rpc server. + */ +public interface MainThreadExecutor { + /** + * Execute the runnable in the main thread of the underlying rpc server. + * + * @param runnable Runnable to be executed + */ + void runAsync(Runnable runnable); + + /** + * Execute the callable in the main thread of the underlying rpc server and return a future for + * the callable result. If the future is not completed within the given timeout, the returned + * future will throw a {@link TimeoutException}. + * + * @param callable Callable to be executed + * @param timeout Timeout for the future to complete + * @param Return value of the callable + * @return Future of the callable result + */ + Future callAsync(Callable callable, Timeout timeout); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java new file mode 100644 index 0000000000000..3d8757f8f77aa --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java @@ -0,0 +1,182 @@ +/* + * 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.rpc; + +import akka.util.Timeout; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.concurrent.ExecutionContext; +import scala.concurrent.Future; + +import java.util.concurrent.Callable; + +/** + * Base class for rpc endpoints. Distributed components which offer remote procedure calls have to + * extend the rpc endpoint base class. + * + * The main idea is that a rpc endpoint is backed by a rpc server which has a single thread + * processing the rpc calls. Thus, by executing all state changing operations within the main + * thread, we don't have to reason about concurrent accesses. The rpc provides provides + * {@link #runAsync(Runnable)}, {@link #callAsync(Callable, Timeout)} and the + * {@link #getMainThreadExecutionContext()} to execute code in the rpc server's main thread. + * + * @param Rpc gateway counterpart for the implementing rpc endpoint + */ +public abstract class RpcEndpoint { + + protected final Logger log = LoggerFactory.getLogger(getClass()); + + /** Rpc service to be used to start the rpc server and to obtain rpc gateways */ + private final RpcService rpcService; + + /** Self gateway which can be used to schedule asynchronous calls on yourself */ + private C self; + + /** + * The main thread execution context to be used to execute future callbacks in the main thread + * of the executing rpc server. + * + * IMPORTANT: The main thread context is only available after the rpc server has been started. + */ + private MainThreadExecutionContext mainThreadExecutionContext; + + public RpcEndpoint(RpcService rpcService) { + this.rpcService = rpcService; + } + + /** + * Get self-gateway which should be used to run asynchronous rpc calls on this endpoint. + * + * IMPORTANT: Always issue local method calls via the self-gateway if the current thread + * is not the main thread of the underlying rpc server, e.g. from within a future callback. + * + * @return Self gateway + */ + public C getSelf() { + return self; + } + + /** + * Execute the runnable in the main thread of the underlying rpc server. + * + * @param runnable Runnable to be executed in the main thread of the underlying rpc server + */ + public void runAsync(Runnable runnable) { + ((MainThreadExecutor) self).runAsync(runnable); + } + + /** + * Execute the callable in the main thread of the underlying rpc server returning a future for + * the result of the callable. If the callable is not completed within the given timeout, then + * the future will be failed with a {@link java.util.concurrent.TimeoutException}. + * + * @param callable Callable to be executed in the main thread of the underlying rpc server + * @param timeout Timeout for the callable to be completed + * @param Return type of the callable + * @return Future for the result of the callable. + */ + public Future callAsync(Callable callable, Timeout timeout) { + return ((MainThreadExecutor) self).callAsync(callable, timeout); + } + + /** + * Gets the main thread execution context. The main thread execution context can be used to + * execute tasks in the main thread of the underlying rpc server. + * + * @return Main thread execution context + */ + public ExecutionContext getMainThreadExecutionContext() { + return mainThreadExecutionContext; + } + + /** + * Gets the used rpc service. + * + * @return Rpc service + */ + public RpcService getRpcService() { + return rpcService; + } + + /** + * Starts the underlying rpc server via the rpc service and creates the main thread execution + * context. This makes the rpc endpoint effectively reachable from the outside. + * + * Can be overriden to add rpc endpoint specific start up code. Should always call the parent + * start method. + */ + public void start() { + self = rpcService.startServer(this); + mainThreadExecutionContext = new MainThreadExecutionContext((MainThreadExecutor) self); + } + + + /** + * Shuts down the underlying rpc server via the rpc service. + * + * Can be overriden to add rpc endpoint specific shut down code. Should always call the parent + * shut down method. + */ + public void shutDown() { + rpcService.stopServer(self); + } + + /** + * Gets the address of the underlying rpc server. The address should be fully qualified so that + * a remote system can connect to this rpc server via this address. + * + * @return Fully qualified address of the underlying rpc server + */ + public String getAddress() { + return rpcService.getAddress(self); + } + + /** + * Execution context which executes runnables in the main thread context. A reported failure + * will cause the underlying rpc server to shut down. + */ + private class MainThreadExecutionContext implements ExecutionContext { + private final MainThreadExecutor gateway; + + MainThreadExecutionContext(MainThreadExecutor gateway) { + this.gateway = gateway; + } + + @Override + public void execute(Runnable runnable) { + gateway.runAsync(runnable); + } + + @Override + public void reportFailure(final Throwable t) { + gateway.runAsync(new Runnable() { + @Override + public void run() { + log.error("Encountered failure in the main thread execution context.", t); + shutDown(); + } + }); + } + + @Override + public ExecutionContext prepare() { + return this; + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcGateway.java new file mode 100644 index 0000000000000..e3a16b47331ca --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcGateway.java @@ -0,0 +1,25 @@ +/* + * 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.rpc; + +/** + * Rpc gateway interface which has to be implemented by Rpc gateways. + */ +public interface RpcGateway { +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcMethod.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcMethod.java new file mode 100644 index 0000000000000..875e557933395 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcMethod.java @@ -0,0 +1,35 @@ +/* + * 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.rpc; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Annotation for rpc method in a {@link RpcEndpoint} implementation. Every rpc method must have a + * respective counterpart in the {@link RpcGateway} implementation for this rpc server. The + * RpcCompletenessTest makes sure that the set of rpc methods in a rpc server and the set of + * gateway methods in the corresponding gateway implementation are identical. + */ +@Target(ElementType.METHOD) +@Retention(RetentionPolicy.RUNTIME) +public @interface RpcMethod { +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java new file mode 100644 index 0000000000000..90ff7b671ce03 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java @@ -0,0 +1,74 @@ +/* + * 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.rpc; + +import scala.concurrent.Future; + +/** + * Interface for rpc services. An rpc service is used to start and connect to a {@link RpcEndpoint}. + * Connecting to a rpc server will return a {@link RpcGateway} which can be used to call remote + * procedures. + */ +public interface RpcService { + + /** + * Connect to a remote rpc server under the provided address. Returns a rpc gateway which can + * be used to communicate with the rpc server. + * + * @param address Address of the remote rpc server + * @param clazz Class of the rpc gateway to return + * @param Type of the rpc gateway to return + * @return Future containing the rpc gateway + */ + Future connect(String address, Class clazz); + + /** + * Start a rpc server which forwards the remote procedure calls to the provided rpc endpoint. + * + * @param rpcEndpoint Rpc protocl to dispath the rpcs to + * @param Type of the rpc endpoint + * @param Type of the self rpc gateway associated with the rpc server + * @return Self gateway to dispatch remote procedure calls to oneself + */ + C startServer(S rpcEndpoint); + + /** + * Stop the underlying rpc server of the provided self gateway. + * + * @param selfGateway Self gateway describing the underlying rpc server + * @param Type of the rpc gateway + */ + void stopServer(C selfGateway); + + /** + * Stop the rpc service shutting down all started rpc servers. + */ + void stopService(); + + /** + * Get the fully qualified address of the underlying rpc server represented by the self gateway. + * It must be possible to connect from a remote host to the rpc server via the returned fully + * qualified address. + * + * @param selfGateway Self gateway associated with the underlying rpc server + * @param Type of the rpc gateway + * @return Fully qualified address + */ + String getAddress(C selfGateway); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcTimeout.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcTimeout.java new file mode 100644 index 0000000000000..3d36d47ee5604 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcTimeout.java @@ -0,0 +1,34 @@ +/* + * 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.rpc; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Annotation for {@link RpcGateway} methods to specify an additional timeout parameter for the + * returned future to be completed. The rest of the provided parameters is passed to the remote rpc + * server for the rpc. + */ +@Target(ElementType.PARAMETER) +@Retention(RetentionPolicy.RUNTIME) +public @interface RpcTimeout { +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaGateway.java new file mode 100644 index 0000000000000..a96a6008ba23a --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaGateway.java @@ -0,0 +1,29 @@ +/* + * 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.rpc.akka; + +import akka.actor.ActorRef; + +/** + * Interface for Akka based rpc gateways + */ +public interface AkkaGateway { + + ActorRef getActorRef(); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java new file mode 100644 index 0000000000000..d55bd135b273b --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java @@ -0,0 +1,145 @@ +/* + * 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.rpc.akka; + +import akka.actor.ActorIdentity; +import akka.actor.ActorRef; +import akka.actor.ActorSelection; +import akka.actor.ActorSystem; +import akka.actor.Identify; +import akka.actor.PoisonPill; +import akka.actor.Props; +import akka.dispatch.Mapper; +import akka.pattern.AskableActorSelection; +import akka.util.Timeout; +import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.rpc.jobmaster.JobMaster; +import org.apache.flink.runtime.rpc.jobmaster.JobMasterGateway; +import org.apache.flink.runtime.rpc.resourcemanager.ResourceManager; +import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.rpc.RpcGateway; +import org.apache.flink.runtime.rpc.RpcEndpoint; +import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.rpc.akka.jobmaster.JobMasterAkkaActor; +import org.apache.flink.runtime.rpc.akka.jobmaster.JobMasterAkkaGateway; +import org.apache.flink.runtime.rpc.akka.resourcemanager.ResourceManagerAkkaActor; +import org.apache.flink.runtime.rpc.akka.resourcemanager.ResourceManagerAkkaGateway; +import org.apache.flink.runtime.rpc.akka.taskexecutor.TaskExecutorAkkaActor; +import org.apache.flink.runtime.rpc.akka.taskexecutor.TaskExecutorAkkaGateway; +import org.apache.flink.runtime.rpc.taskexecutor.TaskExecutorGateway; +import org.apache.flink.runtime.rpc.taskexecutor.TaskExecutor; +import scala.concurrent.Future; + +import java.util.HashSet; +import java.util.Set; + +public class AkkaRpcService implements RpcService { + private final ActorSystem actorSystem; + private final Timeout timeout; + private final Set actors = new HashSet<>(); + + public AkkaRpcService(ActorSystem actorSystem, Timeout timeout) { + this.actorSystem = actorSystem; + this.timeout = timeout; + } + + @Override + public Future connect(String address, final Class clazz) { + ActorSelection actorSel = actorSystem.actorSelection(address); + + AskableActorSelection asker = new AskableActorSelection(actorSel); + + Future identify = asker.ask(new Identify(42), timeout); + + return identify.map(new Mapper(){ + public C apply(Object obj) { + ActorRef actorRef = ((ActorIdentity) obj).getRef(); + + if (clazz == TaskExecutorGateway.class) { + return (C) new TaskExecutorAkkaGateway(actorRef, timeout); + } else if (clazz == ResourceManagerGateway.class) { + return (C) new ResourceManagerAkkaGateway(actorRef, timeout); + } else if (clazz == JobMasterGateway.class) { + return (C) new JobMasterAkkaGateway(actorRef, timeout); + } else { + throw new RuntimeException("Could not find remote endpoint " + clazz); + } + } + }, actorSystem.dispatcher()); + } + + @Override + public C startServer(S rpcEndpoint) { + ActorRef ref; + C self; + if (rpcEndpoint instanceof TaskExecutor) { + ref = actorSystem.actorOf( + Props.create(TaskExecutorAkkaActor.class, rpcEndpoint) + ); + + self = (C) new TaskExecutorAkkaGateway(ref, timeout); + } else if (rpcEndpoint instanceof ResourceManager) { + ref = actorSystem.actorOf( + Props.create(ResourceManagerAkkaActor.class, rpcEndpoint) + ); + + self = (C) new ResourceManagerAkkaGateway(ref, timeout); + } else if (rpcEndpoint instanceof JobMaster) { + ref = actorSystem.actorOf( + Props.create(JobMasterAkkaActor.class, rpcEndpoint) + ); + + self = (C) new JobMasterAkkaGateway(ref, timeout); + } else { + throw new RuntimeException("Could not start RPC server for class " + rpcEndpoint.getClass()); + } + + actors.add(ref); + + return self; + } + + @Override + public void stopServer(C selfGateway) { + if (selfGateway instanceof AkkaGateway) { + AkkaGateway akkaClient = (AkkaGateway) selfGateway; + + if (actors.contains(akkaClient.getActorRef())) { + akkaClient.getActorRef().tell(PoisonPill.getInstance(), ActorRef.noSender()); + } else { + // don't stop this actor since it was not started by this RPC service + } + } + } + + @Override + public void stopService() { + actorSystem.shutdown(); + actorSystem.awaitTermination(); + } + + @Override + public String getAddress(C selfGateway) { + if (selfGateway instanceof AkkaGateway) { + return AkkaUtils.getAkkaURL(actorSystem, ((AkkaGateway) selfGateway).getActorRef()); + } else { + throw new RuntimeException("Cannot get address for non " + AkkaGateway.class.getName() + "."); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/BaseAkkaActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/BaseAkkaActor.java new file mode 100644 index 0000000000000..3cb499cffca1f --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/BaseAkkaActor.java @@ -0,0 +1,50 @@ +/* + * 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.rpc.akka; + +import akka.actor.Status; +import akka.actor.UntypedActor; +import org.apache.flink.runtime.rpc.akka.messages.CallableMessage; +import org.apache.flink.runtime.rpc.akka.messages.RunnableMessage; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class BaseAkkaActor extends UntypedActor { + private static final Logger LOG = LoggerFactory.getLogger(BaseAkkaActor.class); + + @Override + public void onReceive(Object message) throws Exception { + if (message instanceof RunnableMessage) { + try { + ((RunnableMessage) message).getRunnable().run(); + } catch (Exception e) { + LOG.error("Encountered error while executing runnable.", e); + } + } else if (message instanceof CallableMessage) { + try { + Object result = ((CallableMessage) message).getCallable().call(); + sender().tell(new Status.Success(result), getSelf()); + } catch (Exception e) { + sender().tell(new Status.Failure(e), getSelf()); + } + } else { + throw new RuntimeException("Unknown message " + message); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/BaseAkkaGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/BaseAkkaGateway.java new file mode 100644 index 0000000000000..512790d11639a --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/BaseAkkaGateway.java @@ -0,0 +1,41 @@ +/* + * 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.rpc.akka; + +import akka.actor.ActorRef; +import akka.pattern.Patterns; +import akka.util.Timeout; +import org.apache.flink.runtime.rpc.MainThreadExecutor; +import org.apache.flink.runtime.rpc.akka.messages.CallableMessage; +import org.apache.flink.runtime.rpc.akka.messages.RunnableMessage; +import scala.concurrent.Future; + +import java.util.concurrent.Callable; + +public abstract class BaseAkkaGateway implements MainThreadExecutor, AkkaGateway { + @Override + public void runAsync(Runnable runnable) { + getActorRef().tell(new RunnableMessage(runnable), ActorRef.noSender()); + } + + @Override + public Future callAsync(Callable callable, Timeout timeout) { + return (Future) Patterns.ask(getActorRef(), new CallableMessage(callable), timeout); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/jobmaster/JobMasterAkkaActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/jobmaster/JobMasterAkkaActor.java new file mode 100644 index 0000000000000..9e04ea91070aa --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/jobmaster/JobMasterAkkaActor.java @@ -0,0 +1,58 @@ +/* + * 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.rpc.akka.jobmaster; + +import akka.actor.ActorRef; +import akka.actor.Status; +import org.apache.flink.runtime.rpc.akka.BaseAkkaActor; +import org.apache.flink.runtime.rpc.akka.messages.RegisterAtResourceManager; +import org.apache.flink.runtime.rpc.jobmaster.JobMaster; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.rpc.akka.messages.UpdateTaskExecutionState; + +public class JobMasterAkkaActor extends BaseAkkaActor { + private final JobMaster jobMaster; + + public JobMasterAkkaActor(JobMaster jobMaster) { + this.jobMaster = jobMaster; + } + + @Override + public void onReceive(Object message) throws Exception { + if (message instanceof UpdateTaskExecutionState) { + + final ActorRef sender = getSender(); + + UpdateTaskExecutionState updateTaskExecutionState = (UpdateTaskExecutionState) message; + + try { + Acknowledge result = jobMaster.updateTaskExecutionState(updateTaskExecutionState.getTaskExecutionState()); + sender.tell(new Status.Success(result), getSelf()); + } catch (Exception e) { + sender.tell(new Status.Failure(e), getSelf()); + } + } else if (message instanceof RegisterAtResourceManager) { + RegisterAtResourceManager registerAtResourceManager = (RegisterAtResourceManager) message; + + jobMaster.registerAtResourceManager(registerAtResourceManager.getAddress()); + } else { + super.onReceive(message); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/jobmaster/JobMasterAkkaGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/jobmaster/JobMasterAkkaGateway.java new file mode 100644 index 0000000000000..e6bf061701faf --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/jobmaster/JobMasterAkkaGateway.java @@ -0,0 +1,57 @@ +/* + * 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.rpc.akka.jobmaster; + +import akka.actor.ActorRef; +import akka.pattern.AskableActorRef; +import akka.util.Timeout; +import org.apache.flink.runtime.rpc.akka.BaseAkkaGateway; +import org.apache.flink.runtime.rpc.akka.messages.RegisterAtResourceManager; +import org.apache.flink.runtime.rpc.jobmaster.JobMasterGateway; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.rpc.akka.messages.UpdateTaskExecutionState; +import org.apache.flink.runtime.taskmanager.TaskExecutionState; +import scala.concurrent.Future; +import scala.reflect.ClassTag$; + +public class JobMasterAkkaGateway extends BaseAkkaGateway implements JobMasterGateway { + private final AskableActorRef actorRef; + private final Timeout timeout; + + public JobMasterAkkaGateway(ActorRef actorRef, Timeout timeout) { + this.actorRef = new AskableActorRef(actorRef); + this.timeout = timeout; + } + + @Override + public Future updateTaskExecutionState(TaskExecutionState taskExecutionState) { + return actorRef.ask(new UpdateTaskExecutionState(taskExecutionState), timeout) + .mapTo(ClassTag$.MODULE$.apply(Acknowledge.class)); + } + + @Override + public void registerAtResourceManager(String address) { + actorRef.actorRef().tell(new RegisterAtResourceManager(address), actorRef.actorRef()); + } + + @Override + public ActorRef getActorRef() { + return actorRef.actorRef(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CallableMessage.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CallableMessage.java new file mode 100644 index 0000000000000..f0e555f729ca4 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CallableMessage.java @@ -0,0 +1,33 @@ +/* + * 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.rpc.akka.messages; + +import java.util.concurrent.Callable; + +public class CallableMessage { + private final Callable callable; + + public CallableMessage(Callable callable) { + this.callable = callable; + } + + public Callable getCallable() { + return callable; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CancelTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CancelTask.java new file mode 100644 index 0000000000000..0b9e9dc9f6447 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CancelTask.java @@ -0,0 +1,36 @@ +/* + * 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.rpc.akka.messages; + +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; + +import java.io.Serializable; + +public class CancelTask implements Serializable { + private static final long serialVersionUID = -2998176874447950595L; + private final ExecutionAttemptID executionAttemptID; + + public CancelTask(ExecutionAttemptID executionAttemptID) { + this.executionAttemptID = executionAttemptID; + } + + public ExecutionAttemptID getExecutionAttemptID() { + return executionAttemptID; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/ExecuteTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/ExecuteTask.java new file mode 100644 index 0000000000000..a83d539082eb2 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/ExecuteTask.java @@ -0,0 +1,36 @@ +/* + * 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.rpc.akka.messages; + +import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; + +import java.io.Serializable; + +public class ExecuteTask implements Serializable { + private static final long serialVersionUID = -6769958430967048348L; + private final TaskDeploymentDescriptor taskDeploymentDescriptor; + + public ExecuteTask(TaskDeploymentDescriptor taskDeploymentDescriptor) { + this.taskDeploymentDescriptor = taskDeploymentDescriptor; + } + + public TaskDeploymentDescriptor getTaskDeploymentDescriptor() { + return taskDeploymentDescriptor; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RegisterAtResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RegisterAtResourceManager.java new file mode 100644 index 0000000000000..3ade08250b9ea --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RegisterAtResourceManager.java @@ -0,0 +1,36 @@ +/* + * 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.rpc.akka.messages; + +import java.io.Serializable; + +public class RegisterAtResourceManager implements Serializable { + + private static final long serialVersionUID = -4175905742620903602L; + + private final String address; + + public RegisterAtResourceManager(String address) { + this.address = address; + } + + public String getAddress() { + return address; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RegisterJobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RegisterJobMaster.java new file mode 100644 index 0000000000000..b35ea3895aa0e --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RegisterJobMaster.java @@ -0,0 +1,36 @@ +/* + * 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.rpc.akka.messages; + +import org.apache.flink.runtime.rpc.resourcemanager.JobMasterRegistration; + +import java.io.Serializable; + +public class RegisterJobMaster implements Serializable{ + private static final long serialVersionUID = -4616879574192641507L; + private final JobMasterRegistration jobMasterRegistration; + + public RegisterJobMaster(JobMasterRegistration jobMasterRegistration) { + this.jobMasterRegistration = jobMasterRegistration; + } + + public JobMasterRegistration getJobMasterRegistration() { + return jobMasterRegistration; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RequestSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RequestSlot.java new file mode 100644 index 0000000000000..85ceeec6f0738 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RequestSlot.java @@ -0,0 +1,37 @@ +/* + * 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.rpc.akka.messages; + +import org.apache.flink.runtime.rpc.resourcemanager.SlotRequest; + +import java.io.Serializable; + +public class RequestSlot implements Serializable { + private static final long serialVersionUID = 7207463889348525866L; + + private final SlotRequest slotRequest; + + public RequestSlot(SlotRequest slotRequest) { + this.slotRequest = slotRequest; + } + + public SlotRequest getSlotRequest() { + return slotRequest; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunnableMessage.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunnableMessage.java new file mode 100644 index 0000000000000..35567388208bd --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunnableMessage.java @@ -0,0 +1,31 @@ +/* + * 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.rpc.akka.messages; + +public class RunnableMessage { + private final Runnable runnable; + + public RunnableMessage(Runnable runnable) { + this.runnable = runnable; + } + + public Runnable getRunnable() { + return runnable; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/UpdateTaskExecutionState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/UpdateTaskExecutionState.java new file mode 100644 index 0000000000000..f89cd2f516041 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/UpdateTaskExecutionState.java @@ -0,0 +1,37 @@ +/* + * 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.rpc.akka.messages; + +import org.apache.flink.runtime.taskmanager.TaskExecutionState; + +import java.io.Serializable; + +public class UpdateTaskExecutionState implements Serializable{ + private static final long serialVersionUID = -6662229114427331436L; + + private final TaskExecutionState taskExecutionState; + + public UpdateTaskExecutionState(TaskExecutionState taskExecutionState) { + this.taskExecutionState = taskExecutionState; + } + + public TaskExecutionState getTaskExecutionState() { + return taskExecutionState; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/resourcemanager/ResourceManagerAkkaActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/resourcemanager/ResourceManagerAkkaActor.java new file mode 100644 index 0000000000000..13101f9362ff6 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/resourcemanager/ResourceManagerAkkaActor.java @@ -0,0 +1,65 @@ +/* + * 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.rpc.akka.resourcemanager; + +import akka.actor.ActorRef; +import akka.actor.Status; +import akka.pattern.Patterns; +import org.apache.flink.runtime.rpc.akka.BaseAkkaActor; +import org.apache.flink.runtime.rpc.resourcemanager.RegistrationResponse; +import org.apache.flink.runtime.rpc.resourcemanager.ResourceManager; +import org.apache.flink.runtime.rpc.resourcemanager.SlotAssignment; +import org.apache.flink.runtime.rpc.akka.messages.RegisterJobMaster; +import org.apache.flink.runtime.rpc.akka.messages.RequestSlot; +import scala.concurrent.Future; + +public class ResourceManagerAkkaActor extends BaseAkkaActor { + private final ResourceManager resourceManager; + + public ResourceManagerAkkaActor(ResourceManager resourceManager) { + this.resourceManager = resourceManager; + } + + @Override + public void onReceive(Object message) throws Exception { + final ActorRef sender = getSender(); + + if (message instanceof RegisterJobMaster) { + RegisterJobMaster registerJobMaster = (RegisterJobMaster) message; + + try { + Future response = resourceManager.registerJobMaster(registerJobMaster.getJobMasterRegistration()); + Patterns.pipe(response, getContext().dispatcher()).to(sender()); + } catch (Exception e) { + sender.tell(new Status.Failure(e), getSelf()); + } + } else if (message instanceof RequestSlot) { + RequestSlot requestSlot = (RequestSlot) message; + + try { + SlotAssignment response = resourceManager.requestSlot(requestSlot.getSlotRequest()); + sender.tell(new Status.Success(response), getSelf()); + } catch (Exception e) { + sender.tell(new Status.Failure(e), getSelf()); + } + } else { + super.onReceive(message); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/resourcemanager/ResourceManagerAkkaGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/resourcemanager/ResourceManagerAkkaGateway.java new file mode 100644 index 0000000000000..13047072ecad7 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/resourcemanager/ResourceManagerAkkaGateway.java @@ -0,0 +1,67 @@ +/* + * 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.rpc.akka.resourcemanager; + +import akka.actor.ActorRef; +import akka.pattern.AskableActorRef; +import akka.util.Timeout; +import org.apache.flink.runtime.rpc.akka.BaseAkkaGateway; +import org.apache.flink.runtime.rpc.resourcemanager.JobMasterRegistration; +import org.apache.flink.runtime.rpc.resourcemanager.RegistrationResponse; +import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.rpc.resourcemanager.SlotAssignment; +import org.apache.flink.runtime.rpc.resourcemanager.SlotRequest; +import org.apache.flink.runtime.rpc.akka.messages.RegisterJobMaster; +import org.apache.flink.runtime.rpc.akka.messages.RequestSlot; +import scala.concurrent.Future; +import scala.concurrent.duration.FiniteDuration; +import scala.reflect.ClassTag$; + +public class ResourceManagerAkkaGateway extends BaseAkkaGateway implements ResourceManagerGateway { + private final AskableActorRef actorRef; + private final Timeout timeout; + + public ResourceManagerAkkaGateway(ActorRef actorRef, Timeout timeout) { + this.actorRef = new AskableActorRef(actorRef); + this.timeout = timeout; + } + + @Override + public Future registerJobMaster(JobMasterRegistration jobMasterRegistration, FiniteDuration timeout) { + return actorRef.ask(new RegisterJobMaster(jobMasterRegistration), new Timeout(timeout)) + .mapTo(ClassTag$.MODULE$.apply(RegistrationResponse.class)); + } + + @Override + public Future registerJobMaster(JobMasterRegistration jobMasterRegistration) { + return actorRef.ask(new RegisterJobMaster(jobMasterRegistration), timeout) + .mapTo(ClassTag$.MODULE$.apply(RegistrationResponse.class)); + } + + @Override + public Future requestSlot(SlotRequest slotRequest) { + return actorRef.ask(new RequestSlot(slotRequest), timeout) + .mapTo(ClassTag$.MODULE$.apply(SlotAssignment.class)); + } + + @Override + public ActorRef getActorRef() { + return actorRef.actorRef(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/taskexecutor/TaskExecutorAkkaActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/taskexecutor/TaskExecutorAkkaActor.java new file mode 100644 index 0000000000000..ed522cc727378 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/taskexecutor/TaskExecutorAkkaActor.java @@ -0,0 +1,77 @@ +/* + * 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.rpc.akka.taskexecutor; + +import akka.actor.ActorRef; +import akka.actor.Status; +import akka.dispatch.OnComplete; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.rpc.akka.BaseAkkaActor; +import org.apache.flink.runtime.rpc.akka.messages.CancelTask; +import org.apache.flink.runtime.rpc.akka.messages.ExecuteTask; +import org.apache.flink.runtime.rpc.taskexecutor.TaskExecutorGateway; + +public class TaskExecutorAkkaActor extends BaseAkkaActor { + private final TaskExecutorGateway taskExecutor; + + public TaskExecutorAkkaActor(TaskExecutorGateway taskExecutor) { + this.taskExecutor = taskExecutor; + } + + @Override + public void onReceive(Object message) throws Exception { + final ActorRef sender = getSender(); + + if (message instanceof ExecuteTask) { + ExecuteTask executeTask = (ExecuteTask) message; + + taskExecutor.executeTask(executeTask.getTaskDeploymentDescriptor()).onComplete( + new OnComplete() { + @Override + public void onComplete(Throwable failure, Acknowledge success) throws Throwable { + if (failure != null) { + sender.tell(new Status.Failure(failure), getSelf()); + } else { + sender.tell(new Status.Success(Acknowledge.get()), getSelf()); + } + } + }, + getContext().dispatcher() + ); + } else if (message instanceof CancelTask) { + CancelTask cancelTask = (CancelTask) message; + + taskExecutor.cancelTask(cancelTask.getExecutionAttemptID()).onComplete( + new OnComplete() { + @Override + public void onComplete(Throwable failure, Acknowledge success) throws Throwable { + if (failure != null) { + sender.tell(new Status.Failure(failure), getSelf()); + } else { + sender.tell(new Status.Success(Acknowledge.get()), getSelf()); + } + } + }, + getContext().dispatcher() + ); + } else { + super.onReceive(message); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/taskexecutor/TaskExecutorAkkaGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/taskexecutor/TaskExecutorAkkaGateway.java new file mode 100644 index 0000000000000..7f0a52284c6aa --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/taskexecutor/TaskExecutorAkkaGateway.java @@ -0,0 +1,59 @@ +/* + * 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.rpc.akka.taskexecutor; + +import akka.actor.ActorRef; +import akka.pattern.AskableActorRef; +import akka.util.Timeout; +import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.rpc.akka.BaseAkkaGateway; +import org.apache.flink.runtime.rpc.akka.messages.CancelTask; +import org.apache.flink.runtime.rpc.akka.messages.ExecuteTask; +import org.apache.flink.runtime.rpc.taskexecutor.TaskExecutorGateway; +import scala.concurrent.Future; +import scala.reflect.ClassTag$; + +public class TaskExecutorAkkaGateway extends BaseAkkaGateway implements TaskExecutorGateway { + private final AskableActorRef actorRef; + private final Timeout timeout; + + public TaskExecutorAkkaGateway(ActorRef actorRef, Timeout timeout) { + this.actorRef = new AskableActorRef(actorRef); + this.timeout = timeout; + } + + @Override + public Future executeTask(TaskDeploymentDescriptor taskDeploymentDescriptor) { + return actorRef.ask(new ExecuteTask(taskDeploymentDescriptor), timeout) + .mapTo(ClassTag$.MODULE$.apply(Acknowledge.class)); + } + + @Override + public Future cancelTask(ExecutionAttemptID executionAttemptId) { + return actorRef.ask(new CancelTask(executionAttemptId), timeout) + .mapTo(ClassTag$.MODULE$.apply(Acknowledge.class)); + } + + @Override + public ActorRef getActorRef() { + return actorRef.actorRef(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java new file mode 100644 index 0000000000000..b81b19c27321e --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java @@ -0,0 +1,249 @@ +/* + * 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.rpc.jobmaster; + +import akka.dispatch.Futures; +import akka.dispatch.Mapper; +import akka.dispatch.OnComplete; +import org.apache.flink.runtime.instance.InstanceID; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.rpc.RpcMethod; +import org.apache.flink.runtime.rpc.resourcemanager.JobMasterRegistration; +import org.apache.flink.runtime.rpc.resourcemanager.RegistrationResponse; +import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.rpc.RpcEndpoint; +import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.taskmanager.TaskExecutionState; +import scala.Tuple2; +import scala.concurrent.ExecutionContext; +import scala.concurrent.ExecutionContext$; +import scala.concurrent.Future; +import scala.concurrent.duration.Deadline; +import scala.concurrent.duration.FiniteDuration; + +import java.util.UUID; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +/** + * JobMaster implementation. The job master is responsible for the execution of a single + * {@link org.apache.flink.runtime.jobgraph.JobGraph}. + * + * It offers the following methods as part of its rpc interface to interact with the JobMaster + * remotely: + *
    + *
  • {@link #registerAtResourceManager(String)} triggers the registration at the resource manager
  • + *
  • {@link #updateTaskExecutionState(TaskExecutionState)} updates the task execution state for + * given task
  • + *
+ */ +public class JobMaster extends RpcEndpoint { + /** Execution context for future callbacks */ + private final ExecutionContext executionContext; + + /** Execution context for scheduled runnables */ + private final ScheduledExecutorService scheduledExecutorService; + + private final FiniteDuration initialRegistrationTimeout = new FiniteDuration(500, TimeUnit.MILLISECONDS); + private final FiniteDuration maxRegistrationTimeout = new FiniteDuration(30, TimeUnit.SECONDS); + private final FiniteDuration registrationDuration = new FiniteDuration(365, TimeUnit.DAYS); + private final long failedRegistrationDelay = 10000; + + /** Gateway to connected resource manager, null iff not connected */ + private ResourceManagerGateway resourceManager = null; + + /** UUID to filter out old registration runs */ + private UUID currentRegistrationRun; + + public JobMaster(RpcService rpcService, ExecutorService executorService) { + super(rpcService); + executionContext = ExecutionContext$.MODULE$.fromExecutor(executorService); + scheduledExecutorService = new ScheduledThreadPoolExecutor(1); + } + + public ResourceManagerGateway getResourceManager() { + return resourceManager; + } + + //---------------------------------------------------------------------------------------------- + // RPC methods + //---------------------------------------------------------------------------------------------- + + /** + * Updates the task execution state for a given task. + * + * @param taskExecutionState New task execution state for a given task + * @return Acknowledge the task execution state update + */ + @RpcMethod + public Acknowledge updateTaskExecutionState(TaskExecutionState taskExecutionState) { + System.out.println("TaskExecutionState: " + taskExecutionState); + return Acknowledge.get(); + } + + /** + * Triggers the registration of the job master at the resource manager. + * + * @param address Address of the resource manager + */ + @RpcMethod + public void registerAtResourceManager(final String address) { + currentRegistrationRun = UUID.randomUUID(); + + Future resourceManagerFuture = getRpcService().connect(address, ResourceManagerGateway.class); + + handleResourceManagerRegistration( + new JobMasterRegistration(getAddress()), + 1, + resourceManagerFuture, + currentRegistrationRun, + initialRegistrationTimeout, + maxRegistrationTimeout, + registrationDuration.fromNow()); + } + + //---------------------------------------------------------------------------------------------- + // Helper methods + //---------------------------------------------------------------------------------------------- + + /** + * Helper method to handle the resource manager registration process. If a registration attempt + * times out, then a new attempt with the doubled time out is initiated. The whole registration + * process has a deadline. Once this deadline is overdue without successful registration, the + * job master shuts down. + * + * @param jobMasterRegistration Job master registration info which is sent to the resource + * manager + * @param attemptNumber Registration attempt number + * @param resourceManagerFuture Future of the resource manager gateway + * @param registrationRun UUID describing the current registration run + * @param timeout Timeout of the last registration attempt + * @param maxTimeout Maximum timeout between registration attempts + * @param deadline Deadline for the registration + */ + void handleResourceManagerRegistration( + final JobMasterRegistration jobMasterRegistration, + final int attemptNumber, + final Future resourceManagerFuture, + final UUID registrationRun, + final FiniteDuration timeout, + final FiniteDuration maxTimeout, + final Deadline deadline) { + + // filter out concurrent registration runs + if (registrationRun.equals(currentRegistrationRun)) { + + log.info("Start registration attempt #{}.", attemptNumber); + + if (deadline.isOverdue()) { + // we've exceeded our registration deadline. This means that we have to shutdown the JobMaster + log.error("Exceeded registration deadline without successfully registering at the ResourceManager."); + shutDown(); + } else { + Future> registrationResponseFuture = resourceManagerFuture.flatMap(new Mapper>>() { + @Override + public Future> apply(ResourceManagerGateway resourceManagerGateway) { + return resourceManagerGateway.registerJobMaster(jobMasterRegistration, timeout).zip(Futures.successful(resourceManagerGateway)); + } + }, executionContext); + + registrationResponseFuture.onComplete(new OnComplete>() { + @Override + public void onComplete(Throwable failure, Tuple2 tuple) throws Throwable { + if (failure != null) { + if (failure instanceof TimeoutException) { + // we haven't received an answer in the given timeout interval, + // so increase it and try again. + final FiniteDuration newTimeout = timeout.$times(2L).min(maxTimeout); + + handleResourceManagerRegistration( + jobMasterRegistration, + attemptNumber + 1, + resourceManagerFuture, + registrationRun, + newTimeout, + maxTimeout, + deadline); + } else { + log.error("Received unknown error while registering at the ResourceManager.", failure); + shutDown(); + } + } else { + final RegistrationResponse response = tuple._1(); + final ResourceManagerGateway gateway = tuple._2(); + + if (response.isSuccess()) { + finishResourceManagerRegistration(gateway, response.getInstanceID()); + } else { + log.info("The registration was refused. Try again."); + + scheduledExecutorService.schedule(new Runnable() { + @Override + public void run() { + // we have to execute scheduled runnable in the main thread + // because we need consistency wrt currentRegistrationRun + runAsync(new Runnable() { + @Override + public void run() { + // our registration attempt was refused. Start over. + handleResourceManagerRegistration( + jobMasterRegistration, + 1, + resourceManagerFuture, + registrationRun, + initialRegistrationTimeout, + maxTimeout, + deadline); + } + }); + } + }, failedRegistrationDelay, TimeUnit.MILLISECONDS); + } + } + } + }, getMainThreadExecutionContext()); // use the main thread execution context to execute the call back in the main thread + } + } else { + log.info("Discard out-dated registration run."); + } + } + + /** + * Finish the resource manager registration by setting the new resource manager gateway. + * + * @param resourceManager New resource manager gateway + * @param instanceID Instance id assigned by the resource manager + */ + void finishResourceManagerRegistration(ResourceManagerGateway resourceManager, InstanceID instanceID) { + log.info("Successfully registered at the ResourceManager under instance id {}.", instanceID); + this.resourceManager = resourceManager; + } + + /** + * Return if the job master is connected to a resource manager. + * + * @return true if the job master is connected to the resource manager + */ + public boolean isConnected() { + return resourceManager != null; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMasterGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMasterGateway.java new file mode 100644 index 0000000000000..17a4c3ab75eda --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMasterGateway.java @@ -0,0 +1,45 @@ +/* + * 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.rpc.jobmaster; + +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.rpc.RpcGateway; +import org.apache.flink.runtime.taskmanager.TaskExecutionState; +import scala.concurrent.Future; + +/** + * {@link JobMaster} rpc gateway interface + */ +public interface JobMasterGateway extends RpcGateway { + + /** + * Updates the task execution state for a given task. + * + * @param taskExecutionState New task execution state for a given task + * @return Future acknowledge of the task execution state update + */ + Future updateTaskExecutionState(TaskExecutionState taskExecutionState); + + /** + * Triggers the registration of the job master at the resource manager. + * + * @param address Address of the resource manager + */ + void registerAtResourceManager(final String address); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/JobMasterRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/JobMasterRegistration.java new file mode 100644 index 0000000000000..7a2deae20b0b7 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/JobMasterRegistration.java @@ -0,0 +1,35 @@ +/* + * 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.rpc.resourcemanager; + +import java.io.Serializable; + +public class JobMasterRegistration implements Serializable { + private static final long serialVersionUID = 8411214999193765202L; + + private final String address; + + public JobMasterRegistration(String address) { + this.address = address; + } + + public String getAddress() { + return address; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/RegistrationResponse.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/RegistrationResponse.java new file mode 100644 index 0000000000000..8ac9e49ecf1aa --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/RegistrationResponse.java @@ -0,0 +1,43 @@ +/* + * 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.rpc.resourcemanager; + +import org.apache.flink.runtime.instance.InstanceID; + +import java.io.Serializable; + +public class RegistrationResponse implements Serializable { + private static final long serialVersionUID = -2379003255993119993L; + + private final boolean isSuccess; + private final InstanceID instanceID; + + public RegistrationResponse(boolean isSuccess, InstanceID instanceID) { + this.isSuccess = isSuccess; + this.instanceID = instanceID; + } + + public boolean isSuccess() { + return isSuccess; + } + + public InstanceID getInstanceID() { + return instanceID; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java new file mode 100644 index 0000000000000..c7e8def051398 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java @@ -0,0 +1,94 @@ +/* + * 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.rpc.resourcemanager; + +import akka.dispatch.Mapper; +import org.apache.flink.runtime.instance.InstanceID; +import org.apache.flink.runtime.rpc.RpcMethod; +import org.apache.flink.runtime.rpc.RpcEndpoint; +import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.rpc.jobmaster.JobMaster; +import org.apache.flink.runtime.rpc.jobmaster.JobMasterGateway; +import scala.concurrent.ExecutionContext; +import scala.concurrent.ExecutionContext$; +import scala.concurrent.Future; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ExecutorService; + +/** + * ResourceManager implementation. The resource manager is responsible for resource de-/allocation + * and bookkeeping. + * + * It offers the following methods as part of its rpc interface to interact with the him remotely: + *
    + *
  • {@link #registerJobMaster(JobMasterRegistration)} registers a {@link JobMaster} at the resource manager
  • + *
  • {@link #requestSlot(SlotRequest)} requests a slot from the resource manager
  • + *
+ */ +public class ResourceManager extends RpcEndpoint { + private final ExecutionContext executionContext; + private final Map jobMasterGateways; + + public ResourceManager(RpcService rpcService, ExecutorService executorService) { + super(rpcService); + this.executionContext = ExecutionContext$.MODULE$.fromExecutor(executorService); + this.jobMasterGateways = new HashMap<>(); + } + + /** + * Register a {@link JobMaster} at the resource manager. + * + * @param jobMasterRegistration Job master registration information + * @return Future registration response + */ + @RpcMethod + public Future registerJobMaster(JobMasterRegistration jobMasterRegistration) { + Future jobMasterFuture = getRpcService().connect(jobMasterRegistration.getAddress(), JobMasterGateway.class); + + return jobMasterFuture.map(new Mapper() { + @Override + public RegistrationResponse apply(final JobMasterGateway jobMasterGateway) { + InstanceID instanceID; + + if (jobMasterGateways.containsKey(jobMasterGateway)) { + instanceID = jobMasterGateways.get(jobMasterGateway); + } else { + instanceID = new InstanceID(); + jobMasterGateways.put(jobMasterGateway, instanceID); + } + + return new RegistrationResponse(true, instanceID); + } + }, getMainThreadExecutionContext()); + } + + /** + * Requests a slot from the resource manager. + * + * @param slotRequest Slot request + * @return Slot assignment + */ + @RpcMethod + public SlotAssignment requestSlot(SlotRequest slotRequest) { + System.out.println("SlotRequest: " + slotRequest); + return new SlotAssignment(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerGateway.java new file mode 100644 index 0000000000000..464a2618ea7e3 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerGateway.java @@ -0,0 +1,58 @@ +/* + * 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.rpc.resourcemanager; + +import org.apache.flink.runtime.rpc.RpcGateway; +import org.apache.flink.runtime.rpc.RpcTimeout; +import org.apache.flink.runtime.rpc.jobmaster.JobMaster; +import scala.concurrent.Future; +import scala.concurrent.duration.FiniteDuration; + +/** + * {@link ResourceManager} rpc gateway interface. + */ +public interface ResourceManagerGateway extends RpcGateway { + + /** + * Register a {@link JobMaster} at the resource manager. + * + * @param jobMasterRegistration Job master registration information + * @param timeout Timeout for the future to complete + * @return Future registration response + */ + Future registerJobMaster( + JobMasterRegistration jobMasterRegistration, + @RpcTimeout FiniteDuration timeout); + + /** + * Register a {@link JobMaster} at the resource manager. + * + * @param jobMasterRegistration Job master registration information + * @return Future registration response + */ + Future registerJobMaster(JobMasterRegistration jobMasterRegistration); + + /** + * Requests a slot from the resource manager. + * + * @param slotRequest Slot request + * @return Future slot assignment + */ + Future requestSlot(SlotRequest slotRequest); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotAssignment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotAssignment.java new file mode 100644 index 0000000000000..86cd8b76a8cc3 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotAssignment.java @@ -0,0 +1,25 @@ +/* + * 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.rpc.resourcemanager; + +import java.io.Serializable; + +public class SlotAssignment implements Serializable{ + private static final long serialVersionUID = -6990813455942742322L; +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotRequest.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotRequest.java new file mode 100644 index 0000000000000..d8fe2689644ea --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotRequest.java @@ -0,0 +1,25 @@ +/* + * 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.rpc.resourcemanager; + +import java.io.Serializable; + +public class SlotRequest implements Serializable{ + private static final long serialVersionUID = -6586877187990445986L; +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java new file mode 100644 index 0000000000000..cdfc3bd63e1e5 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java @@ -0,0 +1,82 @@ +/* + * 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.rpc.taskexecutor; + +import akka.dispatch.ExecutionContexts$; +import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.rpc.RpcMethod; +import org.apache.flink.runtime.rpc.RpcEndpoint; +import org.apache.flink.runtime.rpc.RpcService; +import scala.concurrent.ExecutionContext; + +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.ExecutorService; + +/** + * TaskExecutor implementation. The task executor is responsible for the execution of multiple + * {@link org.apache.flink.runtime.taskmanager.Task}. + * + * It offers the following methods as part of its rpc interface to interact with him remotely: + *
    + *
  • {@link #executeTask(TaskDeploymentDescriptor)} executes a given task on the TaskExecutor
  • + *
  • {@link #cancelTask(ExecutionAttemptID)} cancels a given task identified by the {@link ExecutionAttemptID}
  • + *
+ */ +public class TaskExecutor extends RpcEndpoint { + private final ExecutionContext executionContext; + private final Set tasks = new HashSet<>(); + + public TaskExecutor(RpcService rpcService, ExecutorService executorService) { + super(rpcService); + this.executionContext = ExecutionContexts$.MODULE$.fromExecutor(executorService); + } + + /** + * Execute the given task on the task executor. The task is described by the provided + * {@link TaskDeploymentDescriptor}. + * + * @param taskDeploymentDescriptor Descriptor for the task to be executed + * @return Acknowledge the start of the task execution + */ + @RpcMethod + public Acknowledge executeTask(TaskDeploymentDescriptor taskDeploymentDescriptor) { + tasks.add(taskDeploymentDescriptor.getExecutionId()); + return Acknowledge.get(); + } + + /** + * Cancel a task identified by it {@link ExecutionAttemptID}. If the task cannot be found, then + * the method throws an {@link Exception}. + * + * @param executionAttemptId Execution attempt ID identifying the task to be canceled. + * @return Acknowledge the task canceling + * @throws Exception if the task with the given execution attempt id could not be found + */ + @RpcMethod + public Acknowledge cancelTask(ExecutionAttemptID executionAttemptId) throws Exception { + if (tasks.contains(executionAttemptId)) { + return Acknowledge.get(); + } else { + throw new Exception("Could not find task."); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorGateway.java new file mode 100644 index 0000000000000..450423e8111ff --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorGateway.java @@ -0,0 +1,48 @@ +/* + * 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.rpc.taskexecutor; + +import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.rpc.RpcGateway; +import scala.concurrent.Future; + +/** + * {@link TaskExecutor} rpc gateway interface + */ +public interface TaskExecutorGateway extends RpcGateway { + /** + * Execute the given task on the task executor. The task is described by the provided + * {@link TaskDeploymentDescriptor}. + * + * @param taskDeploymentDescriptor Descriptor for the task to be executed + * @return Future acknowledge of the start of the task execution + */ + Future executeTask(TaskDeploymentDescriptor taskDeploymentDescriptor); + + /** + * Cancel a task identified by it {@link ExecutionAttemptID}. If the task cannot be found, then + * the method throws an {@link Exception}. + * + * @param executionAttemptId Execution attempt ID identifying the task to be canceled. + * @return Future acknowledge of the task canceling + */ + Future cancelTask(ExecutionAttemptID executionAttemptId); +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java new file mode 100644 index 0000000000000..0ded25e4cc7fc --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java @@ -0,0 +1,327 @@ +/* + * 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.rpc; + +import org.apache.flink.util.TestLogger; +import org.junit.Test; +import org.reflections.Reflections; +import scala.concurrent.Future; + +import java.lang.annotation.Annotation; +import java.lang.reflect.Method; +import java.lang.reflect.ParameterizedType; +import java.lang.reflect.Type; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +public class RpcCompletenessTest extends TestLogger { + private static final Class futureClass = Future.class; + + @Test + public void testRpcCompleteness() { + Reflections reflections = new Reflections("org.apache.flink"); + + Set> classes = reflections.getSubTypesOf(RpcEndpoint.class); + + Class c; + + for (Class rpcEndpoint :classes){ + c = rpcEndpoint; + Type superClass = c.getGenericSuperclass(); + + Class rpcGatewayType = extractTypeParameter(superClass, 0); + + if (rpcGatewayType != null) { + checkCompleteness(rpcEndpoint, (Class) rpcGatewayType); + } else { + fail("Could not retrieve the rpc gateway class for the given rpc endpoint class " + rpcEndpoint.getName()); + } + } + } + + private void checkCompleteness(Class rpcEndpoint, Class rpcGateway) { + Method[] gatewayMethods = rpcGateway.getDeclaredMethods(); + Method[] serverMethods = rpcEndpoint.getDeclaredMethods(); + + Map> rpcMethods = new HashMap<>(); + Set unmatchedRpcMethods = new HashSet<>(); + + for (Method serverMethod : serverMethods) { + if (serverMethod.isAnnotationPresent(RpcMethod.class)) { + if (rpcMethods.containsKey(serverMethod.getName())) { + Set methods = rpcMethods.get(serverMethod.getName()); + methods.add(serverMethod); + + rpcMethods.put(serverMethod.getName(), methods); + } else { + Set methods = new HashSet<>(); + methods.add(serverMethod); + + rpcMethods.put(serverMethod.getName(), methods); + } + + unmatchedRpcMethods.add(serverMethod); + } + } + + for (Method gatewayMethod : gatewayMethods) { + assertTrue( + "The rpc endpoint " + rpcEndpoint.getName() + " does not contain a RpcMethod " + + "annotated method with the same name and signature " + + generateEndpointMethodSignature(gatewayMethod) + ".", + rpcMethods.containsKey(gatewayMethod.getName())); + + checkGatewayMethod(gatewayMethod); + + if (!matchGatewayMethodWithEndpoint(gatewayMethod, rpcMethods.get(gatewayMethod.getName()), unmatchedRpcMethods)) { + fail("Could not find a RpcMethod annotated method in rpc endpoint " + + rpcEndpoint.getName() + " matching the rpc gateway method " + + generateEndpointMethodSignature(gatewayMethod) + " defined in the rpc gateway " + + rpcGateway.getName() + "."); + } + } + + if (!unmatchedRpcMethods.isEmpty()) { + StringBuilder builder = new StringBuilder(); + + for (Method unmatchedRpcMethod : unmatchedRpcMethods) { + builder.append(unmatchedRpcMethod).append("\n"); + } + + fail("The rpc endpoint " + rpcEndpoint.getName() + " contains rpc methods which " + + "are not matched to gateway methods of " + rpcGateway.getName() + ":\n" + + builder.toString()); + } + } + + /** + * Checks whether the gateway method fulfills the gateway method requirements. + *
    + *
  • It checks whether the return type is void or a {@link Future} wrapping the actual result.
  • + *
  • It checks that the method's parameter list contains at most one parameter annotated with {@link RpcTimeout}.
  • + *
+ * + * @param gatewayMethod Gateway method to check + */ + private void checkGatewayMethod(Method gatewayMethod) { + if (!gatewayMethod.getReturnType().equals(Void.TYPE)) { + assertTrue( + "The return type of method " + gatewayMethod.getName() + " in the rpc gateway " + + gatewayMethod.getDeclaringClass().getName() + " is non void and not a " + + "future. Non-void return types have to be returned as a future.", + gatewayMethod.getReturnType().equals(futureClass)); + } + + Annotation[][] parameterAnnotations = gatewayMethod.getParameterAnnotations(); + int rpcTimeoutParameters = 0; + + for (Annotation[] parameterAnnotation : parameterAnnotations) { + for (Annotation annotation : parameterAnnotation) { + if (annotation.equals(RpcTimeout.class)) { + rpcTimeoutParameters++; + } + } + } + + assertTrue("The gateway method " + gatewayMethod + " must have at most one RpcTimeout " + + "annotated parameter.", rpcTimeoutParameters <= 1); + } + + /** + * Checks whether we find a matching overloaded version for the gateway method among the methods + * with the same name in the rpc endpoint. + * + * @param gatewayMethod Gateway method + * @param endpointMethods Set of rpc methods on the rpc endpoint with the same name as the gateway + * method + * @param unmatchedRpcMethods Set of unmatched rpc methods on the endpoint side (so far) + */ + private boolean matchGatewayMethodWithEndpoint(Method gatewayMethod, Set endpointMethods, Set unmatchedRpcMethods) { + for (Method endpointMethod : endpointMethods) { + if (checkMethod(gatewayMethod, endpointMethod)) { + unmatchedRpcMethods.remove(endpointMethod); + return true; + } + } + + return false; + } + + private boolean checkMethod(Method gatewayMethod, Method endpointMethod) { + Class[] gatewayParameterTypes = gatewayMethod.getParameterTypes(); + Annotation[][] gatewayParameterAnnotations = gatewayMethod.getParameterAnnotations(); + + Class[] endpointParameterTypes = endpointMethod.getParameterTypes(); + + List> filteredGatewayParameterTypes = new ArrayList<>(); + + assertEquals(gatewayParameterTypes.length, gatewayParameterAnnotations.length); + + // filter out the RpcTimeout parameters + for (int i = 0; i < gatewayParameterTypes.length; i++) { + if (!isRpcTimeout(gatewayParameterAnnotations[i])) { + filteredGatewayParameterTypes.add(gatewayParameterTypes[i]); + } + } + + if (filteredGatewayParameterTypes.size() != endpointParameterTypes.length) { + return false; + } else { + // check the parameter types + for (int i = 0; i < filteredGatewayParameterTypes.size(); i++) { + if (!checkType(filteredGatewayParameterTypes.get(i), endpointParameterTypes[i])) { + return false; + } + } + + // check the return types + if (endpointMethod.getReturnType() == void.class) { + if (gatewayMethod.getReturnType() != void.class) { + return false; + } + } else { + // has return value. The gateway method should be wrapped in a future + Class futureClass = gatewayMethod.getReturnType(); + + // sanity check that the return type of a gateway method must be void or a future + if (!futureClass.equals(RpcCompletenessTest.futureClass)) { + return false; + } else { + Class valueClass = extractTypeParameter(futureClass, 0); + + if (endpointMethod.getReturnType().equals(futureClass)) { + Class rpcEndpointValueClass = extractTypeParameter(endpointMethod.getReturnType(), 0); + + // check if we have the same future value types + if (valueClass != null && rpcEndpointValueClass != null && !checkType(valueClass, rpcEndpointValueClass)) { + return false; + } + } else { + if (valueClass != null && !checkType(valueClass, endpointMethod.getReturnType())) { + return false; + } + } + } + } + + return gatewayMethod.getName().equals(endpointMethod.getName()); + } + } + + private boolean checkType(Class firstType, Class secondType) { + return firstType.equals(secondType); + } + + /** + * Generates from a gateway rpc method signature the corresponding rpc endpoint signature. + * + * For example the {@link RpcTimeout} annotation adds an additional parameter to the gateway + * signature which is not relevant on the server side. + * + * @param method Method to generate the signature string for + * @return String of the respective server side rpc method signature + */ + private String generateEndpointMethodSignature(Method method) { + StringBuilder builder = new StringBuilder(); + + if (method.getReturnType().equals(Void.TYPE)) { + builder.append("void").append(" "); + } else if (method.getReturnType().equals(futureClass)) { + Class valueClass = extractTypeParameter(method.getGenericReturnType(), 0); + + builder + .append(futureClass.getSimpleName()) + .append("<") + .append(valueClass != null ? valueClass.getSimpleName() : "") + .append(">"); + + if (valueClass != null) { + builder.append("/").append(valueClass.getSimpleName()); + } + + builder.append(" "); + } else { + return "Invalid rpc method signature."; + } + + builder.append(method.getName()).append("("); + + Class[] parameterTypes = method.getParameterTypes(); + Annotation[][] parameterAnnotations = method.getParameterAnnotations(); + + assertEquals(parameterTypes.length, parameterAnnotations.length); + + for (int i = 0; i < parameterTypes.length; i++) { + // filter out the RpcTimeout parameters + if (!isRpcTimeout(parameterAnnotations[i])) { + builder.append(parameterTypes[i].getName()); + + if (i < parameterTypes.length -1) { + builder.append(", "); + } + } + } + + builder.append(")"); + + return builder.toString(); + } + + private Class extractTypeParameter(Type genericType, int position) { + if (genericType instanceof ParameterizedType) { + ParameterizedType parameterizedType = (ParameterizedType) genericType; + + Type[] typeArguments = parameterizedType.getActualTypeArguments(); + + if (position < 0 || position >= typeArguments.length) { + throw new IndexOutOfBoundsException("The generic type " + + parameterizedType.getRawType() + " only has " + typeArguments.length + + " type arguments."); + } else { + Type typeArgument = typeArguments[position]; + + if (typeArgument instanceof Class) { + return (Class) typeArgument; + } else { + return null; + } + } + } else { + return null; + } + } + + private boolean isRpcTimeout(Annotation[] annotations) { + for (Annotation annotation : annotations) { + if (annotation.annotationType().equals(RpcTimeout.class)) { + return true; + } + } + + return false; + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java new file mode 100644 index 0000000000000..c5bac9471cca1 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java @@ -0,0 +1,81 @@ +/* + * 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.rpc.akka; + +import akka.actor.ActorSystem; +import akka.util.Timeout; +import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.rpc.jobmaster.JobMaster; +import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.rpc.resourcemanager.ResourceManager; +import org.apache.flink.util.TestLogger; +import org.junit.Test; +import scala.concurrent.duration.Deadline; +import scala.concurrent.duration.FiniteDuration; + +import java.util.concurrent.ExecutorService; +import java.util.concurrent.ForkJoinPool; +import java.util.concurrent.TimeUnit; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class AkkaRpcServiceTest extends TestLogger { + + /** + * Tests that the {@link JobMaster} can connect to the {@link ResourceManager} using the + * {@link AkkaRpcService}. + */ + @Test + public void testJobMasterResourceManagerRegistration() throws Exception { + Timeout akkaTimeout = new Timeout(10, TimeUnit.SECONDS); + ActorSystem actorSystem = AkkaUtils.createDefaultActorSystem(); + ActorSystem actorSystem2 = AkkaUtils.createDefaultActorSystem(); + AkkaRpcService akkaRpcService = new AkkaRpcService(actorSystem, akkaTimeout); + AkkaRpcService akkaRpcService2 = new AkkaRpcService(actorSystem2, akkaTimeout); + ExecutorService executorService = new ForkJoinPool(); + + ResourceManager resourceManager = new ResourceManager(akkaRpcService, executorService); + JobMaster jobMaster = new JobMaster(akkaRpcService2, executorService); + + resourceManager.start(); + + ResourceManagerGateway rm = resourceManager.getSelf(); + + assertTrue(rm instanceof AkkaGateway); + + AkkaGateway akkaClient = (AkkaGateway) rm; + + jobMaster.start(); + jobMaster.registerAtResourceManager(AkkaUtils.getAkkaURL(actorSystem, akkaClient.getActorRef())); + + // wait for successful registration + FiniteDuration timeout = new FiniteDuration(20, TimeUnit.SECONDS); + Deadline deadline = timeout.fromNow(); + + while (deadline.hasTimeLeft() && !jobMaster.isConnected()) { + Thread.sleep(100); + } + + assertFalse(deadline.isOverdue()); + + jobMaster.shutDown(); + resourceManager.shutDown(); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java new file mode 100644 index 0000000000000..c1435278ad735 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.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.runtime.rpc.taskexecutor; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.blob.BlobKey; +import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; +import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; +import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.util.DirectExecutorService; +import org.apache.flink.util.SerializedValue; +import org.apache.flink.util.TestLogger; +import org.junit.Test; + +import java.net.URL; +import java.util.Collections; + +import static org.junit.Assert.fail; +import static org.mockito.Mockito.mock; + +public class TaskExecutorTest extends TestLogger { + + /** + * Tests that we can deploy and cancel a task on the TaskExecutor without exceptions + */ + @Test + public void testTaskExecution() throws Exception { + RpcService testingRpcService = mock(RpcService.class); + DirectExecutorService directExecutorService = null; + TaskExecutor taskExecutor = new TaskExecutor(testingRpcService, directExecutorService); + + TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor( + new JobID(), + "Test job", + new JobVertexID(), + new ExecutionAttemptID(), + new SerializedValue(null), + "Test task", + 0, + 1, + 0, + new Configuration(), + new Configuration(), + "Invokable", + Collections.emptyList(), + Collections.emptyList(), + Collections.emptyList(), + Collections.emptyList(), + 0 + ); + + Acknowledge ack = taskExecutor.executeTask(tdd); + + ack = taskExecutor.cancelTask(tdd.getExecutionId()); + } + + /** + * Tests that cancelling a non-existing task will return an exception + */ + @Test(expected=Exception.class) + public void testWrongTaskCancellation() throws Exception { + RpcService testingRpcService = mock(RpcService.class); + DirectExecutorService directExecutorService = null; + TaskExecutor taskExecutor = new TaskExecutor(testingRpcService, directExecutorService); + + taskExecutor.cancelTask(new ExecutionAttemptID()); + + fail("The cancellation should have thrown an exception."); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/DirectExecutorService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/DirectExecutorService.java new file mode 100644 index 0000000000000..1d7c971d19c39 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/DirectExecutorService.java @@ -0,0 +1,234 @@ +/* + * 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.util; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.CancellationException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +public class DirectExecutorService implements ExecutorService { + private boolean _shutdown = false; + + @Override + public void shutdown() { + _shutdown = true; + } + + @Override + public List shutdownNow() { + _shutdown = true; + return Collections.emptyList(); + } + + @Override + public boolean isShutdown() { + return _shutdown; + } + + @Override + public boolean isTerminated() { + return _shutdown; + } + + @Override + public boolean awaitTermination(long timeout, TimeUnit unit) throws InterruptedException { + return _shutdown; + } + + @Override + public Future submit(Callable task) { + try { + T result = task.call(); + + return new CompletedFuture<>(result, null); + } catch (Exception e) { + return new CompletedFuture<>(null, e); + } + } + + @Override + public Future submit(Runnable task, T result) { + task.run(); + + return new CompletedFuture<>(result, null); + } + + @Override + public Future submit(Runnable task) { + task.run(); + return new CompletedFuture<>(null, null); + } + + @Override + public List> invokeAll(Collection> tasks) throws InterruptedException { + ArrayList> result = new ArrayList<>(); + + for (Callable task : tasks) { + try { + result.add(new CompletedFuture(task.call(), null)); + } catch (Exception e) { + result.add(new CompletedFuture(null, e)); + } + } + return result; + } + + @Override + public List> invokeAll(Collection> tasks, long timeout, TimeUnit unit) throws InterruptedException { + long end = System.currentTimeMillis() + unit.toMillis(timeout); + Iterator> iterator = tasks.iterator(); + ArrayList> result = new ArrayList<>(); + + while (end > System.currentTimeMillis() && iterator.hasNext()) { + Callable callable = iterator.next(); + + try { + result.add(new CompletedFuture(callable.call(), null)); + } catch (Exception e) { + result.add(new CompletedFuture(null, e)); + } + } + + while(iterator.hasNext()) { + iterator.next(); + result.add(new Future() { + @Override + public boolean cancel(boolean mayInterruptIfRunning) { + return false; + } + + @Override + public boolean isCancelled() { + return true; + } + + @Override + public boolean isDone() { + return false; + } + + @Override + public T get() throws InterruptedException, ExecutionException { + throw new CancellationException("Task has been cancelled."); + } + + @Override + public T get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { + throw new CancellationException("Task has been cancelled."); + } + }); + } + + return result; + } + + @Override + public T invokeAny(Collection> tasks) throws InterruptedException, ExecutionException { + Exception exception = null; + + for (Callable task : tasks) { + try { + return task.call(); + } catch (Exception e) { + // try next task + exception = e; + } + } + + throw new ExecutionException("No tasks finished successfully.", exception); + } + + @Override + public T invokeAny(Collection> tasks, long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { + long end = System.currentTimeMillis() + unit.toMillis(timeout); + Exception exception = null; + + Iterator> iterator = tasks.iterator(); + + while (end > System.currentTimeMillis() && iterator.hasNext()) { + Callable callable = iterator.next(); + + try { + return callable.call(); + } catch (Exception e) { + // ignore exception and try next + exception = e; + } + } + + if (iterator.hasNext()) { + throw new TimeoutException("Could not finish execution of tasks within time."); + } else { + throw new ExecutionException("No tasks finished successfully.", exception); + } + } + + @Override + public void execute(Runnable command) { + command.run(); + } + + public static class CompletedFuture implements Future { + private final V value; + private final Exception exception; + + public CompletedFuture(V value, Exception exception) { + this.value = value; + this.exception = exception; + } + + @Override + public boolean cancel(boolean mayInterruptIfRunning) { + return false; + } + + @Override + public boolean isCancelled() { + return false; + } + + @Override + public boolean isDone() { + return true; + } + + @Override + public V get() throws InterruptedException, ExecutionException { + if (exception != null) { + throw new ExecutionException(exception); + } else { + return value; + } + } + + @Override + public V get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { + return get(); + } + } +} diff --git a/flink-tests/pom.xml b/flink-tests/pom.xml index efc95ab6888ba..d1d0bf3470908 100644 --- a/flink-tests/pom.xml +++ b/flink-tests/pom.xml @@ -202,7 +202,6 @@ under the License. org.reflections reflections - 0.9.10 diff --git a/pom.xml b/pom.xml index 7e517e9aa428a..1647e1d5e6a12 100644 --- a/pom.xml +++ b/pom.xml @@ -413,6 +413,13 @@ under the License. jackson-annotations ${jackson.version} + + + org.reflections + reflections + 0.9.10 + test + From 675bf2aa9e08d9d2eeb2d0caa508342af331fe2c Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Wed, 10 Aug 2016 18:27:21 +0200 Subject: [PATCH 02/50] [FLINK-4368] [distributed runtime] Eagerly initialize the RPC endpoint members This closes #2351 --- .../flink/runtime/rpc/MainThreadExecutor.java | 9 +- .../apache/flink/runtime/rpc/RpcEndpoint.java | 156 +++++++++++------- .../runtime/rpc/akka/AkkaRpcServiceTest.java | 4 +- 3 files changed, 99 insertions(+), 70 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java index e06711e27b6e6..14b2997bee83c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java @@ -26,22 +26,23 @@ /** * Interface to execute {@link Runnable} and {@link Callable} in the main thread of the underlying - * rpc server. + * RPC endpoint. * - * This interface is intended to be implemented by the self gateway in a {@link RpcEndpoint} + *

This interface is intended to be implemented by the self gateway in a {@link RpcEndpoint} * implementation which allows to dispatch local procedures to the main thread of the underlying * rpc server. */ public interface MainThreadExecutor { + /** - * Execute the runnable in the main thread of the underlying rpc server. + * Execute the runnable in the main thread of the underlying RPC endpoint. * * @param runnable Runnable to be executed */ void runAsync(Runnable runnable); /** - * Execute the callable in the main thread of the underlying rpc server and return a future for + * Execute the callable in the main thread of the underlying RPC endpoint and return a future for * the callable result. If the future is not completed within the given timeout, the returned * future will throw a {@link TimeoutException}. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java index 3d8757f8f77aa..0d928a88d1e4a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java @@ -19,85 +19,116 @@ package org.apache.flink.runtime.rpc; import akka.util.Timeout; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import scala.concurrent.ExecutionContext; import scala.concurrent.Future; import java.util.concurrent.Callable; +import static org.apache.flink.util.Preconditions.checkNotNull; + /** - * Base class for rpc endpoints. Distributed components which offer remote procedure calls have to - * extend the rpc endpoint base class. + * Base class for RPC endpoints. Distributed components which offer remote procedure calls have to + * extend the RPC endpoint base class. An RPC endpoint is backed by an {@link RpcService}. + * + *

Endpoint and Gateway

+ * + * To be done... + * + *

Single Threaded Endpoint Execution

+ * + *

All RPC calls on the same endpoint are called by the same thread + * (referred to as the endpoint's main thread). + * Thus, by executing all state changing operations within the main + * thread, we don't have to reason about concurrent accesses, in the same way in the Actor Model + * of Erlang or Akka. * - * The main idea is that a rpc endpoint is backed by a rpc server which has a single thread - * processing the rpc calls. Thus, by executing all state changing operations within the main - * thread, we don't have to reason about concurrent accesses. The rpc provides provides - * {@link #runAsync(Runnable)}, {@link #callAsync(Callable, Timeout)} and the - * {@link #getMainThreadExecutionContext()} to execute code in the rpc server's main thread. + *

The RPC endpoint provides provides {@link #runAsync(Runnable)}, {@link #callAsync(Callable, Timeout)} + * and the {@link #getMainThreadExecutionContext()} to execute code in the RPC endoint's main thread. * - * @param Rpc gateway counterpart for the implementing rpc endpoint + * @param The RPC gateway counterpart for the implementing RPC endpoint */ public abstract class RpcEndpoint { protected final Logger log = LoggerFactory.getLogger(getClass()); - /** Rpc service to be used to start the rpc server and to obtain rpc gateways */ + // ------------------------------------------------------------------------ + + /** RPC service to be used to start the RPC server and to obtain rpc gateways */ private final RpcService rpcService; /** Self gateway which can be used to schedule asynchronous calls on yourself */ - private C self; + private final C self; + + /** the fully qualified address of the this RPC endpoint */ + private final String selfAddress; + + /** The main thread execution context to be used to execute future callbacks in the main thread + * of the executing rpc server. */ + private final MainThreadExecutionContext mainThreadExecutionContext; + /** - * The main thread execution context to be used to execute future callbacks in the main thread - * of the executing rpc server. - * - * IMPORTANT: The main thread context is only available after the rpc server has been started. + * Initializes the RPC endpoint. + * + * @param rpcService The RPC server that dispatches calls to this RPC endpoint. */ - private MainThreadExecutionContext mainThreadExecutionContext; - public RpcEndpoint(RpcService rpcService) { - this.rpcService = rpcService; + this.rpcService = checkNotNull(rpcService, "rpcService"); + this.self = rpcService.startServer(this); + this.selfAddress = rpcService.getAddress(self); + this.mainThreadExecutionContext = new MainThreadExecutionContext((MainThreadExecutor) self); } + // ------------------------------------------------------------------------ + // Shutdown + // ------------------------------------------------------------------------ + /** - * Get self-gateway which should be used to run asynchronous rpc calls on this endpoint. - * - * IMPORTANT: Always issue local method calls via the self-gateway if the current thread - * is not the main thread of the underlying rpc server, e.g. from within a future callback. - * - * @return Self gateway + * Shuts down the underlying RPC endpoint via the RPC service. + * After this method was called, the RPC endpoint will no longer be reachable, neither remotely, + * not via its {@link #getSelf() self gateway}. It will also not accepts executions in main thread + * any more (via {@link #callAsync(Callable, Timeout)} and {@link #runAsync(Runnable)}). + * + *

This method can be overridden to add RPC endpoint specific shut down code. + * The overridden method should always call the parent shut down method. */ - public C getSelf() { - return self; + public void shutDown() { + rpcService.stopServer(self); } + // ------------------------------------------------------------------------ + // Basic RPC endpoint properties + // ------------------------------------------------------------------------ + /** - * Execute the runnable in the main thread of the underlying rpc server. + * Get self-gateway which should be used to run asynchronous RPC calls on this endpoint. + * + *

IMPORTANT: Always issue local method calls via the self-gateway if the current thread + * is not the main thread of the underlying rpc server, e.g. from within a future callback. * - * @param runnable Runnable to be executed in the main thread of the underlying rpc server + * @return The self gateway */ - public void runAsync(Runnable runnable) { - ((MainThreadExecutor) self).runAsync(runnable); + public C getSelf() { + return self; } /** - * Execute the callable in the main thread of the underlying rpc server returning a future for - * the result of the callable. If the callable is not completed within the given timeout, then - * the future will be failed with a {@link java.util.concurrent.TimeoutException}. + * Gets the address of the underlying RPC endpoint. The address should be fully qualified so that + * a remote system can connect to this RPC endpoint via this address. * - * @param callable Callable to be executed in the main thread of the underlying rpc server - * @param timeout Timeout for the callable to be completed - * @param Return type of the callable - * @return Future for the result of the callable. + * @return Fully qualified address of the underlying RPC endpoint */ - public Future callAsync(Callable callable, Timeout timeout) { - return ((MainThreadExecutor) self).callAsync(callable, timeout); + public String getAddress() { + return selfAddress; } /** * Gets the main thread execution context. The main thread execution context can be used to - * execute tasks in the main thread of the underlying rpc server. + * execute tasks in the main thread of the underlying RPC endpoint. * * @return Main thread execution context */ @@ -106,52 +137,51 @@ public ExecutionContext getMainThreadExecutionContext() { } /** - * Gets the used rpc service. + * Gets the endpoint's RPC service. * - * @return Rpc service + * @return The endpoint's RPC service */ public RpcService getRpcService() { return rpcService; } - /** - * Starts the underlying rpc server via the rpc service and creates the main thread execution - * context. This makes the rpc endpoint effectively reachable from the outside. - * - * Can be overriden to add rpc endpoint specific start up code. Should always call the parent - * start method. - */ - public void start() { - self = rpcService.startServer(this); - mainThreadExecutionContext = new MainThreadExecutionContext((MainThreadExecutor) self); - } - + // ------------------------------------------------------------------------ + // Asynchronous executions + // ------------------------------------------------------------------------ /** - * Shuts down the underlying rpc server via the rpc service. + * Execute the runnable in the main thread of the underlying RPC endpoint. * - * Can be overriden to add rpc endpoint specific shut down code. Should always call the parent - * shut down method. + * @param runnable Runnable to be executed in the main thread of the underlying RPC endpoint */ - public void shutDown() { - rpcService.stopServer(self); + public void runAsync(Runnable runnable) { + ((MainThreadExecutor) self).runAsync(runnable); } /** - * Gets the address of the underlying rpc server. The address should be fully qualified so that - * a remote system can connect to this rpc server via this address. + * Execute the callable in the main thread of the underlying RPC service, returning a future for + * the result of the callable. If the callable is not completed within the given timeout, then + * the future will be failed with a {@link java.util.concurrent.TimeoutException}. * - * @return Fully qualified address of the underlying rpc server + * @param callable Callable to be executed in the main thread of the underlying rpc server + * @param timeout Timeout for the callable to be completed + * @param Return type of the callable + * @return Future for the result of the callable. */ - public String getAddress() { - return rpcService.getAddress(self); + public Future callAsync(Callable callable, Timeout timeout) { + return ((MainThreadExecutor) self).callAsync(callable, timeout); } + // ------------------------------------------------------------------------ + // Utilities + // ------------------------------------------------------------------------ + /** * Execution context which executes runnables in the main thread context. A reported failure * will cause the underlying rpc server to shut down. */ private class MainThreadExecutionContext implements ExecutionContext { + private final MainThreadExecutor gateway; MainThreadExecutionContext(MainThreadExecutor gateway) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java index c5bac9471cca1..642a380593906 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java @@ -54,15 +54,13 @@ public void testJobMasterResourceManagerRegistration() throws Exception { ResourceManager resourceManager = new ResourceManager(akkaRpcService, executorService); JobMaster jobMaster = new JobMaster(akkaRpcService2, executorService); - resourceManager.start(); - ResourceManagerGateway rm = resourceManager.getSelf(); assertTrue(rm instanceof AkkaGateway); AkkaGateway akkaClient = (AkkaGateway) rm; - jobMaster.start(); + jobMaster.registerAtResourceManager(AkkaUtils.getAkkaURL(actorSystem, akkaClient.getActorRef())); // wait for successful registration From 9e6ff60ed550f6cbf3816a757ce53ebfc3b99070 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 10 Aug 2016 18:42:26 +0200 Subject: [PATCH 03/50] [FLINK-4362] [rpc] Auto generate rpc gateways via Java proxies This PR introduces a generic AkkaRpcActor which receives rpc calls as a RpcInvocation message. The RpcInvocation message is generated by the AkkaInvocationHandler which gets them from automatically generated Java Proxies. Add documentation for proxy based akka rpc service Log unknown message type in AkkaRpcActor but do not fail actor Use ReflectionUtil to extract RpcGateway type from RpcEndpoint This closes #2357. --- .../org/apache/flink/util/ReflectionUtil.java | 10 +- .../flink/runtime/rpc/MainThreadExecutor.java | 4 +- .../apache/flink/runtime/rpc/RpcEndpoint.java | 22 +- .../apache/flink/runtime/rpc/RpcService.java | 2 +- .../flink/runtime/rpc/akka/AkkaGateway.java | 4 +- .../rpc/akka/AkkaInvocationHandler.java | 226 ++++++++++++++++++ .../flink/runtime/rpc/akka/AkkaRpcActor.java | 175 ++++++++++++++ .../runtime/rpc/akka/AkkaRpcService.java | 121 +++++----- .../flink/runtime/rpc/akka/BaseAkkaActor.java | 50 ---- .../runtime/rpc/akka/BaseAkkaGateway.java | 41 ---- .../akka/jobmaster/JobMasterAkkaActor.java | 58 ----- .../akka/jobmaster/JobMasterAkkaGateway.java | 57 ----- .../{CallableMessage.java => CallAsync.java} | 18 +- .../runtime/rpc/akka/messages/CancelTask.java | 36 --- .../rpc/akka/messages/ExecuteTask.java | 36 --- .../messages/RegisterAtResourceManager.java | 36 --- .../rpc/akka/messages/RegisterJobMaster.java | 36 --- .../rpc/akka/messages/RequestSlot.java | 37 --- .../rpc/akka/messages/RpcInvocation.java | 98 ++++++++ .../{RunnableMessage.java => RunAsync.java} | 17 +- .../messages/UpdateTaskExecutionState.java | 37 --- .../ResourceManagerAkkaActor.java | 65 ----- .../ResourceManagerAkkaGateway.java | 67 ------ .../taskexecutor/TaskExecutorAkkaActor.java | 77 ------ .../taskexecutor/TaskExecutorAkkaGateway.java | 59 ----- .../runtime/rpc/jobmaster/JobMaster.java | 4 +- .../rpc/resourcemanager/ResourceManager.java | 4 +- .../rpc/taskexecutor/TaskExecutor.java | 4 +- .../runtime/rpc/RpcCompletenessTest.java | 50 ++-- .../runtime/rpc/akka/AkkaRpcServiceTest.java | 4 +- .../rpc/taskexecutor/TaskExecutorTest.java | 2 +- 31 files changed, 645 insertions(+), 812 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/BaseAkkaActor.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/BaseAkkaGateway.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/jobmaster/JobMasterAkkaActor.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/jobmaster/JobMasterAkkaGateway.java rename flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/{CallableMessage.java => CallAsync.java} (68%) delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CancelTask.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/ExecuteTask.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RegisterAtResourceManager.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RegisterJobMaster.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RequestSlot.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RpcInvocation.java rename flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/{RunnableMessage.java => RunAsync.java} (70%) delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/UpdateTaskExecutionState.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/resourcemanager/ResourceManagerAkkaActor.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/resourcemanager/ResourceManagerAkkaGateway.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/taskexecutor/TaskExecutorAkkaActor.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/taskexecutor/TaskExecutorAkkaGateway.java diff --git a/flink-core/src/main/java/org/apache/flink/util/ReflectionUtil.java b/flink-core/src/main/java/org/apache/flink/util/ReflectionUtil.java index fe2d4c01f1d12..b851ebacdfd88 100644 --- a/flink-core/src/main/java/org/apache/flink/util/ReflectionUtil.java +++ b/flink-core/src/main/java/org/apache/flink/util/ReflectionUtil.java @@ -48,6 +48,14 @@ public static Class getTemplateType1(Class clazz) { return getTemplateType(clazz, 0); } + public static Class getTemplateType1(Type type) { + if (type instanceof ParameterizedType) { + return (Class) getTemplateTypes((ParameterizedType) type)[0]; + } else { + throw new IllegalArgumentException(); + } + } + public static Class getTemplateType2(Class clazz) { return getTemplateType(clazz, 1); } @@ -123,7 +131,7 @@ public static Class[] getTemplateTypes(ParameterizedType paramterizedType) { Class[] types = new Class[paramterizedType.getActualTypeArguments().length]; int i = 0; for (Type templateArgument : paramterizedType.getActualTypeArguments()) { - assert (templateArgument instanceof Class); + assert templateArgument instanceof Class; types[i++] = (Class) templateArgument; } return types; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java index 14b2997bee83c..882c1b751e5e3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java @@ -47,9 +47,9 @@ public interface MainThreadExecutor { * future will throw a {@link TimeoutException}. * * @param callable Callable to be executed - * @param timeout Timeout for the future to complete + * @param callTimeout Timeout for the future to complete * @param Return value of the callable * @return Future of the callable result */ - Future callAsync(Callable callable, Timeout timeout); + Future callAsync(Callable callable, Timeout callTimeout); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java index 0d928a88d1e4a..aef08031093ca 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java @@ -20,6 +20,7 @@ import akka.util.Timeout; +import org.apache.flink.util.ReflectionUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -60,6 +61,9 @@ public abstract class RpcEndpoint { /** RPC service to be used to start the RPC server and to obtain rpc gateways */ private final RpcService rpcService; + /** Class of the self gateway */ + private final Class selfGatewayType; + /** Self gateway which can be used to schedule asynchronous calls on yourself */ private final C self; @@ -70,15 +74,19 @@ public abstract class RpcEndpoint { * of the executing rpc server. */ private final MainThreadExecutionContext mainThreadExecutionContext; - /** * Initializes the RPC endpoint. * * @param rpcService The RPC server that dispatches calls to this RPC endpoint. */ - public RpcEndpoint(RpcService rpcService) { + protected RpcEndpoint(final RpcService rpcService) { this.rpcService = checkNotNull(rpcService, "rpcService"); + + // IMPORTANT: Don't change order of selfGatewayType and self because rpcService.startServer + // requires that selfGatewayType has been initialized + this.selfGatewayType = ReflectionUtil.getTemplateType1(getClass()); this.self = rpcService.startServer(this); + this.selfAddress = rpcService.getAddress(self); this.mainThreadExecutionContext = new MainThreadExecutionContext((MainThreadExecutor) self); } @@ -149,6 +157,7 @@ public RpcService getRpcService() { // Asynchronous executions // ------------------------------------------------------------------------ + /** * Execute the runnable in the main thread of the underlying RPC endpoint. * @@ -172,6 +181,15 @@ public Future callAsync(Callable callable, Timeout timeout) { return ((MainThreadExecutor) self).callAsync(callable, timeout); } + /** + * Returns the class of the self gateway type. + * + * @return Class of the self gateway type + */ + public final Class getSelfGatewayType() { + return selfGatewayType; + } + // ------------------------------------------------------------------------ // Utilities // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java index 90ff7b671ce03..f93be8361c077 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java @@ -46,7 +46,7 @@ public interface RpcService { * @param Type of the self rpc gateway associated with the rpc server * @return Self gateway to dispatch remote procedure calls to oneself */ - C startServer(S rpcEndpoint); + > C startServer(S rpcEndpoint); /** * Stop the underlying rpc server of the provided self gateway. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaGateway.java index a96a6008ba23a..a826e7dcf469e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaGateway.java @@ -23,7 +23,7 @@ /** * Interface for Akka based rpc gateways */ -public interface AkkaGateway { +interface AkkaGateway { - ActorRef getActorRef(); + ActorRef getRpcServer(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java new file mode 100644 index 0000000000000..e8e383a171da6 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java @@ -0,0 +1,226 @@ +/* + * 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.rpc.akka; + +import akka.actor.ActorRef; +import akka.pattern.Patterns; +import akka.util.Timeout; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.rpc.MainThreadExecutor; +import org.apache.flink.runtime.rpc.RpcTimeout; +import org.apache.flink.runtime.rpc.akka.messages.CallAsync; +import org.apache.flink.runtime.rpc.akka.messages.RpcInvocation; +import org.apache.flink.runtime.rpc.akka.messages.RunAsync; +import org.apache.flink.util.Preconditions; +import scala.concurrent.Await; +import scala.concurrent.Future; +import scala.concurrent.duration.FiniteDuration; + +import java.lang.annotation.Annotation; +import java.lang.reflect.InvocationHandler; +import java.lang.reflect.Method; +import java.util.BitSet; +import java.util.concurrent.Callable; + +/** + * Invocation handler to be used with a {@link AkkaRpcActor}. The invocation handler wraps the + * rpc in a {@link RpcInvocation} message and then sends it to the {@link AkkaRpcActor} where it is + * executed. + */ +class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThreadExecutor { + private final ActorRef rpcServer; + + // default timeout for asks + private final Timeout timeout; + + AkkaInvocationHandler(ActorRef rpcServer, Timeout timeout) { + this.rpcServer = Preconditions.checkNotNull(rpcServer); + this.timeout = Preconditions.checkNotNull(timeout); + } + + @Override + public Object invoke(Object proxy, Method method, Object[] args) throws Throwable { + Class declaringClass = method.getDeclaringClass(); + + Object result; + + if (declaringClass.equals(AkkaGateway.class) || declaringClass.equals(MainThreadExecutor.class) || declaringClass.equals(Object.class)) { + result = method.invoke(this, args); + } else { + String methodName = method.getName(); + Class[] parameterTypes = method.getParameterTypes(); + Annotation[][] parameterAnnotations = method.getParameterAnnotations(); + Timeout futureTimeout = extractRpcTimeout(parameterAnnotations, args, timeout); + + Tuple2[], Object[]> filteredArguments = filterArguments( + parameterTypes, + parameterAnnotations, + args); + + RpcInvocation rpcInvocation = new RpcInvocation( + methodName, + filteredArguments.f0, + filteredArguments.f1); + + Class returnType = method.getReturnType(); + + if (returnType.equals(Void.TYPE)) { + rpcServer.tell(rpcInvocation, ActorRef.noSender()); + + result = null; + } else if (returnType.equals(Future.class)) { + // execute an asynchronous call + result = Patterns.ask(rpcServer, rpcInvocation, futureTimeout); + } else { + // execute a synchronous call + Future futureResult = Patterns.ask(rpcServer, rpcInvocation, futureTimeout); + FiniteDuration duration = timeout.duration(); + + result = Await.result(futureResult, duration); + } + } + + return result; + } + + @Override + public ActorRef getRpcServer() { + return rpcServer; + } + + @Override + public void runAsync(Runnable runnable) { + // Unfortunately I couldn't find a way to allow only local communication. Therefore, the + // runnable field is transient transient + rpcServer.tell(new RunAsync(runnable), ActorRef.noSender()); + } + + @Override + public Future callAsync(Callable callable, Timeout callTimeout) { + // Unfortunately I couldn't find a way to allow only local communication. Therefore, the + // callable field is declared transient + @SuppressWarnings("unchecked") + Future result = (Future) Patterns.ask(rpcServer, new CallAsync(callable), callTimeout); + + return result; + } + + /** + * Extracts the {@link RpcTimeout} annotated rpc timeout value from the list of given method + * arguments. If no {@link RpcTimeout} annotated parameter could be found, then the default + * timeout is returned. + * + * @param parameterAnnotations Parameter annotations + * @param args Array of arguments + * @param defaultTimeout Default timeout to return if no {@link RpcTimeout} annotated parameter + * has been found + * @return Timeout extracted from the array of arguments or the default timeout + */ + private static Timeout extractRpcTimeout(Annotation[][] parameterAnnotations, Object[] args, Timeout defaultTimeout) { + if (args != null) { + Preconditions.checkArgument(parameterAnnotations.length == args.length); + + for (int i = 0; i < parameterAnnotations.length; i++) { + if (isRpcTimeout(parameterAnnotations[i])) { + if (args[i] instanceof FiniteDuration) { + return new Timeout((FiniteDuration) args[i]); + } else { + throw new RuntimeException("The rpc timeout parameter must be of type " + + FiniteDuration.class.getName() + ". The type " + args[i].getClass().getName() + + " is not supported."); + } + } + } + } + + return defaultTimeout; + } + + /** + * Removes all {@link RpcTimeout} annotated parameters from the parameter type and argument + * list. + * + * @param parameterTypes Array of parameter types + * @param parameterAnnotations Array of parameter annotations + * @param args Arary of arguments + * @return Tuple of filtered parameter types and arguments which no longer contain the + * {@link RpcTimeout} annotated parameter types and arguments + */ + private static Tuple2[], Object[]> filterArguments( + Class[] parameterTypes, + Annotation[][] parameterAnnotations, + Object[] args) { + + Class[] filteredParameterTypes; + Object[] filteredArgs; + + if (args == null) { + filteredParameterTypes = parameterTypes; + filteredArgs = null; + } else { + Preconditions.checkArgument(parameterTypes.length == parameterAnnotations.length); + Preconditions.checkArgument(parameterAnnotations.length == args.length); + + BitSet isRpcTimeoutParameter = new BitSet(parameterTypes.length); + int numberRpcParameters = parameterTypes.length; + + for (int i = 0; i < parameterTypes.length; i++) { + if (isRpcTimeout(parameterAnnotations[i])) { + isRpcTimeoutParameter.set(i); + numberRpcParameters--; + } + } + + if (numberRpcParameters == parameterTypes.length) { + filteredParameterTypes = parameterTypes; + filteredArgs = args; + } else { + filteredParameterTypes = new Class[numberRpcParameters]; + filteredArgs = new Object[numberRpcParameters]; + int counter = 0; + + for (int i = 0; i < parameterTypes.length; i++) { + if (!isRpcTimeoutParameter.get(i)) { + filteredParameterTypes[counter] = parameterTypes[i]; + filteredArgs[counter] = args[i]; + counter++; + } + } + } + } + + return Tuple2.of(filteredParameterTypes, filteredArgs); + } + + /** + * Checks whether any of the annotations is of type {@link RpcTimeout} + * + * @param annotations Array of annotations + * @return True if {@link RpcTimeout} was found; otherwise false + */ + private static boolean isRpcTimeout(Annotation[] annotations) { + for (Annotation annotation : annotations) { + if (annotation.annotationType().equals(RpcTimeout.class)) { + return true; + } + } + + return false; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java new file mode 100644 index 0000000000000..57da38a6f7bde --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java @@ -0,0 +1,175 @@ +/* + * 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.rpc.akka; + +import akka.actor.Status; +import akka.actor.UntypedActor; +import akka.pattern.Patterns; +import org.apache.flink.runtime.rpc.RpcEndpoint; +import org.apache.flink.runtime.rpc.RpcGateway; +import org.apache.flink.runtime.rpc.akka.messages.CallAsync; +import org.apache.flink.runtime.rpc.akka.messages.RpcInvocation; +import org.apache.flink.runtime.rpc.akka.messages.RunAsync; +import org.apache.flink.util.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.concurrent.Future; + +import java.lang.reflect.Method; +import java.util.concurrent.Callable; + +/** + * Akka rpc actor which receives {@link RpcInvocation}, {@link RunAsync} and {@link CallAsync} + * messages. + *

+ * The {@link RpcInvocation} designates a rpc and is dispatched to the given {@link RpcEndpoint} + * instance. + *

+ * The {@link RunAsync} and {@link CallAsync} messages contain executable code which is executed + * in the context of the actor thread. + * + * @param Type of the {@link RpcGateway} associated with the {@link RpcEndpoint} + * @param Type of the {@link RpcEndpoint} + */ +class AkkaRpcActor> extends UntypedActor { + private static final Logger LOG = LoggerFactory.getLogger(AkkaRpcActor.class); + + private final T rpcEndpoint; + + AkkaRpcActor(final T rpcEndpoint) { + this.rpcEndpoint = Preconditions.checkNotNull(rpcEndpoint, "rpc endpoint"); + } + + @Override + public void onReceive(final Object message) { + if (message instanceof RunAsync) { + handleRunAsync((RunAsync) message); + } else if (message instanceof CallAsync) { + handleCallAsync((CallAsync) message); + } else if (message instanceof RpcInvocation) { + handleRpcInvocation((RpcInvocation) message); + } else { + LOG.warn("Received message of unknown type {}. Dropping this message!", message.getClass()); + } + } + + /** + * Handle rpc invocations by looking up the rpc method on the rpc endpoint and calling this + * method with the provided method arguments. If the method has a return value, it is returned + * to the sender of the call. + * + * @param rpcInvocation Rpc invocation message + */ + private void handleRpcInvocation(RpcInvocation rpcInvocation) { + Method rpcMethod = null; + + try { + rpcMethod = lookupRpcMethod(rpcInvocation.getMethodName(), rpcInvocation.getParameterTypes()); + } catch (final NoSuchMethodException e) { + LOG.error("Could not find rpc method for rpc invocation: {}.", rpcInvocation, e); + } + + if (rpcMethod != null) { + if (rpcMethod.getReturnType().equals(Void.TYPE)) { + // No return value to send back + try { + rpcMethod.invoke(rpcEndpoint, rpcInvocation.getArgs()); + } catch (Throwable e) { + LOG.error("Error while executing remote procedure call {}.", rpcMethod, e); + } + } else { + try { + Object result = rpcMethod.invoke(rpcEndpoint, rpcInvocation.getArgs()); + + if (result instanceof Future) { + // pipe result to sender + Patterns.pipe((Future) result, getContext().dispatcher()).to(getSender()); + } else { + // tell the sender the result of the computation + getSender().tell(new Status.Success(result), getSelf()); + } + } catch (Throwable e) { + // tell the sender about the failure + getSender().tell(new Status.Failure(e), getSelf()); + } + } + } + } + + /** + * Handle asynchronous {@link Callable}. This method simply executes the given {@link Callable} + * in the context of the actor thread. + * + * @param callAsync Call async message + */ + private void handleCallAsync(CallAsync callAsync) { + if (callAsync.getCallable() == null) { + final String result = "Received a " + callAsync.getClass().getName() + " message with an empty " + + "callable field. This indicates that this message has been serialized " + + "prior to sending the message. The " + callAsync.getClass().getName() + + " is only supported with local communication."; + + LOG.warn(result); + + getSender().tell(new Status.Failure(new Exception(result)), getSelf()); + } else { + try { + Object result = callAsync.getCallable().call(); + + getSender().tell(new Status.Success(result), getSelf()); + } catch (Throwable e) { + getSender().tell(new Status.Failure(e), getSelf()); + } + } + } + + /** + * Handle asynchronous {@link Runnable}. This method simply executes the given {@link Runnable} + * in the context of the actor thread. + * + * @param runAsync Run async message + */ + private void handleRunAsync(RunAsync runAsync) { + if (runAsync.getRunnable() == null) { + LOG.warn("Received a {} message with an empty runnable field. This indicates " + + "that this message has been serialized prior to sending the message. The " + + "{} is only supported with local communication.", + runAsync.getClass().getName(), + runAsync.getClass().getName()); + } else { + try { + runAsync.getRunnable().run(); + } catch (final Throwable e) { + LOG.error("Caught exception while executing runnable in main thread.", e); + } + } + } + + /** + * Look up the rpc method on the given {@link RpcEndpoint} instance. + * + * @param methodName Name of the method + * @param parameterTypes Parameter types of the method + * @return Method of the rpc endpoint + * @throws NoSuchMethodException + */ + private Method lookupRpcMethod(final String methodName, final Class[] parameterTypes) throws NoSuchMethodException { + return rpcEndpoint.getClass().getMethod(methodName, parameterTypes); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java index d55bd135b273b..17983d01b67c4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java @@ -29,88 +29,82 @@ import akka.pattern.AskableActorSelection; import akka.util.Timeout; import org.apache.flink.runtime.akka.AkkaUtils; -import org.apache.flink.runtime.rpc.jobmaster.JobMaster; -import org.apache.flink.runtime.rpc.jobmaster.JobMasterGateway; -import org.apache.flink.runtime.rpc.resourcemanager.ResourceManager; -import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.rpc.MainThreadExecutor; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcService; -import org.apache.flink.runtime.rpc.akka.jobmaster.JobMasterAkkaActor; -import org.apache.flink.runtime.rpc.akka.jobmaster.JobMasterAkkaGateway; -import org.apache.flink.runtime.rpc.akka.resourcemanager.ResourceManagerAkkaActor; -import org.apache.flink.runtime.rpc.akka.resourcemanager.ResourceManagerAkkaGateway; -import org.apache.flink.runtime.rpc.akka.taskexecutor.TaskExecutorAkkaActor; -import org.apache.flink.runtime.rpc.akka.taskexecutor.TaskExecutorAkkaGateway; -import org.apache.flink.runtime.rpc.taskexecutor.TaskExecutorGateway; -import org.apache.flink.runtime.rpc.taskexecutor.TaskExecutor; +import org.apache.flink.util.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import scala.concurrent.Future; +import java.lang.reflect.InvocationHandler; +import java.lang.reflect.Proxy; +import java.util.Collection; import java.util.HashSet; -import java.util.Set; +/** + * Akka based {@link RpcService} implementation. The rpc service starts an Akka actor to receive + * rpcs from a {@link RpcGateway}. + */ public class AkkaRpcService implements RpcService { + private static final Logger LOG = LoggerFactory.getLogger(AkkaRpcService.class); + private final ActorSystem actorSystem; private final Timeout timeout; - private final Set actors = new HashSet<>(); + private final Collection actors = new HashSet<>(4); - public AkkaRpcService(ActorSystem actorSystem, Timeout timeout) { - this.actorSystem = actorSystem; - this.timeout = timeout; + public AkkaRpcService(final ActorSystem actorSystem, final Timeout timeout) { + this.actorSystem = Preconditions.checkNotNull(actorSystem, "actor system"); + this.timeout = Preconditions.checkNotNull(timeout, "timeout"); } @Override - public Future connect(String address, final Class clazz) { - ActorSelection actorSel = actorSystem.actorSelection(address); + public Future connect(final String address, final Class clazz) { + LOG.info("Try to connect to remote rpc server with address {}. Returning a {} gateway.", address, clazz.getName()); - AskableActorSelection asker = new AskableActorSelection(actorSel); + final ActorSelection actorSel = actorSystem.actorSelection(address); - Future identify = asker.ask(new Identify(42), timeout); + final AskableActorSelection asker = new AskableActorSelection(actorSel); + + final Future identify = asker.ask(new Identify(42), timeout); return identify.map(new Mapper(){ + @Override public C apply(Object obj) { ActorRef actorRef = ((ActorIdentity) obj).getRef(); - if (clazz == TaskExecutorGateway.class) { - return (C) new TaskExecutorAkkaGateway(actorRef, timeout); - } else if (clazz == ResourceManagerGateway.class) { - return (C) new ResourceManagerAkkaGateway(actorRef, timeout); - } else if (clazz == JobMasterGateway.class) { - return (C) new JobMasterAkkaGateway(actorRef, timeout); - } else { - throw new RuntimeException("Could not find remote endpoint " + clazz); - } + InvocationHandler akkaInvocationHandler = new AkkaInvocationHandler(actorRef, timeout); + + @SuppressWarnings("unchecked") + C proxy = (C) Proxy.newProxyInstance( + ClassLoader.getSystemClassLoader(), + new Class[] {clazz}, + akkaInvocationHandler); + + return proxy; } }, actorSystem.dispatcher()); } @Override - public C startServer(S rpcEndpoint) { - ActorRef ref; - C self; - if (rpcEndpoint instanceof TaskExecutor) { - ref = actorSystem.actorOf( - Props.create(TaskExecutorAkkaActor.class, rpcEndpoint) - ); - - self = (C) new TaskExecutorAkkaGateway(ref, timeout); - } else if (rpcEndpoint instanceof ResourceManager) { - ref = actorSystem.actorOf( - Props.create(ResourceManagerAkkaActor.class, rpcEndpoint) - ); - - self = (C) new ResourceManagerAkkaGateway(ref, timeout); - } else if (rpcEndpoint instanceof JobMaster) { - ref = actorSystem.actorOf( - Props.create(JobMasterAkkaActor.class, rpcEndpoint) - ); - - self = (C) new JobMasterAkkaGateway(ref, timeout); - } else { - throw new RuntimeException("Could not start RPC server for class " + rpcEndpoint.getClass()); - } + public > C startServer(S rpcEndpoint) { + Preconditions.checkNotNull(rpcEndpoint, "rpc endpoint"); - actors.add(ref); + LOG.info("Start Akka rpc actor to handle rpcs for {}.", rpcEndpoint.getClass().getName()); + + Props akkaRpcActorProps = Props.create(AkkaRpcActor.class, rpcEndpoint); + + ActorRef actorRef = actorSystem.actorOf(akkaRpcActorProps); + actors.add(actorRef); + + InvocationHandler akkaInvocationHandler = new AkkaInvocationHandler(actorRef, timeout); + + @SuppressWarnings("unchecked") + C self = (C) Proxy.newProxyInstance( + ClassLoader.getSystemClassLoader(), + new Class[]{rpcEndpoint.getSelfGatewayType(), MainThreadExecutor.class, AkkaGateway.class}, + akkaInvocationHandler); return self; } @@ -120,16 +114,19 @@ public void stopServer(C selfGateway) { if (selfGateway instanceof AkkaGateway) { AkkaGateway akkaClient = (AkkaGateway) selfGateway; - if (actors.contains(akkaClient.getActorRef())) { - akkaClient.getActorRef().tell(PoisonPill.getInstance(), ActorRef.noSender()); - } else { - // don't stop this actor since it was not started by this RPC service + if (actors.contains(akkaClient.getRpcServer())) { + ActorRef selfActorRef = akkaClient.getRpcServer(); + + LOG.info("Stop Akka rpc actor {}.", selfActorRef.path()); + + selfActorRef.tell(PoisonPill.getInstance(), ActorRef.noSender()); } } } @Override public void stopService() { + LOG.info("Stop Akka rpc service."); actorSystem.shutdown(); actorSystem.awaitTermination(); } @@ -137,9 +134,11 @@ public void stopService() { @Override public String getAddress(C selfGateway) { if (selfGateway instanceof AkkaGateway) { - return AkkaUtils.getAkkaURL(actorSystem, ((AkkaGateway) selfGateway).getActorRef()); + ActorRef actorRef = ((AkkaGateway) selfGateway).getRpcServer(); + return AkkaUtils.getAkkaURL(actorSystem, actorRef); } else { - throw new RuntimeException("Cannot get address for non " + AkkaGateway.class.getName() + "."); + String className = AkkaGateway.class.getName(); + throw new RuntimeException("Cannot get address for non " + className + '.'); } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/BaseAkkaActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/BaseAkkaActor.java deleted file mode 100644 index 3cb499cffca1f..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/BaseAkkaActor.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * 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.rpc.akka; - -import akka.actor.Status; -import akka.actor.UntypedActor; -import org.apache.flink.runtime.rpc.akka.messages.CallableMessage; -import org.apache.flink.runtime.rpc.akka.messages.RunnableMessage; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class BaseAkkaActor extends UntypedActor { - private static final Logger LOG = LoggerFactory.getLogger(BaseAkkaActor.class); - - @Override - public void onReceive(Object message) throws Exception { - if (message instanceof RunnableMessage) { - try { - ((RunnableMessage) message).getRunnable().run(); - } catch (Exception e) { - LOG.error("Encountered error while executing runnable.", e); - } - } else if (message instanceof CallableMessage) { - try { - Object result = ((CallableMessage) message).getCallable().call(); - sender().tell(new Status.Success(result), getSelf()); - } catch (Exception e) { - sender().tell(new Status.Failure(e), getSelf()); - } - } else { - throw new RuntimeException("Unknown message " + message); - } - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/BaseAkkaGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/BaseAkkaGateway.java deleted file mode 100644 index 512790d11639a..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/BaseAkkaGateway.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * 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.rpc.akka; - -import akka.actor.ActorRef; -import akka.pattern.Patterns; -import akka.util.Timeout; -import org.apache.flink.runtime.rpc.MainThreadExecutor; -import org.apache.flink.runtime.rpc.akka.messages.CallableMessage; -import org.apache.flink.runtime.rpc.akka.messages.RunnableMessage; -import scala.concurrent.Future; - -import java.util.concurrent.Callable; - -public abstract class BaseAkkaGateway implements MainThreadExecutor, AkkaGateway { - @Override - public void runAsync(Runnable runnable) { - getActorRef().tell(new RunnableMessage(runnable), ActorRef.noSender()); - } - - @Override - public Future callAsync(Callable callable, Timeout timeout) { - return (Future) Patterns.ask(getActorRef(), new CallableMessage(callable), timeout); - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/jobmaster/JobMasterAkkaActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/jobmaster/JobMasterAkkaActor.java deleted file mode 100644 index 9e04ea91070aa..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/jobmaster/JobMasterAkkaActor.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * 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.rpc.akka.jobmaster; - -import akka.actor.ActorRef; -import akka.actor.Status; -import org.apache.flink.runtime.rpc.akka.BaseAkkaActor; -import org.apache.flink.runtime.rpc.akka.messages.RegisterAtResourceManager; -import org.apache.flink.runtime.rpc.jobmaster.JobMaster; -import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.rpc.akka.messages.UpdateTaskExecutionState; - -public class JobMasterAkkaActor extends BaseAkkaActor { - private final JobMaster jobMaster; - - public JobMasterAkkaActor(JobMaster jobMaster) { - this.jobMaster = jobMaster; - } - - @Override - public void onReceive(Object message) throws Exception { - if (message instanceof UpdateTaskExecutionState) { - - final ActorRef sender = getSender(); - - UpdateTaskExecutionState updateTaskExecutionState = (UpdateTaskExecutionState) message; - - try { - Acknowledge result = jobMaster.updateTaskExecutionState(updateTaskExecutionState.getTaskExecutionState()); - sender.tell(new Status.Success(result), getSelf()); - } catch (Exception e) { - sender.tell(new Status.Failure(e), getSelf()); - } - } else if (message instanceof RegisterAtResourceManager) { - RegisterAtResourceManager registerAtResourceManager = (RegisterAtResourceManager) message; - - jobMaster.registerAtResourceManager(registerAtResourceManager.getAddress()); - } else { - super.onReceive(message); - } - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/jobmaster/JobMasterAkkaGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/jobmaster/JobMasterAkkaGateway.java deleted file mode 100644 index e6bf061701faf..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/jobmaster/JobMasterAkkaGateway.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * 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.rpc.akka.jobmaster; - -import akka.actor.ActorRef; -import akka.pattern.AskableActorRef; -import akka.util.Timeout; -import org.apache.flink.runtime.rpc.akka.BaseAkkaGateway; -import org.apache.flink.runtime.rpc.akka.messages.RegisterAtResourceManager; -import org.apache.flink.runtime.rpc.jobmaster.JobMasterGateway; -import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.rpc.akka.messages.UpdateTaskExecutionState; -import org.apache.flink.runtime.taskmanager.TaskExecutionState; -import scala.concurrent.Future; -import scala.reflect.ClassTag$; - -public class JobMasterAkkaGateway extends BaseAkkaGateway implements JobMasterGateway { - private final AskableActorRef actorRef; - private final Timeout timeout; - - public JobMasterAkkaGateway(ActorRef actorRef, Timeout timeout) { - this.actorRef = new AskableActorRef(actorRef); - this.timeout = timeout; - } - - @Override - public Future updateTaskExecutionState(TaskExecutionState taskExecutionState) { - return actorRef.ask(new UpdateTaskExecutionState(taskExecutionState), timeout) - .mapTo(ClassTag$.MODULE$.apply(Acknowledge.class)); - } - - @Override - public void registerAtResourceManager(String address) { - actorRef.actorRef().tell(new RegisterAtResourceManager(address), actorRef.actorRef()); - } - - @Override - public ActorRef getActorRef() { - return actorRef.actorRef(); - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CallableMessage.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CallAsync.java similarity index 68% rename from flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CallableMessage.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CallAsync.java index f0e555f729ca4..79b7825e8a3ee 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CallableMessage.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CallAsync.java @@ -18,16 +18,24 @@ package org.apache.flink.runtime.rpc.akka.messages; +import org.apache.flink.util.Preconditions; + +import java.io.Serializable; import java.util.concurrent.Callable; -public class CallableMessage { - private final Callable callable; +/** + * Message for asynchronous callable invocations + */ +public final class CallAsync implements Serializable { + private static final long serialVersionUID = 2834204738928484060L; + + private transient Callable callable; - public CallableMessage(Callable callable) { - this.callable = callable; + public CallAsync(Callable callable) { + this.callable = Preconditions.checkNotNull(callable); } - public Callable getCallable() { + public Callable getCallable() { return callable; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CancelTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CancelTask.java deleted file mode 100644 index 0b9e9dc9f6447..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CancelTask.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * 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.rpc.akka.messages; - -import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; - -import java.io.Serializable; - -public class CancelTask implements Serializable { - private static final long serialVersionUID = -2998176874447950595L; - private final ExecutionAttemptID executionAttemptID; - - public CancelTask(ExecutionAttemptID executionAttemptID) { - this.executionAttemptID = executionAttemptID; - } - - public ExecutionAttemptID getExecutionAttemptID() { - return executionAttemptID; - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/ExecuteTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/ExecuteTask.java deleted file mode 100644 index a83d539082eb2..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/ExecuteTask.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * 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.rpc.akka.messages; - -import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; - -import java.io.Serializable; - -public class ExecuteTask implements Serializable { - private static final long serialVersionUID = -6769958430967048348L; - private final TaskDeploymentDescriptor taskDeploymentDescriptor; - - public ExecuteTask(TaskDeploymentDescriptor taskDeploymentDescriptor) { - this.taskDeploymentDescriptor = taskDeploymentDescriptor; - } - - public TaskDeploymentDescriptor getTaskDeploymentDescriptor() { - return taskDeploymentDescriptor; - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RegisterAtResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RegisterAtResourceManager.java deleted file mode 100644 index 3ade08250b9ea..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RegisterAtResourceManager.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * 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.rpc.akka.messages; - -import java.io.Serializable; - -public class RegisterAtResourceManager implements Serializable { - - private static final long serialVersionUID = -4175905742620903602L; - - private final String address; - - public RegisterAtResourceManager(String address) { - this.address = address; - } - - public String getAddress() { - return address; - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RegisterJobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RegisterJobMaster.java deleted file mode 100644 index b35ea3895aa0e..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RegisterJobMaster.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * 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.rpc.akka.messages; - -import org.apache.flink.runtime.rpc.resourcemanager.JobMasterRegistration; - -import java.io.Serializable; - -public class RegisterJobMaster implements Serializable{ - private static final long serialVersionUID = -4616879574192641507L; - private final JobMasterRegistration jobMasterRegistration; - - public RegisterJobMaster(JobMasterRegistration jobMasterRegistration) { - this.jobMasterRegistration = jobMasterRegistration; - } - - public JobMasterRegistration getJobMasterRegistration() { - return jobMasterRegistration; - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RequestSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RequestSlot.java deleted file mode 100644 index 85ceeec6f0738..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RequestSlot.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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.rpc.akka.messages; - -import org.apache.flink.runtime.rpc.resourcemanager.SlotRequest; - -import java.io.Serializable; - -public class RequestSlot implements Serializable { - private static final long serialVersionUID = 7207463889348525866L; - - private final SlotRequest slotRequest; - - public RequestSlot(SlotRequest slotRequest) { - this.slotRequest = slotRequest; - } - - public SlotRequest getSlotRequest() { - return slotRequest; - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RpcInvocation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RpcInvocation.java new file mode 100644 index 0000000000000..5d52ef1c0b298 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RpcInvocation.java @@ -0,0 +1,98 @@ +/* + * 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.rpc.akka.messages; + +import org.apache.flink.util.Preconditions; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; + +/** + * Rpc invocation message containing the remote procedure name, its parameter types and the + * corresponding call arguments. + */ +public final class RpcInvocation implements Serializable { + private static final long serialVersionUID = -7058254033460536037L; + + private final String methodName; + private final Class[] parameterTypes; + private transient Object[] args; + + public RpcInvocation(String methodName, Class[] parameterTypes, Object[] args) { + this.methodName = Preconditions.checkNotNull(methodName); + this.parameterTypes = Preconditions.checkNotNull(parameterTypes); + this.args = args; + } + + public String getMethodName() { + return methodName; + } + + public Class[] getParameterTypes() { + return parameterTypes; + } + + public Object[] getArgs() { + return args; + } + + private void writeObject(ObjectOutputStream oos) throws IOException { + oos.defaultWriteObject(); + + if (args != null) { + // write has args true + oos.writeBoolean(true); + + for (int i = 0; i < args.length; i++) { + try { + oos.writeObject(args[i]); + } catch (IOException e) { + Class argClass = args[i].getClass(); + + throw new IOException("Could not write " + i + "th argument of method " + + methodName + ". The argument type is " + argClass + ". " + + "Make sure that this type is serializable.", e); + } + } + } else { + // write has args false + oos.writeBoolean(false); + } + } + + private void readObject(ObjectInputStream ois) throws IOException, ClassNotFoundException { + ois.defaultReadObject(); + + boolean hasArgs = ois.readBoolean(); + + if (hasArgs) { + int numberArguments = parameterTypes.length; + + args = new Object[numberArguments]; + + for (int i = 0; i < numberArguments; i++) { + args[i] = ois.readObject(); + } + } else { + args = null; + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunnableMessage.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunAsync.java similarity index 70% rename from flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunnableMessage.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunAsync.java index 35567388208bd..fb958525f5079 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunnableMessage.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunAsync.java @@ -18,11 +18,20 @@ package org.apache.flink.runtime.rpc.akka.messages; -public class RunnableMessage { - private final Runnable runnable; +import org.apache.flink.util.Preconditions; - public RunnableMessage(Runnable runnable) { - this.runnable = runnable; +import java.io.Serializable; + +/** + * Message for asynchronous runnable invocations + */ +public final class RunAsync implements Serializable { + private static final long serialVersionUID = -3080595100695371036L; + + private final transient Runnable runnable; + + public RunAsync(Runnable runnable) { + this.runnable = Preconditions.checkNotNull(runnable); } public Runnable getRunnable() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/UpdateTaskExecutionState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/UpdateTaskExecutionState.java deleted file mode 100644 index f89cd2f516041..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/UpdateTaskExecutionState.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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.rpc.akka.messages; - -import org.apache.flink.runtime.taskmanager.TaskExecutionState; - -import java.io.Serializable; - -public class UpdateTaskExecutionState implements Serializable{ - private static final long serialVersionUID = -6662229114427331436L; - - private final TaskExecutionState taskExecutionState; - - public UpdateTaskExecutionState(TaskExecutionState taskExecutionState) { - this.taskExecutionState = taskExecutionState; - } - - public TaskExecutionState getTaskExecutionState() { - return taskExecutionState; - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/resourcemanager/ResourceManagerAkkaActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/resourcemanager/ResourceManagerAkkaActor.java deleted file mode 100644 index 13101f9362ff6..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/resourcemanager/ResourceManagerAkkaActor.java +++ /dev/null @@ -1,65 +0,0 @@ -/* - * 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.rpc.akka.resourcemanager; - -import akka.actor.ActorRef; -import akka.actor.Status; -import akka.pattern.Patterns; -import org.apache.flink.runtime.rpc.akka.BaseAkkaActor; -import org.apache.flink.runtime.rpc.resourcemanager.RegistrationResponse; -import org.apache.flink.runtime.rpc.resourcemanager.ResourceManager; -import org.apache.flink.runtime.rpc.resourcemanager.SlotAssignment; -import org.apache.flink.runtime.rpc.akka.messages.RegisterJobMaster; -import org.apache.flink.runtime.rpc.akka.messages.RequestSlot; -import scala.concurrent.Future; - -public class ResourceManagerAkkaActor extends BaseAkkaActor { - private final ResourceManager resourceManager; - - public ResourceManagerAkkaActor(ResourceManager resourceManager) { - this.resourceManager = resourceManager; - } - - @Override - public void onReceive(Object message) throws Exception { - final ActorRef sender = getSender(); - - if (message instanceof RegisterJobMaster) { - RegisterJobMaster registerJobMaster = (RegisterJobMaster) message; - - try { - Future response = resourceManager.registerJobMaster(registerJobMaster.getJobMasterRegistration()); - Patterns.pipe(response, getContext().dispatcher()).to(sender()); - } catch (Exception e) { - sender.tell(new Status.Failure(e), getSelf()); - } - } else if (message instanceof RequestSlot) { - RequestSlot requestSlot = (RequestSlot) message; - - try { - SlotAssignment response = resourceManager.requestSlot(requestSlot.getSlotRequest()); - sender.tell(new Status.Success(response), getSelf()); - } catch (Exception e) { - sender.tell(new Status.Failure(e), getSelf()); - } - } else { - super.onReceive(message); - } - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/resourcemanager/ResourceManagerAkkaGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/resourcemanager/ResourceManagerAkkaGateway.java deleted file mode 100644 index 13047072ecad7..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/resourcemanager/ResourceManagerAkkaGateway.java +++ /dev/null @@ -1,67 +0,0 @@ -/* - * 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.rpc.akka.resourcemanager; - -import akka.actor.ActorRef; -import akka.pattern.AskableActorRef; -import akka.util.Timeout; -import org.apache.flink.runtime.rpc.akka.BaseAkkaGateway; -import org.apache.flink.runtime.rpc.resourcemanager.JobMasterRegistration; -import org.apache.flink.runtime.rpc.resourcemanager.RegistrationResponse; -import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway; -import org.apache.flink.runtime.rpc.resourcemanager.SlotAssignment; -import org.apache.flink.runtime.rpc.resourcemanager.SlotRequest; -import org.apache.flink.runtime.rpc.akka.messages.RegisterJobMaster; -import org.apache.flink.runtime.rpc.akka.messages.RequestSlot; -import scala.concurrent.Future; -import scala.concurrent.duration.FiniteDuration; -import scala.reflect.ClassTag$; - -public class ResourceManagerAkkaGateway extends BaseAkkaGateway implements ResourceManagerGateway { - private final AskableActorRef actorRef; - private final Timeout timeout; - - public ResourceManagerAkkaGateway(ActorRef actorRef, Timeout timeout) { - this.actorRef = new AskableActorRef(actorRef); - this.timeout = timeout; - } - - @Override - public Future registerJobMaster(JobMasterRegistration jobMasterRegistration, FiniteDuration timeout) { - return actorRef.ask(new RegisterJobMaster(jobMasterRegistration), new Timeout(timeout)) - .mapTo(ClassTag$.MODULE$.apply(RegistrationResponse.class)); - } - - @Override - public Future registerJobMaster(JobMasterRegistration jobMasterRegistration) { - return actorRef.ask(new RegisterJobMaster(jobMasterRegistration), timeout) - .mapTo(ClassTag$.MODULE$.apply(RegistrationResponse.class)); - } - - @Override - public Future requestSlot(SlotRequest slotRequest) { - return actorRef.ask(new RequestSlot(slotRequest), timeout) - .mapTo(ClassTag$.MODULE$.apply(SlotAssignment.class)); - } - - @Override - public ActorRef getActorRef() { - return actorRef.actorRef(); - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/taskexecutor/TaskExecutorAkkaActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/taskexecutor/TaskExecutorAkkaActor.java deleted file mode 100644 index ed522cc727378..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/taskexecutor/TaskExecutorAkkaActor.java +++ /dev/null @@ -1,77 +0,0 @@ -/* - * 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.rpc.akka.taskexecutor; - -import akka.actor.ActorRef; -import akka.actor.Status; -import akka.dispatch.OnComplete; -import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.rpc.akka.BaseAkkaActor; -import org.apache.flink.runtime.rpc.akka.messages.CancelTask; -import org.apache.flink.runtime.rpc.akka.messages.ExecuteTask; -import org.apache.flink.runtime.rpc.taskexecutor.TaskExecutorGateway; - -public class TaskExecutorAkkaActor extends BaseAkkaActor { - private final TaskExecutorGateway taskExecutor; - - public TaskExecutorAkkaActor(TaskExecutorGateway taskExecutor) { - this.taskExecutor = taskExecutor; - } - - @Override - public void onReceive(Object message) throws Exception { - final ActorRef sender = getSender(); - - if (message instanceof ExecuteTask) { - ExecuteTask executeTask = (ExecuteTask) message; - - taskExecutor.executeTask(executeTask.getTaskDeploymentDescriptor()).onComplete( - new OnComplete() { - @Override - public void onComplete(Throwable failure, Acknowledge success) throws Throwable { - if (failure != null) { - sender.tell(new Status.Failure(failure), getSelf()); - } else { - sender.tell(new Status.Success(Acknowledge.get()), getSelf()); - } - } - }, - getContext().dispatcher() - ); - } else if (message instanceof CancelTask) { - CancelTask cancelTask = (CancelTask) message; - - taskExecutor.cancelTask(cancelTask.getExecutionAttemptID()).onComplete( - new OnComplete() { - @Override - public void onComplete(Throwable failure, Acknowledge success) throws Throwable { - if (failure != null) { - sender.tell(new Status.Failure(failure), getSelf()); - } else { - sender.tell(new Status.Success(Acknowledge.get()), getSelf()); - } - } - }, - getContext().dispatcher() - ); - } else { - super.onReceive(message); - } - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/taskexecutor/TaskExecutorAkkaGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/taskexecutor/TaskExecutorAkkaGateway.java deleted file mode 100644 index 7f0a52284c6aa..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/taskexecutor/TaskExecutorAkkaGateway.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * 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.rpc.akka.taskexecutor; - -import akka.actor.ActorRef; -import akka.pattern.AskableActorRef; -import akka.util.Timeout; -import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; -import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; -import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.rpc.akka.BaseAkkaGateway; -import org.apache.flink.runtime.rpc.akka.messages.CancelTask; -import org.apache.flink.runtime.rpc.akka.messages.ExecuteTask; -import org.apache.flink.runtime.rpc.taskexecutor.TaskExecutorGateway; -import scala.concurrent.Future; -import scala.reflect.ClassTag$; - -public class TaskExecutorAkkaGateway extends BaseAkkaGateway implements TaskExecutorGateway { - private final AskableActorRef actorRef; - private final Timeout timeout; - - public TaskExecutorAkkaGateway(ActorRef actorRef, Timeout timeout) { - this.actorRef = new AskableActorRef(actorRef); - this.timeout = timeout; - } - - @Override - public Future executeTask(TaskDeploymentDescriptor taskDeploymentDescriptor) { - return actorRef.ask(new ExecuteTask(taskDeploymentDescriptor), timeout) - .mapTo(ClassTag$.MODULE$.apply(Acknowledge.class)); - } - - @Override - public Future cancelTask(ExecutionAttemptID executionAttemptId) { - return actorRef.ask(new CancelTask(executionAttemptId), timeout) - .mapTo(ClassTag$.MODULE$.apply(Acknowledge.class)); - } - - @Override - public ActorRef getActorRef() { - return actorRef.actorRef(); - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java index b81b19c27321e..e53cd68f93fdb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java @@ -30,6 +30,7 @@ import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.taskmanager.TaskExecutionState; +import org.apache.flink.util.Preconditions; import scala.Tuple2; import scala.concurrent.ExecutionContext; import scala.concurrent.ExecutionContext$; @@ -76,7 +77,8 @@ public class JobMaster extends RpcEndpoint { public JobMaster(RpcService rpcService, ExecutorService executorService) { super(rpcService); - executionContext = ExecutionContext$.MODULE$.fromExecutor(executorService); + executionContext = ExecutionContext$.MODULE$.fromExecutor( + Preconditions.checkNotNull(executorService)); scheduledExecutorService = new ScheduledThreadPoolExecutor(1); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java index c7e8def051398..729ef0c5f41e8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java @@ -25,6 +25,7 @@ import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.jobmaster.JobMaster; import org.apache.flink.runtime.rpc.jobmaster.JobMasterGateway; +import org.apache.flink.util.Preconditions; import scala.concurrent.ExecutionContext; import scala.concurrent.ExecutionContext$; import scala.concurrent.Future; @@ -49,7 +50,8 @@ public class ResourceManager extends RpcEndpoint { public ResourceManager(RpcService rpcService, ExecutorService executorService) { super(rpcService); - this.executionContext = ExecutionContext$.MODULE$.fromExecutor(executorService); + this.executionContext = ExecutionContext$.MODULE$.fromExecutor( + Preconditions.checkNotNull(executorService)); this.jobMasterGateways = new HashMap<>(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java index cdfc3bd63e1e5..3a7dd9f0ce558 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java @@ -25,6 +25,7 @@ import org.apache.flink.runtime.rpc.RpcMethod; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.util.Preconditions; import scala.concurrent.ExecutionContext; import java.util.HashSet; @@ -47,7 +48,8 @@ public class TaskExecutor extends RpcEndpoint { public TaskExecutor(RpcService rpcService, ExecutorService executorService) { super(rpcService); - this.executionContext = ExecutionContexts$.MODULE$.fromExecutor(executorService); + this.executionContext = ExecutionContexts$.MODULE$.fromExecutor( + Preconditions.checkNotNull(executorService)); } /** diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java index 0ded25e4cc7fc..e50533e2a1538 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java @@ -18,15 +18,15 @@ package org.apache.flink.runtime.rpc; +import org.apache.flink.util.ReflectionUtil; import org.apache.flink.util.TestLogger; import org.junit.Test; import org.reflections.Reflections; import scala.concurrent.Future; +import scala.concurrent.duration.FiniteDuration; import java.lang.annotation.Annotation; import java.lang.reflect.Method; -import java.lang.reflect.ParameterizedType; -import java.lang.reflect.Type; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -51,9 +51,8 @@ public void testRpcCompleteness() { for (Class rpcEndpoint :classes){ c = rpcEndpoint; - Type superClass = c.getGenericSuperclass(); - Class rpcGatewayType = extractTypeParameter(superClass, 0); + Class rpcGatewayType = ReflectionUtil.getTemplateType1(c); if (rpcGatewayType != null) { checkCompleteness(rpcEndpoint, (Class) rpcGatewayType); @@ -137,13 +136,16 @@ private void checkGatewayMethod(Method gatewayMethod) { } Annotation[][] parameterAnnotations = gatewayMethod.getParameterAnnotations(); + Class[] parameterTypes = gatewayMethod.getParameterTypes(); int rpcTimeoutParameters = 0; - for (Annotation[] parameterAnnotation : parameterAnnotations) { - for (Annotation annotation : parameterAnnotation) { - if (annotation.equals(RpcTimeout.class)) { - rpcTimeoutParameters++; - } + for (int i = 0; i < parameterAnnotations.length; i++) { + if (isRpcTimeout(parameterAnnotations[i])) { + assertTrue( + "The rpc timeout has to be of type " + FiniteDuration.class.getName() + ".", + parameterTypes[i].equals(FiniteDuration.class)); + + rpcTimeoutParameters++; } } @@ -211,10 +213,10 @@ private boolean checkMethod(Method gatewayMethod, Method endpointMethod) { if (!futureClass.equals(RpcCompletenessTest.futureClass)) { return false; } else { - Class valueClass = extractTypeParameter(futureClass, 0); + Class valueClass = ReflectionUtil.getTemplateType1(gatewayMethod.getGenericReturnType()); if (endpointMethod.getReturnType().equals(futureClass)) { - Class rpcEndpointValueClass = extractTypeParameter(endpointMethod.getReturnType(), 0); + Class rpcEndpointValueClass = ReflectionUtil.getTemplateType1(endpointMethod.getGenericReturnType()); // check if we have the same future value types if (valueClass != null && rpcEndpointValueClass != null && !checkType(valueClass, rpcEndpointValueClass)) { @@ -251,7 +253,7 @@ private String generateEndpointMethodSignature(Method method) { if (method.getReturnType().equals(Void.TYPE)) { builder.append("void").append(" "); } else if (method.getReturnType().equals(futureClass)) { - Class valueClass = extractTypeParameter(method.getGenericReturnType(), 0); + Class valueClass = ReflectionUtil.getTemplateType1(method.getGenericReturnType()); builder .append(futureClass.getSimpleName()) @@ -291,30 +293,6 @@ private String generateEndpointMethodSignature(Method method) { return builder.toString(); } - private Class extractTypeParameter(Type genericType, int position) { - if (genericType instanceof ParameterizedType) { - ParameterizedType parameterizedType = (ParameterizedType) genericType; - - Type[] typeArguments = parameterizedType.getActualTypeArguments(); - - if (position < 0 || position >= typeArguments.length) { - throw new IndexOutOfBoundsException("The generic type " + - parameterizedType.getRawType() + " only has " + typeArguments.length + - " type arguments."); - } else { - Type typeArgument = typeArguments[position]; - - if (typeArgument instanceof Class) { - return (Class) typeArgument; - } else { - return null; - } - } - } else { - return null; - } - } - private boolean isRpcTimeout(Annotation[] annotations) { for (Annotation annotation : annotations) { if (annotation.annotationType().equals(RpcTimeout.class)) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java index 642a380593906..a4e1d7f3d3606 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java @@ -61,10 +61,10 @@ public void testJobMasterResourceManagerRegistration() throws Exception { AkkaGateway akkaClient = (AkkaGateway) rm; - jobMaster.registerAtResourceManager(AkkaUtils.getAkkaURL(actorSystem, akkaClient.getActorRef())); + jobMaster.registerAtResourceManager(AkkaUtils.getAkkaURL(actorSystem, akkaClient.getRpcServer())); // wait for successful registration - FiniteDuration timeout = new FiniteDuration(20, TimeUnit.SECONDS); + FiniteDuration timeout = new FiniteDuration(200, TimeUnit.SECONDS); Deadline deadline = timeout.fromNow(); while (deadline.hasTimeLeft() && !jobMaster.isConnected()) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java index c1435278ad735..33c9cb61ca4e6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java @@ -48,7 +48,7 @@ public class TaskExecutorTest extends TestLogger { @Test public void testTaskExecution() throws Exception { RpcService testingRpcService = mock(RpcService.class); - DirectExecutorService directExecutorService = null; + DirectExecutorService directExecutorService = new DirectExecutorService(); TaskExecutor taskExecutor = new TaskExecutor(testingRpcService, directExecutorService); TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor( From d31e631414ac2e5d90e519c5e60b517f382c357b Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 11 Aug 2016 19:10:48 +0200 Subject: [PATCH 04/50] [FLINK-4384] [rpc] Add "scheduleRunAsync()" to the RpcEndpoint This closes #2360 --- .../flink/runtime/rpc/MainThreadExecutor.java | 9 + .../apache/flink/runtime/rpc/RpcEndpoint.java | 12 + .../rpc/akka/AkkaInvocationHandler.java | 13 +- .../flink/runtime/rpc/akka/AkkaRpcActor.java | 15 +- .../runtime/rpc/akka/messages/RunAsync.java | 24 +- .../runtime/rpc/akka/AkkaRpcServiceTest.java | 3 + .../runtime/rpc/akka/AsyncCallsTest.java | 216 ++++++++++++++++++ 7 files changed, 286 insertions(+), 6 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AsyncCallsTest.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java index 882c1b751e5e3..4efb382639c28 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java @@ -52,4 +52,13 @@ public interface MainThreadExecutor { * @return Future of the callable result */ Future callAsync(Callable callable, Timeout callTimeout); + + /** + * Execute the runnable in the main thread of the underlying RPC endpoint, with + * a delay of the given number of milliseconds. + * + * @param runnable Runnable to be executed + * @param delay The delay, in milliseconds, after which the runnable will be executed + */ + void scheduleRunAsync(Runnable runnable, long delay); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java index aef08031093ca..44933d5837b64 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java @@ -28,6 +28,7 @@ import scala.concurrent.Future; import java.util.concurrent.Callable; +import java.util.concurrent.TimeUnit; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -167,6 +168,17 @@ public void runAsync(Runnable runnable) { ((MainThreadExecutor) self).runAsync(runnable); } + /** + * Execute the runnable in the main thread of the underlying RPC endpoint, with + * a delay of the given number of milliseconds. + * + * @param runnable Runnable to be executed + * @param delay The delay after which the runnable will be executed + */ + public void scheduleRunAsync(Runnable runnable, long delay, TimeUnit unit) { + ((MainThreadExecutor) self).scheduleRunAsync(runnable, unit.toMillis(delay)); + } + /** * Execute the callable in the main thread of the underlying RPC service, returning a future for * the result of the callable. If the callable is not completed within the given timeout, then diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java index e8e383a171da6..580b161bd041b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java @@ -38,6 +38,9 @@ import java.util.BitSet; import java.util.concurrent.Callable; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkArgument; + /** * Invocation handler to be used with a {@link AkkaRpcActor}. The invocation handler wraps the * rpc in a {@link RpcInvocation} message and then sends it to the {@link AkkaRpcActor} where it is @@ -106,9 +109,17 @@ public ActorRef getRpcServer() { @Override public void runAsync(Runnable runnable) { + scheduleRunAsync(runnable, 0); + } + + @Override + public void scheduleRunAsync(Runnable runnable, long delay) { + checkNotNull(runnable, "runnable"); + checkArgument(delay >= 0, "delay must be zero or greater"); + // Unfortunately I couldn't find a way to allow only local communication. Therefore, the // runnable field is transient transient - rpcServer.tell(new RunAsync(runnable), ActorRef.noSender()); + rpcServer.tell(new RunAsync(runnable, delay), ActorRef.noSender()); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java index 57da38a6f7bde..18ccf1b51e47c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.rpc.akka; +import akka.actor.ActorRef; import akka.actor.Status; import akka.actor.UntypedActor; import akka.pattern.Patterns; @@ -30,9 +31,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import scala.concurrent.Future; +import scala.concurrent.duration.FiniteDuration; import java.lang.reflect.Method; import java.util.concurrent.Callable; +import java.util.concurrent.TimeUnit; /** * Akka rpc actor which receives {@link RpcInvocation}, {@link RunAsync} and {@link CallAsync} @@ -152,13 +155,23 @@ private void handleRunAsync(RunAsync runAsync) { "{} is only supported with local communication.", runAsync.getClass().getName(), runAsync.getClass().getName()); - } else { + } + else if (runAsync.getDelay() == 0) { + // run immediately try { runAsync.getRunnable().run(); } catch (final Throwable e) { LOG.error("Caught exception while executing runnable in main thread.", e); } } + else { + // schedule for later. send a new message after the delay, which will then be immediately executed + FiniteDuration delay = new FiniteDuration(runAsync.getDelay(), TimeUnit.MILLISECONDS); + RunAsync message = new RunAsync(runAsync.getRunnable(), 0); + + getContext().system().scheduler().scheduleOnce(delay, getSelf(), message, + getContext().dispatcher(), ActorRef.noSender()); + } } /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunAsync.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunAsync.java index fb958525f5079..c18906ca21637 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunAsync.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunAsync.java @@ -18,23 +18,39 @@ package org.apache.flink.runtime.rpc.akka.messages; -import org.apache.flink.util.Preconditions; - import java.io.Serializable; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkArgument; + /** * Message for asynchronous runnable invocations */ public final class RunAsync implements Serializable { private static final long serialVersionUID = -3080595100695371036L; + /** The runnable to be executed. Transient, so it gets lost upon serialization */ private final transient Runnable runnable; - public RunAsync(Runnable runnable) { - this.runnable = Preconditions.checkNotNull(runnable); + /** The delay after which the runnable should be called */ + private final long delay; + + /** + * + * @param runnable The Runnable to run. + * @param delay The delay in milliseconds. Zero indicates immediate execution. + */ + public RunAsync(Runnable runnable, long delay) { + checkArgument(delay >= 0); + this.runnable = checkNotNull(runnable); + this.delay = delay; } public Runnable getRunnable() { return runnable; } + + public long getDelay() { + return delay; + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java index a4e1d7f3d3606..5e37e10ff0dc5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java @@ -21,6 +21,9 @@ import akka.actor.ActorSystem; import akka.util.Timeout; import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.rpc.RpcEndpoint; +import org.apache.flink.runtime.rpc.RpcGateway; +import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.jobmaster.JobMaster; import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.rpc.resourcemanager.ResourceManager; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AsyncCallsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AsyncCallsTest.java new file mode 100644 index 0000000000000..f2ce52d99fbdb --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AsyncCallsTest.java @@ -0,0 +1,216 @@ +/* + * 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.rpc.akka; + +import akka.actor.ActorSystem; +import akka.util.Timeout; + +import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.rpc.RpcEndpoint; +import org.apache.flink.runtime.rpc.RpcGateway; +import org.apache.flink.runtime.rpc.RpcMethod; +import org.apache.flink.runtime.rpc.RpcService; + +import org.junit.AfterClass; +import org.junit.Test; + +import scala.concurrent.Await; +import scala.concurrent.Future; +import scala.concurrent.duration.FiniteDuration; + +import java.util.concurrent.Callable; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.ReentrantLock; + +import static org.junit.Assert.*; + +public class AsyncCallsTest { + + // ------------------------------------------------------------------------ + // shared test members + // ------------------------------------------------------------------------ + + private static ActorSystem actorSystem = AkkaUtils.createDefaultActorSystem(); + + private static AkkaRpcService akkaRpcService = + new AkkaRpcService(actorSystem, new Timeout(10000, TimeUnit.MILLISECONDS)); + + @AfterClass + public static void shutdown() { + akkaRpcService.stopService(); + actorSystem.shutdown(); + } + + + // ------------------------------------------------------------------------ + // tests + // ------------------------------------------------------------------------ + + @Test + public void testScheduleWithNoDelay() throws Exception { + + // to collect all the thread references + final ReentrantLock lock = new ReentrantLock(); + final AtomicBoolean concurrentAccess = new AtomicBoolean(false); + + TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService, lock); + TestGateway gateway = testEndpoint.getSelf(); + + // a bunch of gateway calls + gateway.someCall(); + gateway.anotherCall(); + gateway.someCall(); + + // run something asynchronously + for (int i = 0; i < 10000; i++) { + testEndpoint.runAsync(new Runnable() { + @Override + public void run() { + boolean holdsLock = lock.tryLock(); + if (holdsLock) { + lock.unlock(); + } else { + concurrentAccess.set(true); + } + } + }); + } + + Future result = testEndpoint.callAsync(new Callable() { + @Override + public String call() throws Exception { + boolean holdsLock = lock.tryLock(); + if (holdsLock) { + lock.unlock(); + } else { + concurrentAccess.set(true); + } + return "test"; + } + }, new Timeout(30, TimeUnit.SECONDS)); + String str = Await.result(result, new FiniteDuration(30, TimeUnit.SECONDS)); + assertEquals("test", str); + + // validate that no concurrent access happened + assertFalse("Rpc Endpoint had concurrent access", testEndpoint.hasConcurrentAccess()); + assertFalse("Rpc Endpoint had concurrent access", concurrentAccess.get()); + + akkaRpcService.stopServer(testEndpoint.getSelf()); + } + + @Test + public void testScheduleWithDelay() throws Exception { + + // to collect all the thread references + final ReentrantLock lock = new ReentrantLock(); + final AtomicBoolean concurrentAccess = new AtomicBoolean(false); + final OneShotLatch latch = new OneShotLatch(); + + final long delay = 200; + + TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService, lock); + + // run something asynchronously + testEndpoint.runAsync(new Runnable() { + @Override + public void run() { + boolean holdsLock = lock.tryLock(); + if (holdsLock) { + lock.unlock(); + } else { + concurrentAccess.set(true); + } + } + }); + + final long start = System.nanoTime(); + + testEndpoint.scheduleRunAsync(new Runnable() { + @Override + public void run() { + boolean holdsLock = lock.tryLock(); + if (holdsLock) { + lock.unlock(); + } else { + concurrentAccess.set(true); + } + latch.trigger(); + } + }, delay, TimeUnit.MILLISECONDS); + + latch.await(); + final long stop = System.nanoTime(); + + // validate that no concurrent access happened + assertFalse("Rpc Endpoint had concurrent access", testEndpoint.hasConcurrentAccess()); + assertFalse("Rpc Endpoint had concurrent access", concurrentAccess.get()); + + assertTrue("call was not properly delayed", ((stop - start) / 1000000) >= delay); + } + + // ------------------------------------------------------------------------ + // test RPC endpoint + // ------------------------------------------------------------------------ + + interface TestGateway extends RpcGateway { + + void someCall(); + + void anotherCall(); + } + + @SuppressWarnings("unused") + public static class TestEndpoint extends RpcEndpoint { + + private final ReentrantLock lock; + + private volatile boolean concurrentAccess; + + public TestEndpoint(RpcService rpcService, ReentrantLock lock) { + super(rpcService); + this.lock = lock; + } + + @RpcMethod + public void someCall() { + boolean holdsLock = lock.tryLock(); + if (holdsLock) { + lock.unlock(); + } else { + concurrentAccess = true; + } + } + + @RpcMethod + public void anotherCall() { + boolean holdsLock = lock.tryLock(); + if (holdsLock) { + lock.unlock(); + } else { + concurrentAccess = true; + } + } + + public boolean hasConcurrentAccess() { + return concurrentAccess; + } + } +} From d881403bf87714dc5349471c830dbfbadc98b9d5 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Sat, 13 Aug 2016 19:11:47 +0200 Subject: [PATCH 05/50] [FLINK-4392] [rpc] Make RPC Service thread-safe --- .../flink/runtime/rpc/akka/AkkaGateway.java | 3 +- .../runtime/rpc/akka/AkkaRpcService.java | 92 ++++++++++++++----- 2 files changed, 70 insertions(+), 25 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaGateway.java index a826e7dcf469e..ec3091c839c05 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaGateway.java @@ -19,11 +19,12 @@ package org.apache.flink.runtime.rpc.akka; import akka.actor.ActorRef; +import org.apache.flink.runtime.rpc.RpcGateway; /** * Interface for Akka based rpc gateways */ -interface AkkaGateway { +interface AkkaGateway extends RpcGateway { ActorRef getRpcServer(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java index 17983d01b67c4..448216c9d02dd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java @@ -28,47 +28,61 @@ import akka.dispatch.Mapper; import akka.pattern.AskableActorSelection; import akka.util.Timeout; + import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.rpc.MainThreadExecutor; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcService; -import org.apache.flink.util.Preconditions; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import scala.concurrent.Future; +import javax.annotation.concurrent.ThreadSafe; import java.lang.reflect.InvocationHandler; import java.lang.reflect.Proxy; -import java.util.Collection; import java.util.HashSet; +import java.util.Set; + +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; /** - * Akka based {@link RpcService} implementation. The rpc service starts an Akka actor to receive - * rpcs from a {@link RpcGateway}. + * Akka based {@link RpcService} implementation. The RPC service starts an Akka actor to receive + * RPC invocations from a {@link RpcGateway}. */ +@ThreadSafe public class AkkaRpcService implements RpcService { + private static final Logger LOG = LoggerFactory.getLogger(AkkaRpcService.class); + private final Object lock = new Object(); + private final ActorSystem actorSystem; private final Timeout timeout; - private final Collection actors = new HashSet<>(4); + private final Set actors = new HashSet<>(4); + + private volatile boolean stopped; public AkkaRpcService(final ActorSystem actorSystem, final Timeout timeout) { - this.actorSystem = Preconditions.checkNotNull(actorSystem, "actor system"); - this.timeout = Preconditions.checkNotNull(timeout, "timeout"); + this.actorSystem = checkNotNull(actorSystem, "actor system"); + this.timeout = checkNotNull(timeout, "timeout"); } + // this method does not mutate state and is thus thread-safe @Override public Future connect(final String address, final Class clazz) { - LOG.info("Try to connect to remote rpc server with address {}. Returning a {} gateway.", address, clazz.getName()); + checkState(!stopped, "RpcService is stopped"); - final ActorSelection actorSel = actorSystem.actorSelection(address); + LOG.debug("Try to connect to remote RPC endpoint with address {}. Returning a {} gateway.", + address, clazz.getName()); + final ActorSelection actorSel = actorSystem.actorSelection(address); final AskableActorSelection asker = new AskableActorSelection(actorSel); final Future identify = asker.ask(new Identify(42), timeout); - return identify.map(new Mapper(){ @Override public C apply(Object obj) { @@ -89,20 +103,29 @@ public C apply(Object obj) { @Override public > C startServer(S rpcEndpoint) { - Preconditions.checkNotNull(rpcEndpoint, "rpc endpoint"); - - LOG.info("Start Akka rpc actor to handle rpcs for {}.", rpcEndpoint.getClass().getName()); + checkNotNull(rpcEndpoint, "rpc endpoint"); Props akkaRpcActorProps = Props.create(AkkaRpcActor.class, rpcEndpoint); + ActorRef actorRef; + + synchronized (lock) { + checkState(!stopped, "RpcService is stopped"); + actorRef = actorSystem.actorOf(akkaRpcActorProps); + actors.add(actorRef); + } - ActorRef actorRef = actorSystem.actorOf(akkaRpcActorProps); - actors.add(actorRef); + LOG.info("Starting RPC endpoint for {} at {} .", rpcEndpoint.getClass().getName(), actorRef.path()); InvocationHandler akkaInvocationHandler = new AkkaInvocationHandler(actorRef, timeout); + // Rather than using the System ClassLoader directly, we derive the ClassLoader + // from this class . That works better in cases where Flink runs embedded and all Flink + // code is loaded dynamically (for example from an OSGI bundle) through a custom ClassLoader + ClassLoader classLoader = getClass().getClassLoader(); + @SuppressWarnings("unchecked") C self = (C) Proxy.newProxyInstance( - ClassLoader.getSystemClassLoader(), + classLoader, new Class[]{rpcEndpoint.getSelfGatewayType(), MainThreadExecutor.class, AkkaGateway.class}, akkaInvocationHandler); @@ -110,35 +133,56 @@ public > C startServer(S rpcEndpo } @Override - public void stopServer(C selfGateway) { + public void stopServer(RpcGateway selfGateway) { if (selfGateway instanceof AkkaGateway) { AkkaGateway akkaClient = (AkkaGateway) selfGateway; - if (actors.contains(akkaClient.getRpcServer())) { - ActorRef selfActorRef = akkaClient.getRpcServer(); - - LOG.info("Stop Akka rpc actor {}.", selfActorRef.path()); + boolean fromThisService; + synchronized (lock) { + if (stopped) { + return; + } else { + fromThisService = actors.remove(akkaClient.getRpcServer()); + } + } + if (fromThisService) { + ActorRef selfActorRef = akkaClient.getRpcServer(); + LOG.info("Stopping RPC endpoint {}.", selfActorRef.path()); selfActorRef.tell(PoisonPill.getInstance(), ActorRef.noSender()); + } else { + LOG.debug("RPC endpoint {} already stopped or from different RPC service"); } } } @Override public void stopService() { - LOG.info("Stop Akka rpc service."); - actorSystem.shutdown(); + LOG.info("Stopping Akka RPC service."); + + synchronized (lock) { + if (stopped) { + return; + } + + stopped = true; + actorSystem.shutdown(); + actors.clear(); + } + actorSystem.awaitTermination(); } @Override public String getAddress(C selfGateway) { + checkState(!stopped, "RpcService is stopped"); + if (selfGateway instanceof AkkaGateway) { ActorRef actorRef = ((AkkaGateway) selfGateway).getRpcServer(); return AkkaUtils.getAkkaURL(actorSystem, actorRef); } else { String className = AkkaGateway.class.getName(); - throw new RuntimeException("Cannot get address for non " + className + '.'); + throw new IllegalArgumentException("Cannot get address for non " + className + '.'); } } } From f0d5b7b8f45208ebfdb659f83cbe00fb4c91fee0 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 11 Aug 2016 20:30:54 +0200 Subject: [PATCH 06/50] [FLINK-4386] [rpc] Add a utility to verify calls happen in the Rpc Endpoint's main thread --- .../flink/runtime/rpc/MainThreadExecutor.java | 2 +- .../runtime/rpc/MainThreadValidatorUtil.java | 47 +++++++++ .../apache/flink/runtime/rpc/RpcEndpoint.java | 38 +++++++- .../flink/runtime/rpc/akka/AkkaRpcActor.java | 37 ++++--- .../runtime/rpc/akka/AkkaRpcService.java | 2 +- .../rpc/akka/MainThreadValidationTest.java | 97 +++++++++++++++++++ 6 files changed, 205 insertions(+), 18 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadValidatorUtil.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java index 4efb382639c28..5e4fead8c5531 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java @@ -30,7 +30,7 @@ * *

This interface is intended to be implemented by the self gateway in a {@link RpcEndpoint} * implementation which allows to dispatch local procedures to the main thread of the underlying - * rpc server. + * RPC endpoint. */ public interface MainThreadExecutor { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadValidatorUtil.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadValidatorUtil.java new file mode 100644 index 0000000000000..b3fea7703b4d8 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadValidatorUtil.java @@ -0,0 +1,47 @@ +/* + * 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.rpc; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * This utility exists to bridge between the visibility of the + * {@code currentMainThread} field in the {@link RpcEndpoint}. + * + * The {@code currentMainThread} can be hidden from {@code RpcEndpoint} implementations + * and only be accessed via this utility from other packages. + */ +public final class MainThreadValidatorUtil { + + private final RpcEndpoint endpoint; + + public MainThreadValidatorUtil(RpcEndpoint endpoint) { + this.endpoint = checkNotNull(endpoint); + } + + public void enterMainThread() { + assert(endpoint.currentMainThread.compareAndSet(null, Thread.currentThread())) : + "The RpcEndpoint has concurrent access from " + endpoint.currentMainThread.get(); + } + + public void exitMainThread() { + assert(endpoint.currentMainThread.compareAndSet(Thread.currentThread(), null)) : + "The RpcEndpoint has concurrent access from " + endpoint.currentMainThread.get(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java index 44933d5837b64..d36a283f26557 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java @@ -29,6 +29,7 @@ import java.util.concurrent.Callable; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -75,6 +76,9 @@ public abstract class RpcEndpoint { * of the executing rpc server. */ private final MainThreadExecutionContext mainThreadExecutionContext; + /** A reference to the endpoint's main thread, if the current method is called by the main thread */ + final AtomicReference currentMainThread = new AtomicReference<>(null); + /** * Initializes the RPC endpoint. * @@ -92,6 +96,15 @@ protected RpcEndpoint(final RpcService rpcService) { this.mainThreadExecutionContext = new MainThreadExecutionContext((MainThreadExecutor) self); } + /** + * Returns the class of the self gateway type. + * + * @return Class of the self gateway type + */ + public final Class getSelfGatewayType() { + return selfGatewayType; + } + // ------------------------------------------------------------------------ // Shutdown // ------------------------------------------------------------------------ @@ -193,13 +206,28 @@ public Future callAsync(Callable callable, Timeout timeout) { return ((MainThreadExecutor) self).callAsync(callable, timeout); } + // ------------------------------------------------------------------------ + // Main Thread Validation + // ------------------------------------------------------------------------ + /** - * Returns the class of the self gateway type. - * - * @return Class of the self gateway type + * Validates that the method call happens in the RPC endpoint's main thread. + * + *

IMPORTANT: This check only happens when assertions are enabled, + * such as when running tests. + * + *

This can be used for additional checks, like + *

{@code
+	 * protected void concurrencyCriticalMethod() {
+	 *     validateRunsInMainThread();
+	 *     
+	 *     // some critical stuff
+	 * }
+	 * }
*/ - public final Class getSelfGatewayType() { - return selfGatewayType; + public void validateRunsInMainThread() { + // because the initialization is lazy, it can be that certain methods are + assert currentMainThread.get() == Thread.currentThread(); } // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java index 18ccf1b51e47c..5e0a7da000931 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java @@ -22,14 +22,16 @@ import akka.actor.Status; import akka.actor.UntypedActor; import akka.pattern.Patterns; +import org.apache.flink.runtime.rpc.MainThreadValidatorUtil; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.akka.messages.CallAsync; import org.apache.flink.runtime.rpc.akka.messages.RpcInvocation; import org.apache.flink.runtime.rpc.akka.messages.RunAsync; -import org.apache.flink.util.Preconditions; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import scala.concurrent.Future; import scala.concurrent.duration.FiniteDuration; @@ -37,6 +39,8 @@ import java.util.concurrent.Callable; import java.util.concurrent.TimeUnit; +import static org.apache.flink.util.Preconditions.checkNotNull; + /** * Akka rpc actor which receives {@link RpcInvocation}, {@link RunAsync} and {@link CallAsync} * messages. @@ -51,24 +55,35 @@ * @param Type of the {@link RpcEndpoint} */ class AkkaRpcActor> extends UntypedActor { + private static final Logger LOG = LoggerFactory.getLogger(AkkaRpcActor.class); + /** the endpoint to invoke the methods on */ private final T rpcEndpoint; + /** the helper that tracks whether calls come from the main thread */ + private final MainThreadValidatorUtil mainThreadValidator; + AkkaRpcActor(final T rpcEndpoint) { - this.rpcEndpoint = Preconditions.checkNotNull(rpcEndpoint, "rpc endpoint"); + this.rpcEndpoint = checkNotNull(rpcEndpoint, "rpc endpoint"); + this.mainThreadValidator = new MainThreadValidatorUtil(rpcEndpoint); } @Override - public void onReceive(final Object message) { - if (message instanceof RunAsync) { - handleRunAsync((RunAsync) message); - } else if (message instanceof CallAsync) { - handleCallAsync((CallAsync) message); - } else if (message instanceof RpcInvocation) { - handleRpcInvocation((RpcInvocation) message); - } else { - LOG.warn("Received message of unknown type {}. Dropping this message!", message.getClass()); + public void onReceive(final Object message) { + mainThreadValidator.enterMainThread(); + try { + if (message instanceof RunAsync) { + handleRunAsync((RunAsync) message); + } else if (message instanceof CallAsync) { + handleCallAsync((CallAsync) message); + } else if (message instanceof RpcInvocation) { + handleRpcInvocation((RpcInvocation) message); + } else { + LOG.warn("Received message of unknown type {}. Dropping this message!", message.getClass()); + } + } finally { + mainThreadValidator.exitMainThread(); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java index 448216c9d02dd..db40f10e10f39 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java @@ -174,7 +174,7 @@ public void stopService() { } @Override - public String getAddress(C selfGateway) { + public String getAddress(RpcGateway selfGateway) { checkState(!stopped, "RpcService is stopped"); if (selfGateway instanceof AkkaGateway) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java new file mode 100644 index 0000000000000..b85414320e790 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java @@ -0,0 +1,97 @@ +/* + * 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.rpc.akka; + +import akka.util.Timeout; + +import org.apache.flink.runtime.akka.AkkaUtils; + +import org.apache.flink.runtime.rpc.RpcEndpoint; +import org.apache.flink.runtime.rpc.RpcGateway; +import org.apache.flink.runtime.rpc.RpcMethod; +import org.apache.flink.runtime.rpc.RpcService; + +import org.junit.Test; + +import java.util.concurrent.TimeUnit; + +import static org.junit.Assert.assertTrue; + +public class MainThreadValidationTest { + + @Test + public void failIfNotInMainThread() { + // test if assertions are activated. The test only works if assertions are loaded. + try { + assert false; + // apparently they are not activated + return; + } catch (AssertionError ignored) {} + + // actual test + AkkaRpcService akkaRpcService = new AkkaRpcService( + AkkaUtils.createDefaultActorSystem(), + new Timeout(10000, TimeUnit.MILLISECONDS)); + + try { + TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService); + + // this works, because it is executed as an RPC call + testEndpoint.getSelf().someConcurrencyCriticalFunction(); + + // this fails, because it is executed directly + boolean exceptionThrown; + try { + testEndpoint.someConcurrencyCriticalFunction(); + exceptionThrown = false; + } + catch (AssertionError e) { + exceptionThrown = true; + } + assertTrue("should fail with an assertion error", exceptionThrown); + + akkaRpcService.stopServer(testEndpoint.getSelf()); + } + finally { + akkaRpcService.stopService(); + } + } + + // ------------------------------------------------------------------------ + // test RPC endpoint + // ------------------------------------------------------------------------ + + interface TestGateway extends RpcGateway { + + void someConcurrencyCriticalFunction(); + } + + @SuppressWarnings("unused") + public static class TestEndpoint extends RpcEndpoint { + + public TestEndpoint(RpcService rpcService) { + super(rpcService); + } + + @RpcMethod + public void someConcurrencyCriticalFunction() { + validateRunsInMainThread(); + } + } +} From 89ea0cfe27e3ea4737d7bdc10f482e1a170c9f43 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 12 Aug 2016 10:32:30 +0200 Subject: [PATCH 07/50] [FLINK-4383] [rpc] Eagerly serialize remote rpc invocation messages This PR introduces an eager serialization for remote rpc invocation messages. That way it is possible to check whether the message is serializable and whether it exceeds the maximum allowed akka frame size. If either of these constraints is violated, a proper exception is thrown instead of simply swallowing the exception as Akka does it. Address PR comments This closes #2365. --- .../flink/runtime/rpc/akka/AkkaGateway.java | 2 +- .../rpc/akka/AkkaInvocationHandler.java | 83 +++++-- .../flink/runtime/rpc/akka/AkkaRpcActor.java | 26 ++- .../runtime/rpc/akka/AkkaRpcService.java | 20 +- .../rpc/akka/messages/LocalRpcInvocation.java | 54 +++++ .../akka/messages/RemoteRpcInvocation.java | 206 +++++++++++++++++ .../rpc/akka/messages/RpcInvocation.java | 106 +++------ .../runtime/rpc/akka/AkkaRpcServiceTest.java | 2 +- .../rpc/akka/MessageSerializationTest.java | 210 ++++++++++++++++++ 9 files changed, 597 insertions(+), 112 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/LocalRpcInvocation.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RemoteRpcInvocation.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaGateway.java index ec3091c839c05..f6125dc0a99da 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaGateway.java @@ -26,5 +26,5 @@ */ interface AkkaGateway extends RpcGateway { - ActorRef getRpcServer(); + ActorRef getRpcEndpoint(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java index 580b161bd041b..297104b4beac3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java @@ -25,13 +25,17 @@ import org.apache.flink.runtime.rpc.MainThreadExecutor; import org.apache.flink.runtime.rpc.RpcTimeout; import org.apache.flink.runtime.rpc.akka.messages.CallAsync; +import org.apache.flink.runtime.rpc.akka.messages.LocalRpcInvocation; +import org.apache.flink.runtime.rpc.akka.messages.RemoteRpcInvocation; import org.apache.flink.runtime.rpc.akka.messages.RpcInvocation; import org.apache.flink.runtime.rpc.akka.messages.RunAsync; import org.apache.flink.util.Preconditions; +import org.apache.log4j.Logger; import scala.concurrent.Await; import scala.concurrent.Future; import scala.concurrent.duration.FiniteDuration; +import java.io.IOException; import java.lang.annotation.Annotation; import java.lang.reflect.InvocationHandler; import java.lang.reflect.Method; @@ -42,19 +46,28 @@ import static org.apache.flink.util.Preconditions.checkArgument; /** - * Invocation handler to be used with a {@link AkkaRpcActor}. The invocation handler wraps the - * rpc in a {@link RpcInvocation} message and then sends it to the {@link AkkaRpcActor} where it is + * Invocation handler to be used with an {@link AkkaRpcActor}. The invocation handler wraps the + * rpc in a {@link LocalRpcInvocation} message and then sends it to the {@link AkkaRpcActor} where it is * executed. */ class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThreadExecutor { - private final ActorRef rpcServer; + private static final Logger LOG = Logger.getLogger(AkkaInvocationHandler.class); + + private final ActorRef rpcEndpoint; + + // whether the actor ref is local and thus no message serialization is needed + private final boolean isLocal; // default timeout for asks private final Timeout timeout; - AkkaInvocationHandler(ActorRef rpcServer, Timeout timeout) { - this.rpcServer = Preconditions.checkNotNull(rpcServer); + private final long maximumFramesize; + + AkkaInvocationHandler(ActorRef rpcEndpoint, Timeout timeout, long maximumFramesize) { + this.rpcEndpoint = Preconditions.checkNotNull(rpcEndpoint); + this.isLocal = this.rpcEndpoint.path().address().hasLocalScope(); this.timeout = Preconditions.checkNotNull(timeout); + this.maximumFramesize = maximumFramesize; } @Override @@ -76,23 +89,43 @@ public Object invoke(Object proxy, Method method, Object[] args) throws Throwabl parameterAnnotations, args); - RpcInvocation rpcInvocation = new RpcInvocation( - methodName, - filteredArguments.f0, - filteredArguments.f1); + RpcInvocation rpcInvocation; + + if (isLocal) { + rpcInvocation = new LocalRpcInvocation( + methodName, + filteredArguments.f0, + filteredArguments.f1); + } else { + try { + RemoteRpcInvocation remoteRpcInvocation = new RemoteRpcInvocation( + methodName, + filteredArguments.f0, + filteredArguments.f1); + + if (remoteRpcInvocation.getSize() > maximumFramesize) { + throw new IOException("The rpc invocation size exceeds the maximum akka framesize."); + } else { + rpcInvocation = remoteRpcInvocation; + } + } catch (IOException e) { + LOG.warn("Could not create remote rpc invocation message. Failing rpc invocation because...", e); + throw e; + } + } Class returnType = method.getReturnType(); if (returnType.equals(Void.TYPE)) { - rpcServer.tell(rpcInvocation, ActorRef.noSender()); + rpcEndpoint.tell(rpcInvocation, ActorRef.noSender()); result = null; } else if (returnType.equals(Future.class)) { // execute an asynchronous call - result = Patterns.ask(rpcServer, rpcInvocation, futureTimeout); + result = Patterns.ask(rpcEndpoint, rpcInvocation, futureTimeout); } else { // execute a synchronous call - Future futureResult = Patterns.ask(rpcServer, rpcInvocation, futureTimeout); + Future futureResult = Patterns.ask(rpcEndpoint, rpcInvocation, futureTimeout); FiniteDuration duration = timeout.duration(); result = Await.result(futureResult, duration); @@ -103,8 +136,8 @@ public Object invoke(Object proxy, Method method, Object[] args) throws Throwabl } @Override - public ActorRef getRpcServer() { - return rpcServer; + public ActorRef getRpcEndpoint() { + return rpcEndpoint; } @Override @@ -117,19 +150,25 @@ public void scheduleRunAsync(Runnable runnable, long delay) { checkNotNull(runnable, "runnable"); checkArgument(delay >= 0, "delay must be zero or greater"); - // Unfortunately I couldn't find a way to allow only local communication. Therefore, the - // runnable field is transient transient - rpcServer.tell(new RunAsync(runnable, delay), ActorRef.noSender()); + if (isLocal) { + rpcEndpoint.tell(new RunAsync(runnable, delay), ActorRef.noSender()); + } else { + throw new RuntimeException("Trying to send a Runnable to a remote actor at " + + rpcEndpoint.path() + ". This is not supported."); + } } @Override public Future callAsync(Callable callable, Timeout callTimeout) { - // Unfortunately I couldn't find a way to allow only local communication. Therefore, the - // callable field is declared transient - @SuppressWarnings("unchecked") - Future result = (Future) Patterns.ask(rpcServer, new CallAsync(callable), callTimeout); + if(isLocal) { + @SuppressWarnings("unchecked") + Future result = (Future) Patterns.ask(rpcEndpoint, new CallAsync(callable), callTimeout); - return result; + return result; + } else { + throw new RuntimeException("Trying to send a Callable to a remote actor at " + + rpcEndpoint.path() + ". This is not supported."); + } } /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java index 5e0a7da000931..dfcbcc3924374 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java @@ -26,6 +26,7 @@ import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.akka.messages.CallAsync; +import org.apache.flink.runtime.rpc.akka.messages.LocalRpcInvocation; import org.apache.flink.runtime.rpc.akka.messages.RpcInvocation; import org.apache.flink.runtime.rpc.akka.messages.RunAsync; @@ -35,6 +36,7 @@ import scala.concurrent.Future; import scala.concurrent.duration.FiniteDuration; +import java.io.IOException; import java.lang.reflect.Method; import java.util.concurrent.Callable; import java.util.concurrent.TimeUnit; @@ -42,10 +44,10 @@ import static org.apache.flink.util.Preconditions.checkNotNull; /** - * Akka rpc actor which receives {@link RpcInvocation}, {@link RunAsync} and {@link CallAsync} + * Akka rpc actor which receives {@link LocalRpcInvocation}, {@link RunAsync} and {@link CallAsync} * messages. *

- * The {@link RpcInvocation} designates a rpc and is dispatched to the given {@link RpcEndpoint} + * The {@link LocalRpcInvocation} designates a rpc and is dispatched to the given {@link RpcEndpoint} * instance. *

* The {@link RunAsync} and {@link CallAsync} messages contain executable code which is executed @@ -95,15 +97,12 @@ public void onReceive(final Object message) { * @param rpcInvocation Rpc invocation message */ private void handleRpcInvocation(RpcInvocation rpcInvocation) { - Method rpcMethod = null; - try { - rpcMethod = lookupRpcMethod(rpcInvocation.getMethodName(), rpcInvocation.getParameterTypes()); - } catch (final NoSuchMethodException e) { - LOG.error("Could not find rpc method for rpc invocation: {}.", rpcInvocation, e); - } + String methodName = rpcInvocation.getMethodName(); + Class[] parameterTypes = rpcInvocation.getParameterTypes(); + + Method rpcMethod = lookupRpcMethod(methodName, parameterTypes); - if (rpcMethod != null) { if (rpcMethod.getReturnType().equals(Void.TYPE)) { // No return value to send back try { @@ -127,6 +126,12 @@ private void handleRpcInvocation(RpcInvocation rpcInvocation) { getSender().tell(new Status.Failure(e), getSelf()); } } + } catch(ClassNotFoundException e) { + LOG.error("Could not load method arguments.", e); + } catch (IOException e) { + LOG.error("Could not deserialize rpc invocation message.", e); + } catch (final NoSuchMethodException e) { + LOG.error("Could not find rpc method for rpc invocation: {}.", rpcInvocation, e); } } @@ -195,7 +200,8 @@ else if (runAsync.getDelay() == 0) { * @param methodName Name of the method * @param parameterTypes Parameter types of the method * @return Method of the rpc endpoint - * @throws NoSuchMethodException + * @throws NoSuchMethodException Thrown if the method with the given name and parameter types + * cannot be found at the rpc endpoint */ private Method lookupRpcMethod(final String methodName, final Class[] parameterTypes) throws NoSuchMethodException { return rpcEndpoint.getClass().getMethod(methodName, parameterTypes); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java index db40f10e10f39..b963c53a93a87 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java @@ -58,17 +58,27 @@ public class AkkaRpcService implements RpcService { private static final Logger LOG = LoggerFactory.getLogger(AkkaRpcService.class); + static final String MAXIMUM_FRAME_SIZE_PATH = "akka.remote.netty.tcp.maximum-frame-size"; + private final Object lock = new Object(); private final ActorSystem actorSystem; private final Timeout timeout; private final Set actors = new HashSet<>(4); + private final long maximumFramesize; private volatile boolean stopped; public AkkaRpcService(final ActorSystem actorSystem, final Timeout timeout) { this.actorSystem = checkNotNull(actorSystem, "actor system"); this.timeout = checkNotNull(timeout, "timeout"); + + if (actorSystem.settings().config().hasPath(MAXIMUM_FRAME_SIZE_PATH)) { + maximumFramesize = actorSystem.settings().config().getBytes(MAXIMUM_FRAME_SIZE_PATH); + } else { + // only local communication + maximumFramesize = Long.MAX_VALUE; + } } // this method does not mutate state and is thus thread-safe @@ -88,7 +98,7 @@ public Future connect(final String address, final Clas public C apply(Object obj) { ActorRef actorRef = ((ActorIdentity) obj).getRef(); - InvocationHandler akkaInvocationHandler = new AkkaInvocationHandler(actorRef, timeout); + InvocationHandler akkaInvocationHandler = new AkkaInvocationHandler(actorRef, timeout, maximumFramesize); @SuppressWarnings("unchecked") C proxy = (C) Proxy.newProxyInstance( @@ -116,7 +126,7 @@ public > C startServer(S rpcEndpo LOG.info("Starting RPC endpoint for {} at {} .", rpcEndpoint.getClass().getName(), actorRef.path()); - InvocationHandler akkaInvocationHandler = new AkkaInvocationHandler(actorRef, timeout); + InvocationHandler akkaInvocationHandler = new AkkaInvocationHandler(actorRef, timeout, maximumFramesize); // Rather than using the System ClassLoader directly, we derive the ClassLoader // from this class . That works better in cases where Flink runs embedded and all Flink @@ -142,12 +152,12 @@ public void stopServer(RpcGateway selfGateway) { if (stopped) { return; } else { - fromThisService = actors.remove(akkaClient.getRpcServer()); + fromThisService = actors.remove(akkaClient.getRpcEndpoint()); } } if (fromThisService) { - ActorRef selfActorRef = akkaClient.getRpcServer(); + ActorRef selfActorRef = akkaClient.getRpcEndpoint(); LOG.info("Stopping RPC endpoint {}.", selfActorRef.path()); selfActorRef.tell(PoisonPill.getInstance(), ActorRef.noSender()); } else { @@ -178,7 +188,7 @@ public String getAddress(RpcGateway selfGateway) { checkState(!stopped, "RpcService is stopped"); if (selfGateway instanceof AkkaGateway) { - ActorRef actorRef = ((AkkaGateway) selfGateway).getRpcServer(); + ActorRef actorRef = ((AkkaGateway) selfGateway).getRpcEndpoint(); return AkkaUtils.getAkkaURL(actorSystem, actorRef); } else { String className = AkkaGateway.class.getName(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/LocalRpcInvocation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/LocalRpcInvocation.java new file mode 100644 index 0000000000000..97c10d71bf141 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/LocalRpcInvocation.java @@ -0,0 +1,54 @@ +/* + * 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.rpc.akka.messages; + +import org.apache.flink.util.Preconditions; + +/** + * Local rpc invocation message containing the remote procedure name, its parameter types and the + * corresponding call arguments. This message will only be sent if the communication is local and, + * thus, the message does not have to be serialized. + */ +public final class LocalRpcInvocation implements RpcInvocation { + + private final String methodName; + private final Class[] parameterTypes; + private final Object[] args; + + public LocalRpcInvocation(String methodName, Class[] parameterTypes, Object[] args) { + this.methodName = Preconditions.checkNotNull(methodName); + this.parameterTypes = Preconditions.checkNotNull(parameterTypes); + this.args = args; + } + + @Override + public String getMethodName() { + return methodName; + } + + @Override + public Class[] getParameterTypes() { + return parameterTypes; + } + + @Override + public Object[] getArgs() { + return args; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RemoteRpcInvocation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RemoteRpcInvocation.java new file mode 100644 index 0000000000000..bc26a29715c91 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RemoteRpcInvocation.java @@ -0,0 +1,206 @@ +/* + * 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.rpc.akka.messages; + +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.SerializedValue; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; + +/** + * Remote rpc invocation message which is used when the actor communication is remote and, thus, the + * message has to be serialized. + *

+ * In order to fail fast and report an appropriate error message to the user, the method name, the + * parameter types and the arguments are eagerly serialized. In case the the invocation call + * contains a non-serializable object, then an {@link IOException} is thrown. + */ +public class RemoteRpcInvocation implements RpcInvocation, Serializable { + private static final long serialVersionUID = 6179354390913843809L; + + // Serialized invocation data + private SerializedValue serializedMethodInvocation; + + // Transient field which is lazily initialized upon first access to the invocation data + private transient RemoteRpcInvocation.MethodInvocation methodInvocation; + + public RemoteRpcInvocation( + final String methodName, + final Class[] parameterTypes, + final Object[] args) throws IOException { + + serializedMethodInvocation = new SerializedValue<>(new RemoteRpcInvocation.MethodInvocation(methodName, parameterTypes, args)); + methodInvocation = null; + } + + @Override + public String getMethodName() throws IOException, ClassNotFoundException { + deserializeMethodInvocation(); + + return methodInvocation.getMethodName(); + } + + @Override + public Class[] getParameterTypes() throws IOException, ClassNotFoundException { + deserializeMethodInvocation(); + + return methodInvocation.getParameterTypes(); + } + + @Override + public Object[] getArgs() throws IOException, ClassNotFoundException { + deserializeMethodInvocation(); + + return methodInvocation.getArgs(); + } + + /** + * Size (#bytes of the serialized data) of the rpc invocation message. + * + * @return Size of the remote rpc invocation message + */ + public long getSize() { + return serializedMethodInvocation.getByteArray().length; + } + + private void deserializeMethodInvocation() throws IOException, ClassNotFoundException { + if (methodInvocation == null) { + methodInvocation = serializedMethodInvocation.deserializeValue(ClassLoader.getSystemClassLoader()); + } + } + + // ------------------------------------------------------------------- + // Serialization methods + // ------------------------------------------------------------------- + + private void writeObject(ObjectOutputStream oos) throws IOException { + oos.writeObject(serializedMethodInvocation); + } + + @SuppressWarnings("unchecked") + private void readObject(ObjectInputStream ois) throws IOException, ClassNotFoundException { + serializedMethodInvocation = (SerializedValue) ois.readObject(); + methodInvocation = null; + } + + // ------------------------------------------------------------------- + // Utility classes + // ------------------------------------------------------------------- + + /** + * Wrapper class for the method invocation information + */ + private static final class MethodInvocation implements Serializable { + private static final long serialVersionUID = 9187962608946082519L; + + private String methodName; + private Class[] parameterTypes; + private Object[] args; + + private MethodInvocation(final String methodName, final Class[] parameterTypes, final Object[] args) { + this.methodName = methodName; + this.parameterTypes = Preconditions.checkNotNull(parameterTypes); + this.args = args; + } + + String getMethodName() { + return methodName; + } + + Class[] getParameterTypes() { + return parameterTypes; + } + + Object[] getArgs() { + return args; + } + + private void writeObject(ObjectOutputStream oos) throws IOException { + oos.writeUTF(methodName); + + oos.writeInt(parameterTypes.length); + + for (Class parameterType : parameterTypes) { + oos.writeObject(parameterType); + } + + if (args != null) { + oos.writeBoolean(true); + + for (int i = 0; i < args.length; i++) { + try { + oos.writeObject(args[i]); + } catch (IOException e) { + throw new IOException("Could not serialize " + i + "th argument of method " + + methodName + ". This indicates that the argument type " + + args.getClass().getName() + " is not serializable. Arguments have to " + + "be serializable for remote rpc calls.", e); + } + } + } else { + oos.writeBoolean(false); + } + } + + private void readObject(ObjectInputStream ois) throws IOException, ClassNotFoundException { + methodName = ois.readUTF(); + + int length = ois.readInt(); + + parameterTypes = new Class[length]; + + for (int i = 0; i < length; i++) { + try { + parameterTypes[i] = (Class) ois.readObject(); + } catch (IOException e) { + throw new IOException("Could not deserialize " + i + "th parameter type of method " + + methodName + '.', e); + } catch (ClassNotFoundException e) { + throw new ClassNotFoundException("Could not deserialize " + i + "th " + + "parameter type of method " + methodName + ". This indicates that the parameter " + + "type is not part of the system class loader.", e); + } + } + + boolean hasArgs = ois.readBoolean(); + + if (hasArgs) { + args = new Object[length]; + + for (int i = 0; i < length; i++) { + try { + args[i] = ois.readObject(); + } catch (IOException e) { + throw new IOException("Could not deserialize " + i + "th argument of method " + + methodName + '.', e); + } catch (ClassNotFoundException e) { + throw new ClassNotFoundException("Could not deserialize " + i + "th " + + "argument of method " + methodName + ". This indicates that the argument " + + "type is not part of the system class loader.", e); + } + } + } else { + args = null; + } + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RpcInvocation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RpcInvocation.java index 5d52ef1c0b298..b174c99a4d37c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RpcInvocation.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RpcInvocation.java @@ -18,81 +18,41 @@ package org.apache.flink.runtime.rpc.akka.messages; -import org.apache.flink.util.Preconditions; - import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import java.io.Serializable; /** - * Rpc invocation message containing the remote procedure name, its parameter types and the - * corresponding call arguments. + * Interface for rpc invocation messages. The interface allows to request all necessary information + * to lookup a method and call it with the corresponding arguments. */ -public final class RpcInvocation implements Serializable { - private static final long serialVersionUID = -7058254033460536037L; - - private final String methodName; - private final Class[] parameterTypes; - private transient Object[] args; - - public RpcInvocation(String methodName, Class[] parameterTypes, Object[] args) { - this.methodName = Preconditions.checkNotNull(methodName); - this.parameterTypes = Preconditions.checkNotNull(parameterTypes); - this.args = args; - } - - public String getMethodName() { - return methodName; - } - - public Class[] getParameterTypes() { - return parameterTypes; - } - - public Object[] getArgs() { - return args; - } - - private void writeObject(ObjectOutputStream oos) throws IOException { - oos.defaultWriteObject(); - - if (args != null) { - // write has args true - oos.writeBoolean(true); - - for (int i = 0; i < args.length; i++) { - try { - oos.writeObject(args[i]); - } catch (IOException e) { - Class argClass = args[i].getClass(); - - throw new IOException("Could not write " + i + "th argument of method " + - methodName + ". The argument type is " + argClass + ". " + - "Make sure that this type is serializable.", e); - } - } - } else { - // write has args false - oos.writeBoolean(false); - } - } - - private void readObject(ObjectInputStream ois) throws IOException, ClassNotFoundException { - ois.defaultReadObject(); - - boolean hasArgs = ois.readBoolean(); - - if (hasArgs) { - int numberArguments = parameterTypes.length; - - args = new Object[numberArguments]; - - for (int i = 0; i < numberArguments; i++) { - args[i] = ois.readObject(); - } - } else { - args = null; - } - } +public interface RpcInvocation { + + /** + * Returns the method's name. + * + * @return Method name + * @throws IOException if the rpc invocation message is a remote message and could not be deserialized + * @throws ClassNotFoundException if the rpc invocation message is a remote message and contains + * serialized classes which cannot be found on the receiving side + */ + String getMethodName() throws IOException, ClassNotFoundException; + + /** + * Returns the method's parameter types + * + * @return Method's parameter types + * @throws IOException if the rpc invocation message is a remote message and could not be deserialized + * @throws ClassNotFoundException if the rpc invocation message is a remote message and contains + * serialized classes which cannot be found on the receiving side + */ + Class[] getParameterTypes() throws IOException, ClassNotFoundException; + + /** + * Returns the arguments of the remote procedure call + * + * @return Arguments of the remote procedure call + * @throws IOException if the rpc invocation message is a remote message and could not be deserialized + * @throws ClassNotFoundException if the rpc invocation message is a remote message and contains + * serialized classes which cannot be found on the receiving side + */ + Object[] getArgs() throws IOException, ClassNotFoundException; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java index 5e37e10ff0dc5..f26b40b8a997e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java @@ -64,7 +64,7 @@ public void testJobMasterResourceManagerRegistration() throws Exception { AkkaGateway akkaClient = (AkkaGateway) rm; - jobMaster.registerAtResourceManager(AkkaUtils.getAkkaURL(actorSystem, akkaClient.getRpcServer())); + jobMaster.registerAtResourceManager(AkkaUtils.getAkkaURL(actorSystem, akkaClient.getRpcEndpoint())); // wait for successful registration FiniteDuration timeout = new FiniteDuration(200, TimeUnit.SECONDS); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java new file mode 100644 index 0000000000000..ca8179c63f5e1 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java @@ -0,0 +1,210 @@ +/* + * 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.rpc.akka; + +import akka.actor.ActorSystem; +import akka.util.Timeout; +import com.typesafe.config.Config; +import com.typesafe.config.ConfigValueFactory; +import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.rpc.RpcEndpoint; +import org.apache.flink.runtime.rpc.RpcGateway; +import org.apache.flink.runtime.rpc.RpcMethod; +import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.util.TestLogger; +import org.hamcrest.core.Is; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import scala.concurrent.Await; +import scala.concurrent.Future; +import scala.concurrent.duration.FiniteDuration; + +import java.io.IOException; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; + +import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; + +/** + * Tests that akka rpc invocation messages are properly serialized and errors reported + */ +public class MessageSerializationTest extends TestLogger { + private static ActorSystem actorSystem1; + private static ActorSystem actorSystem2; + private static AkkaRpcService akkaRpcService1; + private static AkkaRpcService akkaRpcService2; + + private static final FiniteDuration timeout = new FiniteDuration(10L, TimeUnit.SECONDS); + private static final int maxFrameSize = 32000; + + @BeforeClass + public static void setup() { + Config akkaConfig = AkkaUtils.getDefaultAkkaConfig(); + Config modifiedAkkaConfig = akkaConfig.withValue(AkkaRpcService.MAXIMUM_FRAME_SIZE_PATH, ConfigValueFactory.fromAnyRef(maxFrameSize + "b")); + + actorSystem1 = AkkaUtils.createActorSystem(modifiedAkkaConfig); + actorSystem2 = AkkaUtils.createActorSystem(modifiedAkkaConfig); + + akkaRpcService1 = new AkkaRpcService(actorSystem1, new Timeout(timeout)); + akkaRpcService2 = new AkkaRpcService(actorSystem2, new Timeout(timeout)); + } + + @AfterClass + public static void teardown() { + akkaRpcService1.stopService(); + akkaRpcService2.stopService(); + + actorSystem1.shutdown(); + actorSystem2.shutdown(); + + actorSystem1.awaitTermination(); + actorSystem2.awaitTermination(); + } + + /** + * Tests that a local rpc call with a non serializable argument can be executed. + */ + @Test + public void testNonSerializableLocalMessageTransfer() throws InterruptedException, IOException { + LinkedBlockingQueue linkedBlockingQueue = new LinkedBlockingQueue<>(); + TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService1, linkedBlockingQueue); + + TestGateway testGateway = testEndpoint.getSelf(); + + NonSerializableObject expected = new NonSerializableObject(42); + + testGateway.foobar(expected); + + assertThat(linkedBlockingQueue.take(), Is.is(expected)); + } + + /** + * Tests that a remote rpc call with a non-serializable argument fails with an + * {@link IOException} (or an {@link java.lang.reflect.UndeclaredThrowableException} if the + * the method declaration does not include the {@link IOException} as throwable). + */ + @Test(expected = IOException.class) + public void testNonSerializableRemoteMessageTransfer() throws Exception { + LinkedBlockingQueue linkedBlockingQueue = new LinkedBlockingQueue<>(); + + TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService1, linkedBlockingQueue); + + String address = testEndpoint.getAddress(); + + Future remoteGatewayFuture = akkaRpcService2.connect(address, TestGateway.class); + + TestGateway remoteGateway = Await.result(remoteGatewayFuture, timeout); + + remoteGateway.foobar(new Object()); + + fail("Should have failed because Object is not serializable."); + } + + /** + * Tests that a remote rpc call with a serializable argument can be successfully executed. + */ + @Test + public void testSerializableRemoteMessageTransfer() throws Exception { + LinkedBlockingQueue linkedBlockingQueue = new LinkedBlockingQueue<>(); + + TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService1, linkedBlockingQueue); + + String address = testEndpoint.getAddress(); + + Future remoteGatewayFuture = akkaRpcService2.connect(address, TestGateway.class); + + TestGateway remoteGateway = Await.result(remoteGatewayFuture, timeout); + + int expected = 42; + + remoteGateway.foobar(expected); + + assertThat(linkedBlockingQueue.take(), Is.is(expected)); + } + + /** + * Tests that a message which exceeds the maximum frame size is detected and a corresponding + * exception is thrown. + */ + @Test(expected = IOException.class) + public void testMaximumFramesizeRemoteMessageTransfer() throws Exception { + LinkedBlockingQueue linkedBlockingQueue = new LinkedBlockingQueue<>(); + + TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService1, linkedBlockingQueue); + + String address = testEndpoint.getAddress(); + + Future remoteGatewayFuture = akkaRpcService2.connect(address, TestGateway.class); + + TestGateway remoteGateway = Await.result(remoteGatewayFuture, timeout); + + int bufferSize = maxFrameSize + 1; + byte[] buffer = new byte[bufferSize]; + + remoteGateway.foobar(buffer); + + fail("Should have failed due to exceeding the maximum framesize."); + } + + private interface TestGateway extends RpcGateway { + void foobar(Object object) throws IOException, InterruptedException; + } + + private static class TestEndpoint extends RpcEndpoint { + + private final LinkedBlockingQueue queue; + + protected TestEndpoint(RpcService rpcService, LinkedBlockingQueue queue) { + super(rpcService); + this.queue = queue; + } + + @RpcMethod + public void foobar(Object object) throws InterruptedException { + queue.put(object); + } + } + + private static class NonSerializableObject { + private final Object object = new Object(); + private final int value; + + NonSerializableObject(int value) { + this.value = value; + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof NonSerializableObject) { + NonSerializableObject nonSerializableObject = (NonSerializableObject) obj; + + return value == nonSerializableObject.value; + } else { + return false; + } + } + + @Override + public int hashCode() { + return value * 41; + } + } +} From d17afc1c586295467b06b282c9c826e436e5f073 Mon Sep 17 00:00:00 2001 From: Kurt Young Date: Fri, 12 Aug 2016 11:05:48 +0800 Subject: [PATCH 08/50] [FLINK-4373] [cluster management] Introduce SlotID, AllocationID, ResourceProfile [FLINK-4373] [cluster management] address comments This closes #2370. --- .../clusterframework/types/AllocationID.java | 32 +++++++ .../types/ResourceProfile.java | 68 +++++++++++++++ .../clusterframework/types/SlotID.java | 83 +++++++++++++++++++ .../types/ResourceProfileTest.java | 49 +++++++++++ 4 files changed, 232 insertions(+) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/AllocationID.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/SlotID.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/ResourceProfileTest.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/AllocationID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/AllocationID.java new file mode 100644 index 0000000000000..f7ae6eeda5768 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/AllocationID.java @@ -0,0 +1,32 @@ +/* + * 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.clusterframework.types; + +import org.apache.flink.util.AbstractID; + +/** + * Unique identifier for the attempt to allocate a slot, normally created by JobManager when requesting a slot, + * constant across re-tries. This can also be used to identify responses by the ResourceManager and to identify + * deployment calls towards the TaskManager that was allocated from. + */ +public class AllocationID extends AbstractID { + + private static final long serialVersionUID = 1L; + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java new file mode 100644 index 0000000000000..cbe709f946a77 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java @@ -0,0 +1,68 @@ +/* + * 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.clusterframework.types; + +import java.io.Serializable; + +/** + * Describe the resource profile of the slot, either when requiring or offering it. The profile can be + * checked whether it can match another profile's requirement, and furthermore we may calculate a matching + * score to decide which profile we should choose when we have lots of candidate slots. + */ +public class ResourceProfile implements Serializable { + + private static final long serialVersionUID = -784900073893060124L; + + /** How many cpu cores are needed, use double so we can specify cpu like 0.1 */ + private final double cpuCores; + + /** How many memory in mb are needed */ + private final long memoryInMB; + + public ResourceProfile(double cpuCores, long memoryInMB) { + this.cpuCores = cpuCores; + this.memoryInMB = memoryInMB; + } + + /** + * Get the cpu cores needed + * @return The cpu cores, 1.0 means a full cpu thread + */ + public double getCpuCores() { + return cpuCores; + } + + /** + * Get the memory needed in MB + * @return The memory in MB + */ + public long getMemoryInMB() { + return memoryInMB; + } + + /** + * Check whether required resource profile can be matched + * + * @param required the required resource profile + * @return true if the requirement is matched, otherwise false + */ + public boolean isMatching(ResourceProfile required) { + return Double.compare(cpuCores, required.getCpuCores()) >= 0 && memoryInMB >= required.getMemoryInMB(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/SlotID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/SlotID.java new file mode 100644 index 0000000000000..d1b072defda76 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/SlotID.java @@ -0,0 +1,83 @@ +/* + * 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.clusterframework.types; + +import java.io.Serializable; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Unique identifier for a slot which located in TaskManager. + */ +public class SlotID implements ResourceIDRetrievable, Serializable { + + private static final long serialVersionUID = -6399206032549807771L; + + /** The resource id which this slot located */ + private final ResourceID resourceId; + + /** The numeric id for single slot */ + private final int slotId; + + public SlotID(ResourceID resourceId, int slotId) { + this.resourceId = checkNotNull(resourceId, "ResourceID must not be null"); + this.slotId = slotId; + } + + // ------------------------------------------------------------------------ + + @Override + public ResourceID getResourceID() { + return resourceId; + } + + // ------------------------------------------------------------------------ + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + SlotID slotID = (SlotID) o; + + if (slotId != slotID.slotId) { + return false; + } + return resourceId.equals(slotID.resourceId); + } + + @Override + public int hashCode() { + int result = resourceId.hashCode(); + result = 31 * result + slotId; + return result; + } + + @Override + public String toString() { + return "SlotID{" + + "resourceId=" + resourceId + + ", slotId=" + slotId + + '}'; + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/ResourceProfileTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/ResourceProfileTest.java new file mode 100644 index 0000000000000..bc5ddaa8dac6f --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/ResourceProfileTest.java @@ -0,0 +1,49 @@ +/* + * 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.clusterframework.types; + +import org.junit.Test; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class ResourceProfileTest { + + @Test + public void testMatchRequirement() throws Exception { + ResourceProfile rp1 = new ResourceProfile(1.0, 100); + ResourceProfile rp2 = new ResourceProfile(1.0, 200); + ResourceProfile rp3 = new ResourceProfile(2.0, 100); + ResourceProfile rp4 = new ResourceProfile(2.0, 200); + + assertFalse(rp1.isMatching(rp2)); + assertTrue(rp2.isMatching(rp1)); + + assertFalse(rp1.isMatching(rp3)); + assertTrue(rp3.isMatching(rp1)); + + assertFalse(rp2.isMatching(rp3)); + assertFalse(rp3.isMatching(rp2)); + + assertTrue(rp4.isMatching(rp1)); + assertTrue(rp4.isMatching(rp2)); + assertTrue(rp4.isMatching(rp3)); + assertTrue(rp4.isMatching(rp4)); + } +} From afabd78987e31d882de929c80438f28196230368 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 11 Aug 2016 18:13:25 +0200 Subject: [PATCH 09/50] [FLINK-4382] [rpc] Buffer rpc calls until the RpcEndpoint has been started This PR allows the AkkaRpcActor to stash messages until the corresponding RcpEndpoint has been started. When receiving a Processing.START message, the AkkaRpcActor unstashes all messages and starts processing rpcs. When receiving a Processing.STOP message, it will stop processing messages and stash incoming messages again. Add test case for message stashing This closes #2358. --- .../apache/flink/runtime/rpc/RpcEndpoint.java | 15 ++- .../flink/runtime/rpc/StartStoppable.java | 35 ++++++ .../rpc/akka/AkkaInvocationHandler.java | 21 +++- .../flink/runtime/rpc/akka/AkkaRpcActor.java | 39 ++++++- .../runtime/rpc/akka/AkkaRpcService.java | 8 +- .../runtime/rpc/akka/messages/Processing.java | 27 +++++ .../runtime/rpc/RpcCompletenessTest.java | 45 +++++++- .../runtime/rpc/akka/AkkaRpcActorTest.java | 108 ++++++++++++++++++ .../runtime/rpc/akka/AkkaRpcServiceTest.java | 3 + .../runtime/rpc/akka/AsyncCallsTest.java | 5 +- .../rpc/akka/MainThreadValidationTest.java | 4 +- .../rpc/akka/MessageSerializationTest.java | 4 + .../rpc/taskexecutor/TaskExecutorTest.java | 18 +++ 13 files changed, 315 insertions(+), 17 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/StartStoppable.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/Processing.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java index d36a283f26557..67ac182dbf5af 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java @@ -74,7 +74,7 @@ public abstract class RpcEndpoint { /** The main thread execution context to be used to execute future callbacks in the main thread * of the executing rpc server. */ - private final MainThreadExecutionContext mainThreadExecutionContext; + private final ExecutionContext mainThreadExecutionContext; /** A reference to the endpoint's main thread, if the current method is called by the main thread */ final AtomicReference currentMainThread = new AtomicReference<>(null); @@ -106,9 +106,20 @@ public final Class getSelfGatewayType() { } // ------------------------------------------------------------------------ - // Shutdown + // Start & Shutdown // ------------------------------------------------------------------------ + /** + * Starts the rpc endpoint. This tells the underlying rpc server that the rpc endpoint is ready + * to process remote procedure calls. + * + * IMPORTANT: Whenever you override this method, call the parent implementation to enable + * rpc processing. It is advised to make the parent call last. + */ + public void start() { + ((StartStoppable) self).start(); + } + /** * Shuts down the underlying RPC endpoint via the RPC service. * After this method was called, the RPC endpoint will no longer be reachable, neither remotely, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/StartStoppable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/StartStoppable.java new file mode 100644 index 0000000000000..dd5595fc77501 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/StartStoppable.java @@ -0,0 +1,35 @@ +/* + * 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.rpc; + +/** + * Interface to start and stop the processing of rpc calls in the rpc server. + */ +public interface StartStoppable { + + /** + * Starts the processing of remote procedure calls. + */ + void start(); + + /** + * Stops the processing of remote procedure calls. + */ + void stop(); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java index 297104b4beac3..524bf7480097d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java @@ -24,8 +24,10 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.rpc.MainThreadExecutor; import org.apache.flink.runtime.rpc.RpcTimeout; +import org.apache.flink.runtime.rpc.StartStoppable; import org.apache.flink.runtime.rpc.akka.messages.CallAsync; import org.apache.flink.runtime.rpc.akka.messages.LocalRpcInvocation; +import org.apache.flink.runtime.rpc.akka.messages.Processing; import org.apache.flink.runtime.rpc.akka.messages.RemoteRpcInvocation; import org.apache.flink.runtime.rpc.akka.messages.RpcInvocation; import org.apache.flink.runtime.rpc.akka.messages.RunAsync; @@ -50,7 +52,7 @@ * rpc in a {@link LocalRpcInvocation} message and then sends it to the {@link AkkaRpcActor} where it is * executed. */ -class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThreadExecutor { +class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThreadExecutor, StartStoppable { private static final Logger LOG = Logger.getLogger(AkkaInvocationHandler.class); private final ActorRef rpcEndpoint; @@ -76,7 +78,8 @@ public Object invoke(Object proxy, Method method, Object[] args) throws Throwabl Object result; - if (declaringClass.equals(AkkaGateway.class) || declaringClass.equals(MainThreadExecutor.class) || declaringClass.equals(Object.class)) { + if (declaringClass.equals(AkkaGateway.class) || declaringClass.equals(MainThreadExecutor.class) || + declaringClass.equals(Object.class) || declaringClass.equals(StartStoppable.class)) { result = method.invoke(this, args); } else { String methodName = method.getName(); @@ -171,6 +174,20 @@ public Future callAsync(Callable callable, Timeout callTimeout) { } } + @Override + public void start() { + rpcEndpoint.tell(Processing.START, ActorRef.noSender()); + } + + @Override + public void stop() { + rpcEndpoint.tell(Processing.STOP, ActorRef.noSender()); + } + + // ------------------------------------------------------------------------ + // Helper methods + // ------------------------------------------------------------------------ + /** * Extracts the {@link RpcTimeout} annotated rpc timeout value from the list of given method * arguments. If no {@link RpcTimeout} annotated parameter could be found, then the default diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java index dfcbcc3924374..2373be9414ed2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java @@ -20,13 +20,15 @@ import akka.actor.ActorRef; import akka.actor.Status; -import akka.actor.UntypedActor; +import akka.actor.UntypedActorWithStash; +import akka.japi.Procedure; import akka.pattern.Patterns; import org.apache.flink.runtime.rpc.MainThreadValidatorUtil; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.akka.messages.CallAsync; import org.apache.flink.runtime.rpc.akka.messages.LocalRpcInvocation; +import org.apache.flink.runtime.rpc.akka.messages.Processing; import org.apache.flink.runtime.rpc.akka.messages.RpcInvocation; import org.apache.flink.runtime.rpc.akka.messages.RunAsync; @@ -45,18 +47,23 @@ /** * Akka rpc actor which receives {@link LocalRpcInvocation}, {@link RunAsync} and {@link CallAsync} - * messages. + * {@link Processing} messages. *

* The {@link LocalRpcInvocation} designates a rpc and is dispatched to the given {@link RpcEndpoint} * instance. *

* The {@link RunAsync} and {@link CallAsync} messages contain executable code which is executed * in the context of the actor thread. + *

+ * The {@link Processing} message controls the processing behaviour of the akka rpc actor. A + * {@link Processing#START} message unstashes all stashed messages and starts processing incoming + * messages. A {@link Processing#STOP} message stops processing messages and stashes incoming + * messages. * * @param Type of the {@link RpcGateway} associated with the {@link RpcEndpoint} * @param Type of the {@link RpcEndpoint} */ -class AkkaRpcActor> extends UntypedActor { +class AkkaRpcActor> extends UntypedActorWithStash { private static final Logger LOG = LoggerFactory.getLogger(AkkaRpcActor.class); @@ -73,6 +80,27 @@ class AkkaRpcActor> extends Untyp @Override public void onReceive(final Object message) { + if (message.equals(Processing.START)) { + unstashAll(); + getContext().become(new Procedure() { + @Override + public void apply(Object message) throws Exception { + if (message.equals(Processing.STOP)) { + getContext().unbecome(); + } else { + handleMessage(message); + } + } + }); + } else { + LOG.info("The rpc endpoint {} has not been started yet. Stashing message {} until processing is started.", + rpcEndpoint.getClass().getName(), + message.getClass().getName()); + stash(); + } + } + + private void handleMessage(Object message) { mainThreadValidator.enterMainThread(); try { if (message instanceof RunAsync) { @@ -82,7 +110,10 @@ public void onReceive(final Object message) { } else if (message instanceof RpcInvocation) { handleRpcInvocation((RpcInvocation) message); } else { - LOG.warn("Received message of unknown type {}. Dropping this message!", message.getClass()); + LOG.warn( + "Received message of unknown type {} with value {}. Dropping this message!", + message.getClass().getName(), + message); } } finally { mainThreadValidator.exitMainThread(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java index b963c53a93a87..7b3352401283d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java @@ -34,7 +34,7 @@ import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcService; - +import org.apache.flink.runtime.rpc.StartStoppable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -136,7 +136,11 @@ public > C startServer(S rpcEndpo @SuppressWarnings("unchecked") C self = (C) Proxy.newProxyInstance( classLoader, - new Class[]{rpcEndpoint.getSelfGatewayType(), MainThreadExecutor.class, AkkaGateway.class}, + new Class[]{ + rpcEndpoint.getSelfGatewayType(), + MainThreadExecutor.class, + StartStoppable.class, + AkkaGateway.class}, akkaInvocationHandler); return self; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/Processing.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/Processing.java new file mode 100644 index 0000000000000..5c7df5dcd6021 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/Processing.java @@ -0,0 +1,27 @@ +/* + * 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.rpc.akka.messages; + +/** + * Controls the processing behaviour of the {@link org.apache.flink.runtime.rpc.akka.AkkaRpcActor} + */ +public enum Processing { + START, // Unstashes all stashed messages and starts processing incoming messages + STOP // Stop processing messages and stashes all incoming messages +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java index e50533e2a1538..97cf0cb35524e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java @@ -18,6 +18,8 @@ package org.apache.flink.runtime.rpc; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.util.ReflectionUtil; import org.apache.flink.util.TestLogger; import org.junit.Test; @@ -140,7 +142,7 @@ private void checkGatewayMethod(Method gatewayMethod) { int rpcTimeoutParameters = 0; for (int i = 0; i < parameterAnnotations.length; i++) { - if (isRpcTimeout(parameterAnnotations[i])) { + if (RpcCompletenessTest.isRpcTimeout(parameterAnnotations[i])) { assertTrue( "The rpc timeout has to be of type " + FiniteDuration.class.getName() + ".", parameterTypes[i].equals(FiniteDuration.class)); @@ -185,7 +187,7 @@ private boolean checkMethod(Method gatewayMethod, Method endpointMethod) { // filter out the RpcTimeout parameters for (int i = 0; i < gatewayParameterTypes.length; i++) { - if (!isRpcTimeout(gatewayParameterAnnotations[i])) { + if (!RpcCompletenessTest.isRpcTimeout(gatewayParameterAnnotations[i])) { filteredGatewayParameterTypes.add(gatewayParameterTypes[i]); } } @@ -235,7 +237,22 @@ private boolean checkMethod(Method gatewayMethod, Method endpointMethod) { } private boolean checkType(Class firstType, Class secondType) { - return firstType.equals(secondType); + Class firstResolvedType; + Class secondResolvedType; + + if (firstType.isPrimitive()) { + firstResolvedType = RpcCompletenessTest.resolvePrimitiveType(firstType); + } else { + firstResolvedType = firstType; + } + + if (secondType.isPrimitive()) { + secondResolvedType = RpcCompletenessTest.resolvePrimitiveType(secondType); + } else { + secondResolvedType = secondType; + } + + return firstResolvedType.equals(secondResolvedType); } /** @@ -279,7 +296,7 @@ private String generateEndpointMethodSignature(Method method) { for (int i = 0; i < parameterTypes.length; i++) { // filter out the RpcTimeout parameters - if (!isRpcTimeout(parameterAnnotations[i])) { + if (!RpcCompletenessTest.isRpcTimeout(parameterAnnotations[i])) { builder.append(parameterTypes[i].getName()); if (i < parameterTypes.length -1) { @@ -293,7 +310,7 @@ private String generateEndpointMethodSignature(Method method) { return builder.toString(); } - private boolean isRpcTimeout(Annotation[] annotations) { + private static boolean isRpcTimeout(Annotation[] annotations) { for (Annotation annotation : annotations) { if (annotation.annotationType().equals(RpcTimeout.class)) { return true; @@ -302,4 +319,22 @@ private boolean isRpcTimeout(Annotation[] annotations) { return false; } + + /** + * Returns the boxed type for a primitive type. + * + * @param primitveType Primitive type to resolve + * @return Boxed type for the given primitive type + */ + private static Class resolvePrimitiveType(Class primitveType) { + assert primitveType.isPrimitive(); + + TypeInformation typeInformation = BasicTypeInfo.getInfoFor(primitveType); + + if (typeInformation != null) { + return typeInformation.getTypeClass(); + } else { + throw new RuntimeException("Could not retrive basic type information for primitive type " + primitveType + '.'); + } + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java new file mode 100644 index 0000000000000..1653facb492a7 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java @@ -0,0 +1,108 @@ +/* + * 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.rpc.akka; + +import akka.actor.ActorSystem; +import akka.util.Timeout; +import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.rpc.RpcEndpoint; +import org.apache.flink.runtime.rpc.RpcGateway; +import org.apache.flink.runtime.rpc.RpcMethod; +import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.util.TestLogger; +import org.hamcrest.core.Is; +import org.junit.AfterClass; +import org.junit.Test; +import scala.concurrent.Await; +import scala.concurrent.Future; + +import java.util.concurrent.TimeUnit; + +import static org.junit.Assert.assertThat; + +public class AkkaRpcActorTest extends TestLogger { + + // ------------------------------------------------------------------------ + // shared test members + // ------------------------------------------------------------------------ + + private static ActorSystem actorSystem = AkkaUtils.createDefaultActorSystem(); + + private static Timeout timeout = new Timeout(10000, TimeUnit.MILLISECONDS); + + private static AkkaRpcService akkaRpcService = + new AkkaRpcService(actorSystem, timeout); + + @AfterClass + public static void shutdown() { + akkaRpcService.stopService(); + actorSystem.shutdown(); + actorSystem.awaitTermination(); + } + + /** + * Tests that the {@link AkkaRpcActor} stashes messages until the corresponding + * {@link RpcEndpoint} has been started. + */ + @Test + public void testMessageStashing() throws Exception { + int expectedValue = 1337; + + DummyRpcEndpoint rpcEndpoint = new DummyRpcEndpoint(akkaRpcService); + + DummyRpcGateway rpcGateway = rpcEndpoint.getSelf(); + + // this message should not be processed until we've started the rpc endpoint + Future result = rpcGateway.foobar(); + + // set a new value which we expect to be returned + rpcEndpoint.setFoobar(expectedValue); + + // now process the rpc + rpcEndpoint.start(); + + Integer actualValue = Await.result(result, timeout.duration()); + + assertThat("The new foobar value should have been returned.", actualValue, Is.is(expectedValue)); + + rpcEndpoint.shutDown(); + } + + private interface DummyRpcGateway extends RpcGateway { + Future foobar(); + } + + private static class DummyRpcEndpoint extends RpcEndpoint { + + private volatile int _foobar = 42; + + protected DummyRpcEndpoint(RpcService rpcService) { + super(rpcService); + } + + @RpcMethod + public int foobar() { + return _foobar; + } + + public void setFoobar(int value) { + _foobar = value; + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java index f26b40b8a997e..fd55904d20de8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java @@ -57,6 +57,9 @@ public void testJobMasterResourceManagerRegistration() throws Exception { ResourceManager resourceManager = new ResourceManager(akkaRpcService, executorService); JobMaster jobMaster = new JobMaster(akkaRpcService2, executorService); + resourceManager.start(); + jobMaster.start(); + ResourceManagerGateway rm = resourceManager.getSelf(); assertTrue(rm instanceof AkkaGateway); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AsyncCallsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AsyncCallsTest.java index f2ce52d99fbdb..d33987ccf6d69 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AsyncCallsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AsyncCallsTest.java @@ -28,6 +28,7 @@ import org.apache.flink.runtime.rpc.RpcMethod; import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.util.TestLogger; import org.junit.AfterClass; import org.junit.Test; @@ -42,7 +43,7 @@ import static org.junit.Assert.*; -public class AsyncCallsTest { +public class AsyncCallsTest extends TestLogger { // ------------------------------------------------------------------------ // shared test members @@ -72,6 +73,7 @@ public void testScheduleWithNoDelay() throws Exception { final AtomicBoolean concurrentAccess = new AtomicBoolean(false); TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService, lock); + testEndpoint.start(); TestGateway gateway = testEndpoint.getSelf(); // a bunch of gateway calls @@ -127,6 +129,7 @@ public void testScheduleWithDelay() throws Exception { final long delay = 200; TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService, lock); + testEndpoint.start(); // run something asynchronously testEndpoint.runAsync(new Runnable() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java index b85414320e790..9ffafdaec46b6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java @@ -27,13 +27,14 @@ import org.apache.flink.runtime.rpc.RpcMethod; import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.util.TestLogger; import org.junit.Test; import java.util.concurrent.TimeUnit; import static org.junit.Assert.assertTrue; -public class MainThreadValidationTest { +public class MainThreadValidationTest extends TestLogger { @Test public void failIfNotInMainThread() { @@ -51,6 +52,7 @@ public void failIfNotInMainThread() { try { TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService); + testEndpoint.start(); // this works, because it is executed as an RPC call testEndpoint.getSelf().someConcurrencyCriticalFunction(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java index ca8179c63f5e1..9d2ed99c080cd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java @@ -86,6 +86,7 @@ public static void teardown() { public void testNonSerializableLocalMessageTransfer() throws InterruptedException, IOException { LinkedBlockingQueue linkedBlockingQueue = new LinkedBlockingQueue<>(); TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService1, linkedBlockingQueue); + testEndpoint.start(); TestGateway testGateway = testEndpoint.getSelf(); @@ -106,6 +107,7 @@ public void testNonSerializableRemoteMessageTransfer() throws Exception { LinkedBlockingQueue linkedBlockingQueue = new LinkedBlockingQueue<>(); TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService1, linkedBlockingQueue); + testEndpoint.start(); String address = testEndpoint.getAddress(); @@ -126,6 +128,7 @@ public void testSerializableRemoteMessageTransfer() throws Exception { LinkedBlockingQueue linkedBlockingQueue = new LinkedBlockingQueue<>(); TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService1, linkedBlockingQueue); + testEndpoint.start(); String address = testEndpoint.getAddress(); @@ -149,6 +152,7 @@ public void testMaximumFramesizeRemoteMessageTransfer() throws Exception { LinkedBlockingQueue linkedBlockingQueue = new LinkedBlockingQueue<>(); TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService1, linkedBlockingQueue); + testEndpoint.start(); String address = testEndpoint.getAddress(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java index 33c9cb61ca4e6..c96f4f6cdea9f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java @@ -28,17 +28,26 @@ import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.rpc.MainThreadExecutor; +import org.apache.flink.runtime.rpc.RpcEndpoint; +import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.rpc.StartStoppable; import org.apache.flink.runtime.util.DirectExecutorService; import org.apache.flink.util.SerializedValue; import org.apache.flink.util.TestLogger; import org.junit.Test; +import org.mockito.Matchers; +import org.mockito.cglib.proxy.InvocationHandler; +import org.mockito.cglib.proxy.Proxy; +import scala.concurrent.Future; import java.net.URL; import java.util.Collections; import static org.junit.Assert.fail; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; public class TaskExecutorTest extends TestLogger { @@ -48,8 +57,13 @@ public class TaskExecutorTest extends TestLogger { @Test public void testTaskExecution() throws Exception { RpcService testingRpcService = mock(RpcService.class); + InvocationHandler invocationHandler = mock(InvocationHandler.class); + Object selfGateway = Proxy.newProxyInstance(ClassLoader.getSystemClassLoader(), new Class[] {TaskExecutorGateway.class, MainThreadExecutor.class, StartStoppable.class}, invocationHandler); + when(testingRpcService.startServer(Matchers.any(RpcEndpoint.class))).thenReturn((RpcGateway)selfGateway); + DirectExecutorService directExecutorService = new DirectExecutorService(); TaskExecutor taskExecutor = new TaskExecutor(testingRpcService, directExecutorService); + taskExecutor.start(); TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor( new JobID(), @@ -82,8 +96,12 @@ public void testTaskExecution() throws Exception { @Test(expected=Exception.class) public void testWrongTaskCancellation() throws Exception { RpcService testingRpcService = mock(RpcService.class); + InvocationHandler invocationHandler = mock(InvocationHandler.class); + Object selfGateway = Proxy.newProxyInstance(ClassLoader.getSystemClassLoader(), new Class[] {TaskExecutorGateway.class, MainThreadExecutor.class, StartStoppable.class}, invocationHandler); + when(testingRpcService.startServer(Matchers.any(RpcEndpoint.class))).thenReturn((RpcGateway)selfGateway); DirectExecutorService directExecutorService = null; TaskExecutor taskExecutor = new TaskExecutor(testingRpcService, directExecutorService); + taskExecutor.start(); taskExecutor.cancelTask(new ExecutionAttemptID()); From 2738709cf75b9c10215c4c2cce848f3381ac19bb Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Wed, 10 Aug 2016 20:42:45 +0200 Subject: [PATCH 10/50] [FLINK-4355] [cluster management] Implement TaskManager side of registration at ResourceManager. This closes #2353 --- .../HighAvailabilityServices.java | 39 +++ .../highavailability/NonHaServices.java | 59 ++++ .../StandaloneLeaderRetrievalService.java | 72 +++-- .../apache/flink/runtime/rpc/RpcEndpoint.java | 1 - .../apache/flink/runtime/rpc/RpcService.java | 27 ++ .../runtime/rpc/akka/AkkaRpcService.java | 18 ++ .../runtime/rpc/akka/messages/RunAsync.java | 1 + .../registration/RegistrationResponse.java | 84 +++++ .../registration/RetryingRegistration.java | 292 ++++++++++++++++++ .../rpc/resourcemanager/ResourceManager.java | 23 ++ .../ResourceManagerGateway.java | 21 +- .../runtime/rpc/taskexecutor/SlotReport.java | 38 +++ .../rpc/taskexecutor/TaskExecutor.java | 169 +++++++--- .../rpc/taskexecutor/TaskExecutorGateway.java | 29 +- .../TaskExecutorRegistrationSuccess.java | 75 +++++ ...skExecutorToResourceManagerConnection.java | 194 ++++++++++++ .../runtime/rpc/akka/AkkaRpcServiceTest.java | 51 ++- .../rpc/taskexecutor/TaskExecutorTest.java | 87 +----- 18 files changed, 1105 insertions(+), 175 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RegistrationResponse.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RetryingRegistration.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotReport.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorRegistrationSuccess.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorToResourceManagerConnection.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java new file mode 100644 index 0000000000000..094d36f0c6d9c --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java @@ -0,0 +1,39 @@ +/* + * 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.highavailability; + +import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; + +/** + * This class gives access to all services needed for + * + *
    + *
  • ResourceManager leader election and leader retrieval
  • + *
  • JobManager leader election and leader retrieval
  • + *
  • Persistence for checkpoint metadata
  • + *
  • Registering the latest completed checkpoint(s)
  • + *
+ */ +public interface HighAvailabilityServices { + + /** + * Gets the leader retriever for the cluster's resource manager. + */ + LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception; +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java new file mode 100644 index 0000000000000..b8c2ed89adff5 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java @@ -0,0 +1,59 @@ +/* + * 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.highavailability; + +import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; +import org.apache.flink.runtime.leaderretrieval.StandaloneLeaderRetrievalService; + +import java.util.UUID; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * An implementation of the {@link HighAvailabilityServices} for the non-high-availability case. + * This implementation can be used for testing, and for cluster setups that do not + * tolerate failures of the master processes (JobManager, ResourceManager). + * + *

This implementation has no dependencies on any external services. It returns fix + * pre-configured leaders, and stores checkpoints and metadata simply on the heap and therefore + * in volatile memory. + */ +public class NonHaServices implements HighAvailabilityServices { + + /** The fix address of the ResourceManager */ + private final String resourceManagerAddress; + + /** + * Creates a new services class for the fix pre-defined leaders. + * + * @param resourceManagerAddress The fix address of the ResourceManager + */ + public NonHaServices(String resourceManagerAddress) { + this.resourceManagerAddress = checkNotNull(resourceManagerAddress); + } + + // ------------------------------------------------------------------------ + // Services + // ------------------------------------------------------------------------ + + @Override + public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception { + return new StandaloneLeaderRetrievalService(resourceManagerAddress, new UUID(0, 0)); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/StandaloneLeaderRetrievalService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/StandaloneLeaderRetrievalService.java index 26a34aa1fa737..16b163cf0b238 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/StandaloneLeaderRetrievalService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/StandaloneLeaderRetrievalService.java @@ -18,44 +18,74 @@ package org.apache.flink.runtime.leaderretrieval; -import org.apache.flink.util.Preconditions; +import java.util.UUID; + +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; /** - * Standalone implementation of the {@link LeaderRetrievalService}. The standalone implementation - * assumes that there is only a single {@link org.apache.flink.runtime.jobmanager.JobManager} whose - * address is given to the service when creating it. This address is directly given to the - * {@link LeaderRetrievalListener} when the service is started. + * Standalone implementation of the {@link LeaderRetrievalService}. This implementation + * assumes that there is only a single contender for leadership + * (e.g., a single JobManager or ResourceManager process) and that this process is + * reachable under a constant address. + * + *

As soon as this service is started, it immediately notifies the leader listener + * of the leader contender with the pre-configured address. */ public class StandaloneLeaderRetrievalService implements LeaderRetrievalService { - /** Address of the only JobManager */ - private final String jobManagerAddress; + private final Object startStopLock = new Object(); + + /** The fix address of the leader */ + private final String leaderAddress; + + /** The fix leader ID (leader lock fencing token) */ + private final UUID leaderId; - /** Listener which wants to be notified about the new leader */ - private LeaderRetrievalListener leaderListener; + /** Flag whether this service is started */ + private boolean started; /** - * Creates a StandaloneLeaderRetrievalService with the given JobManager address. + * Creates a StandaloneLeaderRetrievalService with the given leader address. + * The leaderId will be null. * - * @param jobManagerAddress The JobManager's address which is returned to the - * {@link LeaderRetrievalListener} + * @param leaderAddress The leader's pre-configured address */ - public StandaloneLeaderRetrievalService(String jobManagerAddress) { - this.jobManagerAddress = jobManagerAddress; + public StandaloneLeaderRetrievalService(String leaderAddress) { + this.leaderAddress = checkNotNull(leaderAddress); + this.leaderId = null; } + /** + * Creates a StandaloneLeaderRetrievalService with the given leader address. + * + * @param leaderAddress The leader's pre-configured address + * @param leaderId The constant leaderId. + */ + public StandaloneLeaderRetrievalService(String leaderAddress, UUID leaderId) { + this.leaderAddress = checkNotNull(leaderAddress); + this.leaderId = checkNotNull(leaderId); + } + + // ------------------------------------------------------------------------ + @Override public void start(LeaderRetrievalListener listener) { - Preconditions.checkNotNull(listener, "Listener must not be null."); - Preconditions.checkState(leaderListener == null, "StandaloneLeaderRetrievalService can " + - "only be started once."); + checkNotNull(listener, "Listener must not be null."); - leaderListener = listener; + synchronized (startStopLock) { + checkState(!started, "StandaloneLeaderRetrievalService can only be started once."); + started = true; - // directly notify the listener, because we already know the leading JobManager's address - leaderListener.notifyLeaderAddress(jobManagerAddress, null); + // directly notify the listener, because we already know the leading JobManager's address + listener.notifyLeaderAddress(leaderAddress, leaderId); + } } @Override - public void stop() {} + public void stop() { + synchronized (startStopLock) { + started = false; + } + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java index 67ac182dbf5af..a28bc1437c020 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java @@ -237,7 +237,6 @@ public Future callAsync(Callable callable, Timeout timeout) { * } */ public void validateRunsInMainThread() { - // because the initialization is lazy, it can be that certain methods are assert currentMainThread.get() == Thread.currentThread(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java index f93be8361c077..fabdb057514c3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java @@ -18,8 +18,11 @@ package org.apache.flink.runtime.rpc; +import scala.concurrent.ExecutionContext; import scala.concurrent.Future; +import java.util.concurrent.TimeUnit; + /** * Interface for rpc services. An rpc service is used to start and connect to a {@link RpcEndpoint}. * Connecting to a rpc server will return a {@link RpcGateway} which can be used to call remote @@ -71,4 +74,28 @@ public interface RpcService { * @return Fully qualified address */ String getAddress(C selfGateway); + + /** + * Gets the execution context, provided by this RPC service. This execution + * context can be used for example for the {@code onComplete(...)} or {@code onSuccess(...)} + * methods of Futures. + * + *

IMPORTANT: This execution context does not isolate the method invocations against + * any concurrent invocations and is therefore not suitable to run completion methods of futures + * that modify state of an {@link RpcEndpoint}. For such operations, one needs to use the + * {@link RpcEndpoint#getMainThreadExecutionContext() MainThreadExecutionContext} of that + * {@code RpcEndpoint}. + * + * @return The execution context provided by the RPC service + */ + ExecutionContext getExecutionContext(); + + /** + * Execute the runnable in the execution context of this RPC Service, as returned by + * {@link #getExecutionContext()}, after a scheduled delay. + * + * @param runnable Runnable to be executed + * @param delay The delay after which the runnable will be executed + */ + void scheduleRunnable(Runnable runnable, long delay, TimeUnit unit); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java index 7b3352401283d..b647bbd2bba90 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java @@ -38,14 +38,18 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import scala.concurrent.ExecutionContext; import scala.concurrent.Future; +import scala.concurrent.duration.FiniteDuration; import javax.annotation.concurrent.ThreadSafe; import java.lang.reflect.InvocationHandler; import java.lang.reflect.Proxy; import java.util.HashSet; import java.util.Set; +import java.util.concurrent.TimeUnit; +import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.apache.flink.util.Preconditions.checkState; @@ -199,4 +203,18 @@ public String getAddress(RpcGateway selfGateway) { throw new IllegalArgumentException("Cannot get address for non " + className + '.'); } } + + @Override + public ExecutionContext getExecutionContext() { + return actorSystem.dispatcher(); + } + + @Override + public void scheduleRunnable(Runnable runnable, long delay, TimeUnit unit) { + checkNotNull(runnable, "runnable"); + checkNotNull(unit, "unit"); + checkArgument(delay >= 0, "delay must be zero or larger"); + + actorSystem.scheduler().scheduleOnce(new FiniteDuration(delay, unit), runnable, getExecutionContext()); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunAsync.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunAsync.java index c18906ca21637..ce4f9d6ca9065 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunAsync.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunAsync.java @@ -36,6 +36,7 @@ public final class RunAsync implements Serializable { private final long delay; /** + * Creates a new {@code RunAsync} message. * * @param runnable The Runnable to run. * @param delay The delay in milliseconds. Zero indicates immediate execution. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RegistrationResponse.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RegistrationResponse.java new file mode 100644 index 0000000000000..2de560a25bce9 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RegistrationResponse.java @@ -0,0 +1,84 @@ +/* + * 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.rpc.registration; + +import java.io.Serializable; + +/** + * Base class for responses given to registration attempts from {@link RetryingRegistration}. + */ +public abstract class RegistrationResponse implements Serializable { + + private static final long serialVersionUID = 1L; + + // ---------------------------------------------------------------------------- + + /** + * Base class for a successful registration. Concrete registration implementations + * will typically extend this class to attach more information. + */ + public static class Success extends RegistrationResponse { + private static final long serialVersionUID = 1L; + + @Override + public String toString() { + return "Registration Successful"; + } + } + + // ---------------------------------------------------------------------------- + + /** + * A rejected (declined) registration. + */ + public static final class Decline extends RegistrationResponse { + private static final long serialVersionUID = 1L; + + /** the rejection reason */ + private final String reason; + + /** + * Creates a new rejection message. + * + * @param reason The reason for the rejection. + */ + public Decline(String reason) { + this.reason = reason != null ? reason : "(unknown)"; + } + + /** + * Gets the reason for the rejection. + */ + public String getReason() { + return reason; + } + + @Override + public String toString() { + return "Registration Declined (" + reason + ')'; + } + } +} + + + + + + + diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RetryingRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RetryingRegistration.java new file mode 100644 index 0000000000000..4c93684b9fde4 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RetryingRegistration.java @@ -0,0 +1,292 @@ +/* + * 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.rpc.registration; + +import akka.dispatch.OnFailure; +import akka.dispatch.OnSuccess; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.rpc.RpcGateway; +import org.apache.flink.runtime.rpc.RpcService; + +import org.slf4j.Logger; + +import scala.concurrent.Future; +import scala.concurrent.Promise; +import scala.concurrent.impl.Promise.DefaultPromise; + +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + + +/** + * This utility class implements the basis of registering one component at another component, + * for example registering the TaskExecutor at the ResourceManager. + * This {@code RetryingRegistration} implements both the initial address resolution + * and the retries-with-backoff strategy. + * + *

The registration gives access to a future that is completed upon successful registration. + * The registration can be canceled, for example when the target where it tries to register + * at looses leader status. + * + * @param The type of the gateway to connect to. + * @param The type of the successful registration responses. + */ +public abstract class RetryingRegistration { + + // ------------------------------------------------------------------------ + // default configuration values + // ------------------------------------------------------------------------ + + private static final long INITIAL_REGISTRATION_TIMEOUT_MILLIS = 100; + + private static final long MAX_REGISTRATION_TIMEOUT_MILLIS = 30000; + + private static final long ERROR_REGISTRATION_DELAY_MILLIS = 10000; + + private static final long REFUSED_REGISTRATION_DELAY_MILLIS = 30000; + + // ------------------------------------------------------------------------ + // Fields + // ------------------------------------------------------------------------ + + private final Logger log; + + private final RpcService rpcService; + + private final String targetName; + + private final Class targetType; + + private final String targetAddress; + + private final UUID leaderId; + + private final Promise> completionPromise; + + private final long initialRegistrationTimeout; + + private final long maxRegistrationTimeout; + + private final long delayOnError; + + private final long delayOnRefusedRegistration; + + private volatile boolean canceled; + + // ------------------------------------------------------------------------ + + public RetryingRegistration( + Logger log, + RpcService rpcService, + String targetName, + Class targetType, + String targetAddress, + UUID leaderId) { + this(log, rpcService, targetName, targetType, targetAddress, leaderId, + INITIAL_REGISTRATION_TIMEOUT_MILLIS, MAX_REGISTRATION_TIMEOUT_MILLIS, + ERROR_REGISTRATION_DELAY_MILLIS, REFUSED_REGISTRATION_DELAY_MILLIS); + } + + public RetryingRegistration( + Logger log, + RpcService rpcService, + String targetName, + Class targetType, + String targetAddress, + UUID leaderId, + long initialRegistrationTimeout, + long maxRegistrationTimeout, + long delayOnError, + long delayOnRefusedRegistration) { + + checkArgument(initialRegistrationTimeout > 0, "initial registration timeout must be greater than zero"); + checkArgument(maxRegistrationTimeout > 0, "maximum registration timeout must be greater than zero"); + checkArgument(delayOnError >= 0, "delay on error must be non-negative"); + checkArgument(delayOnRefusedRegistration >= 0, "delay on refused registration must be non-negative"); + + this.log = checkNotNull(log); + this.rpcService = checkNotNull(rpcService); + this.targetName = checkNotNull(targetName); + this.targetType = checkNotNull(targetType); + this.targetAddress = checkNotNull(targetAddress); + this.leaderId = checkNotNull(leaderId); + this.initialRegistrationTimeout = initialRegistrationTimeout; + this.maxRegistrationTimeout = maxRegistrationTimeout; + this.delayOnError = delayOnError; + this.delayOnRefusedRegistration = delayOnRefusedRegistration; + + this.completionPromise = new DefaultPromise<>(); + } + + // ------------------------------------------------------------------------ + // completion and cancellation + // ------------------------------------------------------------------------ + + public Future> getFuture() { + return completionPromise.future(); + } + + /** + * Cancels the registration procedure. + */ + public void cancel() { + canceled = true; + } + + /** + * Checks if the registration was canceled. + * @return True if the registration was canceled, false otherwise. + */ + public boolean isCanceled() { + return canceled; + } + + // ------------------------------------------------------------------------ + // registration + // ------------------------------------------------------------------------ + + protected abstract Future invokeRegistration( + Gateway gateway, UUID leaderId, long timeoutMillis) throws Exception; + + /** + * This method resolves the target address to a callable gateway and starts the + * registration after that. + */ + @SuppressWarnings("unchecked") + public void startRegistration() { + try { + // trigger resolution of the resource manager address to a callable gateway + Future resourceManagerFuture = rpcService.connect(targetAddress, targetType); + + // upon success, start the registration attempts + resourceManagerFuture.onSuccess(new OnSuccess() { + @Override + public void onSuccess(Gateway result) { + log.info("Resolved {} address, beginning registration", targetName); + register(result, 1, initialRegistrationTimeout); + } + }, rpcService.getExecutionContext()); + + // upon failure, retry, unless this is cancelled + resourceManagerFuture.onFailure(new OnFailure() { + @Override + public void onFailure(Throwable failure) { + if (!isCanceled()) { + log.warn("Could not resolve {} address {}, retrying...", targetName, targetAddress); + startRegistration(); + } + } + }, rpcService.getExecutionContext()); + } + catch (Throwable t) { + cancel(); + completionPromise.tryFailure(t); + } + } + + /** + * This method performs a registration attempt and triggers either a success notification or a retry, + * depending on the result. + */ + @SuppressWarnings("unchecked") + private void register(final Gateway gateway, final int attempt, final long timeoutMillis) { + // eager check for canceling to avoid some unnecessary work + if (canceled) { + return; + } + + try { + log.info("Registration at {} attempt {} (timeout={}ms)", targetName, attempt, timeoutMillis); + Future registrationFuture = invokeRegistration(gateway, leaderId, timeoutMillis); + + // if the registration was successful, let the TaskExecutor know + registrationFuture.onSuccess(new OnSuccess() { + + @Override + public void onSuccess(RegistrationResponse result) throws Throwable { + if (!isCanceled()) { + if (result instanceof RegistrationResponse.Success) { + // registration successful! + Success success = (Success) result; + completionPromise.success(new Tuple2<>(gateway, success)); + } + else { + // registration refused or unknown + if (result instanceof RegistrationResponse.Decline) { + RegistrationResponse.Decline decline = (RegistrationResponse.Decline) result; + log.info("Registration at {} was declined: {}", targetName, decline.getReason()); + } else { + log.error("Received unknown response to registration attempt: " + result); + } + + log.info("Pausing and re-attempting registration in {} ms", delayOnRefusedRegistration); + registerLater(gateway, 1, initialRegistrationTimeout, delayOnRefusedRegistration); + } + } + } + }, rpcService.getExecutionContext()); + + // upon failure, retry + registrationFuture.onFailure(new OnFailure() { + @Override + public void onFailure(Throwable failure) { + if (!isCanceled()) { + if (failure instanceof TimeoutException) { + // we simply have not received a response in time. maybe the timeout was + // very low (initial fast registration attempts), maybe the target endpoint is + // currently down. + if (log.isDebugEnabled()) { + log.debug("Registration at {} ({}) attempt {} timed out after {} ms", + targetName, targetAddress, attempt, timeoutMillis); + } + + long newTimeoutMillis = Math.min(2 * timeoutMillis, maxRegistrationTimeout); + register(gateway, attempt + 1, newTimeoutMillis); + } + else { + // a serious failure occurred. we still should not give up, but keep trying + log.error("Registration at " + targetName + " failed due to an error", failure); + log.info("Pausing and re-attempting registration in {} ms", delayOnError); + + registerLater(gateway, 1, initialRegistrationTimeout, delayOnError); + } + } + } + }, rpcService.getExecutionContext()); + } + catch (Throwable t) { + cancel(); + completionPromise.tryFailure(t); + } + } + + private void registerLater(final Gateway gateway, final int attempt, final long timeoutMillis, long delay) { + rpcService.scheduleRunnable(new Runnable() { + @Override + public void run() { + register(gateway, attempt, timeoutMillis); + } + }, delay, TimeUnit.MILLISECONDS); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java index 729ef0c5f41e8..6f34465740760 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java @@ -19,19 +19,24 @@ package org.apache.flink.runtime.rpc.resourcemanager; import akka.dispatch.Mapper; + +import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.rpc.RpcMethod; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.jobmaster.JobMaster; import org.apache.flink.runtime.rpc.jobmaster.JobMasterGateway; +import org.apache.flink.runtime.rpc.taskexecutor.TaskExecutorRegistrationSuccess; import org.apache.flink.util.Preconditions; + import scala.concurrent.ExecutionContext; import scala.concurrent.ExecutionContext$; import scala.concurrent.Future; import java.util.HashMap; import java.util.Map; +import java.util.UUID; import java.util.concurrent.ExecutorService; /** @@ -93,4 +98,22 @@ public SlotAssignment requestSlot(SlotRequest slotRequest) { System.out.println("SlotRequest: " + slotRequest); return new SlotAssignment(); } + + + /** + * + * @param resourceManagerLeaderId The fencing token for the ResourceManager leader + * @param taskExecutorAddress The address of the TaskExecutor that registers + * @param resourceID The resource ID of the TaskExecutor that registers + * + * @return The response by the ResourceManager. + */ + @RpcMethod + public org.apache.flink.runtime.rpc.registration.RegistrationResponse registerTaskExecutor( + UUID resourceManagerLeaderId, + String taskExecutorAddress, + ResourceID resourceID) { + + return new TaskExecutorRegistrationSuccess(new InstanceID(), 5000); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerGateway.java index 464a2618ea7e3..afddb01d6b128 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerGateway.java @@ -18,14 +18,18 @@ package org.apache.flink.runtime.rpc.resourcemanager; +import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcTimeout; import org.apache.flink.runtime.rpc.jobmaster.JobMaster; + import scala.concurrent.Future; import scala.concurrent.duration.FiniteDuration; +import java.util.UUID; + /** - * {@link ResourceManager} rpc gateway interface. + * The {@link ResourceManager}'s RPC gateway interface. */ public interface ResourceManagerGateway extends RpcGateway { @@ -55,4 +59,19 @@ Future registerJobMaster( * @return Future slot assignment */ Future requestSlot(SlotRequest slotRequest); + + /** + * + * @param resourceManagerLeaderId The fencing token for the ResourceManager leader + * @param taskExecutorAddress The address of the TaskExecutor that registers + * @param resourceID The resource ID of the TaskExecutor that registers + * @param timeout The timeout for the response. + * + * @return The future to the response by the ResourceManager. + */ + Future registerTaskExecutor( + UUID resourceManagerLeaderId, + String taskExecutorAddress, + ResourceID resourceID, + @RpcTimeout FiniteDuration timeout); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotReport.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotReport.java new file mode 100644 index 0000000000000..e42fa4a2359b0 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotReport.java @@ -0,0 +1,38 @@ +/* + * 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.rpc.taskexecutor; + +import java.io.Serializable; + +/** + * A report about the current status of all slots of the TaskExecutor, describing + * which slots are available and allocated, and what jobs (JobManagers) the allocated slots + * have been allocated to. + */ +public class SlotReport implements Serializable{ + + private static final long serialVersionUID = 1L; + + // ------------------------------------------------------------------------ + + @Override + public String toString() { + return "SlotReport"; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java index 3a7dd9f0ce558..1a637bb05f4f9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java @@ -18,67 +18,152 @@ package org.apache.flink.runtime.rpc.taskexecutor; -import akka.dispatch.ExecutionContexts$; -import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; -import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; -import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.rpc.RpcMethod; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener; import org.apache.flink.runtime.rpc.RpcEndpoint; +import org.apache.flink.runtime.rpc.RpcMethod; import org.apache.flink.runtime.rpc.RpcService; -import org.apache.flink.util.Preconditions; -import scala.concurrent.ExecutionContext; -import java.util.HashSet; -import java.util.Set; -import java.util.concurrent.ExecutorService; +import java.util.UUID; + +import static org.apache.flink.util.Preconditions.checkNotNull; /** * TaskExecutor implementation. The task executor is responsible for the execution of multiple * {@link org.apache.flink.runtime.taskmanager.Task}. - * - * It offers the following methods as part of its rpc interface to interact with him remotely: - *

    - *
  • {@link #executeTask(TaskDeploymentDescriptor)} executes a given task on the TaskExecutor
  • - *
  • {@link #cancelTask(ExecutionAttemptID)} cancels a given task identified by the {@link ExecutionAttemptID}
  • - *
*/ public class TaskExecutor extends RpcEndpoint { - private final ExecutionContext executionContext; - private final Set tasks = new HashSet<>(); - public TaskExecutor(RpcService rpcService, ExecutorService executorService) { + /** The unique resource ID of this TaskExecutor */ + private final ResourceID resourceID; + + /** The access to the leader election and metadata storage services */ + private final HighAvailabilityServices haServices; + + // --------- resource manager -------- + + private TaskExecutorToResourceManagerConnection resourceManagerConnection; + + // ------------------------------------------------------------------------ + + public TaskExecutor( + RpcService rpcService, + HighAvailabilityServices haServices, + ResourceID resourceID) { + super(rpcService); - this.executionContext = ExecutionContexts$.MODULE$.fromExecutor( - Preconditions.checkNotNull(executorService)); + + this.haServices = checkNotNull(haServices); + this.resourceID = checkNotNull(resourceID); + } + + // ------------------------------------------------------------------------ + // Properties + // ------------------------------------------------------------------------ + + public ResourceID getResourceID() { + return resourceID; + } + + // ------------------------------------------------------------------------ + // Life cycle + // ------------------------------------------------------------------------ + + @Override + public void start() { + // start by connecting to the ResourceManager + try { + haServices.getResourceManagerLeaderRetriever().start(new ResourceManagerLeaderListener()); + } catch (Exception e) { + onFatalErrorAsync(e); + } + } + + + // ------------------------------------------------------------------------ + // RPC methods - ResourceManager related + // ------------------------------------------------------------------------ + + @RpcMethod + public void notifyOfNewResourceManagerLeader(String newLeaderAddress, UUID newLeaderId) { + if (resourceManagerConnection != null) { + if (newLeaderAddress != null) { + // the resource manager switched to a new leader + log.info("ResourceManager leader changed from {} to {}. Registering at new leader.", + resourceManagerConnection.getResourceManagerAddress(), newLeaderAddress); + } + else { + // address null means that the current leader is lost without a new leader being there, yet + log.info("Current ResourceManager {} lost leader status. Waiting for new ResourceManager leader.", + resourceManagerConnection.getResourceManagerAddress()); + } + + // drop the current connection or connection attempt + if (resourceManagerConnection != null) { + resourceManagerConnection.close(); + resourceManagerConnection = null; + } + } + + // establish a connection to the new leader + if (newLeaderAddress != null) { + log.info("Attempting to register at ResourceManager {}", newLeaderAddress); + resourceManagerConnection = + new TaskExecutorToResourceManagerConnection(log, this, newLeaderAddress, newLeaderId); + resourceManagerConnection.start(); + } } + // ------------------------------------------------------------------------ + // Error handling + // ------------------------------------------------------------------------ + /** - * Execute the given task on the task executor. The task is described by the provided - * {@link TaskDeploymentDescriptor}. - * - * @param taskDeploymentDescriptor Descriptor for the task to be executed - * @return Acknowledge the start of the task execution + * Notifies the TaskExecutor that a fatal error has occurred and it cannot proceed. + * This method should be used when asynchronous threads want to notify the + * TaskExecutor of a fatal error. + * + * @param t The exception describing the fatal error */ - @RpcMethod - public Acknowledge executeTask(TaskDeploymentDescriptor taskDeploymentDescriptor) { - tasks.add(taskDeploymentDescriptor.getExecutionId()); - return Acknowledge.get(); + void onFatalErrorAsync(final Throwable t) { + runAsync(new Runnable() { + @Override + public void run() { + onFatalError(t); + } + }); } /** - * Cancel a task identified by it {@link ExecutionAttemptID}. If the task cannot be found, then - * the method throws an {@link Exception}. - * - * @param executionAttemptId Execution attempt ID identifying the task to be canceled. - * @return Acknowledge the task canceling - * @throws Exception if the task with the given execution attempt id could not be found + * Notifies the TaskExecutor that a fatal error has occurred and it cannot proceed. + * This method must only be called from within the TaskExecutor's main thread. + * + * @param t The exception describing the fatal error */ - @RpcMethod - public Acknowledge cancelTask(ExecutionAttemptID executionAttemptId) throws Exception { - if (tasks.contains(executionAttemptId)) { - return Acknowledge.get(); - } else { - throw new Exception("Could not find task."); + void onFatalError(Throwable t) { + // to be determined, probably delegate to a fatal error handler that + // would either log (mini cluster) ot kill the process (yarn, mesos, ...) + log.error("FATAL ERROR", t); + } + + // ------------------------------------------------------------------------ + // Utility classes + // ------------------------------------------------------------------------ + + /** + * The listener for leader changes of the resource manager + */ + private class ResourceManagerLeaderListener implements LeaderRetrievalListener { + + @Override + public void notifyLeaderAddress(String leaderAddress, UUID leaderSessionID) { + getSelf().notifyOfNewResourceManagerLeader(leaderAddress, leaderSessionID); + } + + @Override + public void handleError(Exception exception) { + onFatalErrorAsync(exception); } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorGateway.java index 450423e8111ff..b0b21b00eb2c3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorGateway.java @@ -18,31 +18,18 @@ package org.apache.flink.runtime.rpc.taskexecutor; -import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; -import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; -import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.rpc.RpcGateway; -import scala.concurrent.Future; + +import java.util.UUID; /** - * {@link TaskExecutor} rpc gateway interface + * {@link TaskExecutor} RPC gateway interface */ public interface TaskExecutorGateway extends RpcGateway { - /** - * Execute the given task on the task executor. The task is described by the provided - * {@link TaskDeploymentDescriptor}. - * - * @param taskDeploymentDescriptor Descriptor for the task to be executed - * @return Future acknowledge of the start of the task execution - */ - Future executeTask(TaskDeploymentDescriptor taskDeploymentDescriptor); - /** - * Cancel a task identified by it {@link ExecutionAttemptID}. If the task cannot be found, then - * the method throws an {@link Exception}. - * - * @param executionAttemptId Execution attempt ID identifying the task to be canceled. - * @return Future acknowledge of the task canceling - */ - Future cancelTask(ExecutionAttemptID executionAttemptId); + // ------------------------------------------------------------------------ + // ResourceManager handlers + // ------------------------------------------------------------------------ + + void notifyOfNewResourceManagerLeader(String address, UUID resourceManagerLeaderId); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorRegistrationSuccess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorRegistrationSuccess.java new file mode 100644 index 0000000000000..641102dda817a --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorRegistrationSuccess.java @@ -0,0 +1,75 @@ +/* + * 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.rpc.taskexecutor; + +import org.apache.flink.runtime.instance.InstanceID; +import org.apache.flink.runtime.rpc.registration.RegistrationResponse; + +import java.io.Serializable; + +/** + * Base class for responses from the ResourceManager to a registration attempt by a + * TaskExecutor. + */ +public final class TaskExecutorRegistrationSuccess extends RegistrationResponse.Success implements Serializable { + + private static final long serialVersionUID = 1L; + + private final InstanceID registrationId; + + private final long heartbeatInterval; + + /** + * Create a new {@code TaskExecutorRegistrationSuccess} message. + * + * @param registrationId The ID that the ResourceManager assigned the registration. + * @param heartbeatInterval The interval in which the ResourceManager will heartbeat the TaskExecutor. + */ + public TaskExecutorRegistrationSuccess(InstanceID registrationId, long heartbeatInterval) { + this.registrationId = registrationId; + this.heartbeatInterval = heartbeatInterval; + } + + /** + * Gets the ID that the ResourceManager assigned the registration. + */ + public InstanceID getRegistrationId() { + return registrationId; + } + + /** + * Gets the interval in which the ResourceManager will heartbeat the TaskExecutor. + */ + public long getHeartbeatInterval() { + return heartbeatInterval; + } + + @Override + public String toString() { + return "TaskExecutorRegistrationSuccess (" + registrationId + " / " + heartbeatInterval + ')'; + } + +} + + + + + + + diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorToResourceManagerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorToResourceManagerConnection.java new file mode 100644 index 0000000000000..ef75862b18679 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorToResourceManagerConnection.java @@ -0,0 +1,194 @@ +/* + * 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.rpc.taskexecutor; + +import akka.dispatch.OnFailure; +import akka.dispatch.OnSuccess; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.instance.InstanceID; +import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.rpc.registration.RegistrationResponse; +import org.apache.flink.runtime.rpc.registration.RetryingRegistration; +import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway; + +import org.slf4j.Logger; + +import scala.concurrent.Future; +import scala.concurrent.duration.FiniteDuration; + +import java.util.UUID; +import java.util.concurrent.TimeUnit; + +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + +public class TaskExecutorToResourceManagerConnection { + + /** the logger for all log messages of this class */ + private final Logger log; + + /** the TaskExecutor whose connection to the ResourceManager this represents */ + private final TaskExecutor taskExecutor; + + private final UUID resourceManagerLeaderId; + + private final String resourceManagerAddress; + + private ResourceManagerRegistration pendingRegistration; + + private ResourceManagerGateway registeredResourceManager; + + private InstanceID registrationId; + + /** flag indicating that the connection is closed */ + private volatile boolean closed; + + + public TaskExecutorToResourceManagerConnection( + Logger log, + TaskExecutor taskExecutor, + String resourceManagerAddress, + UUID resourceManagerLeaderId) { + + this.log = checkNotNull(log); + this.taskExecutor = checkNotNull(taskExecutor); + this.resourceManagerAddress = checkNotNull(resourceManagerAddress); + this.resourceManagerLeaderId = checkNotNull(resourceManagerLeaderId); + } + + // ------------------------------------------------------------------------ + // Life cycle + // ------------------------------------------------------------------------ + + @SuppressWarnings("unchecked") + public void start() { + checkState(!closed, "The connection is already closed"); + checkState(!isRegistered() && pendingRegistration == null, "The connection is already started"); + + ResourceManagerRegistration registration = new ResourceManagerRegistration( + log, taskExecutor.getRpcService(), + resourceManagerAddress, resourceManagerLeaderId, + taskExecutor.getAddress(), taskExecutor.getResourceID()); + + Future> future = registration.getFuture(); + + future.onSuccess(new OnSuccess>() { + @Override + public void onSuccess(Tuple2 result) { + registeredResourceManager = result.f0; + registrationId = result.f1.getRegistrationId(); + } + }, taskExecutor.getMainThreadExecutionContext()); + + // this future should only ever fail if there is a bug, not if the registration is declined + future.onFailure(new OnFailure() { + @Override + public void onFailure(Throwable failure) { + taskExecutor.onFatalError(failure); + } + }, taskExecutor.getMainThreadExecutionContext()); + } + + public void close() { + closed = true; + + // make sure we do not keep re-trying forever + if (pendingRegistration != null) { + pendingRegistration.cancel(); + } + } + + public boolean isClosed() { + return closed; + } + + // ------------------------------------------------------------------------ + // Properties + // ------------------------------------------------------------------------ + + public UUID getResourceManagerLeaderId() { + return resourceManagerLeaderId; + } + + public String getResourceManagerAddress() { + return resourceManagerAddress; + } + + /** + * Gets the ResourceManagerGateway. This returns null until the registration is completed. + */ + public ResourceManagerGateway getResourceManager() { + return registeredResourceManager; + } + + /** + * Gets the ID under which the TaskExecutor is registered at the ResourceManager. + * This returns null until the registration is completed. + */ + public InstanceID getRegistrationId() { + return registrationId; + } + + public boolean isRegistered() { + return registeredResourceManager != null; + } + + // ------------------------------------------------------------------------ + + @Override + public String toString() { + return String.format("Connection to ResourceManager %s (leaderId=%s)", + resourceManagerAddress, resourceManagerLeaderId); + } + + // ------------------------------------------------------------------------ + // Utilities + // ------------------------------------------------------------------------ + + static class ResourceManagerRegistration + extends RetryingRegistration { + + private final String taskExecutorAddress; + + private final ResourceID resourceID; + + public ResourceManagerRegistration( + Logger log, + RpcService rpcService, + String targetAddress, + UUID leaderId, + String taskExecutorAddress, + ResourceID resourceID) { + + super(log, rpcService, "ResourceManager", ResourceManagerGateway.class, targetAddress, leaderId); + this.taskExecutorAddress = checkNotNull(taskExecutorAddress); + this.resourceID = checkNotNull(resourceID); + } + + @Override + protected Future invokeRegistration( + ResourceManagerGateway resourceManager, UUID leaderId, long timeoutMillis) throws Exception { + + FiniteDuration timeout = new FiniteDuration(timeoutMillis, TimeUnit.MILLISECONDS); + return resourceManager.registerTaskExecutor(leaderId, taskExecutorAddress, resourceID, timeout); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java index fd55904d20de8..7b4ab89a84c24 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java @@ -20,15 +20,17 @@ import akka.actor.ActorSystem; import akka.util.Timeout; + +import org.apache.flink.core.testutils.OneShotLatch; import org.apache.flink.runtime.akka.AkkaUtils; -import org.apache.flink.runtime.rpc.RpcEndpoint; -import org.apache.flink.runtime.rpc.RpcGateway; -import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.jobmaster.JobMaster; import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.rpc.resourcemanager.ResourceManager; import org.apache.flink.util.TestLogger; + +import org.junit.AfterClass; import org.junit.Test; + import scala.concurrent.duration.Deadline; import scala.concurrent.duration.FiniteDuration; @@ -41,6 +43,49 @@ public class AkkaRpcServiceTest extends TestLogger { + // ------------------------------------------------------------------------ + // shared test members + // ------------------------------------------------------------------------ + + private static ActorSystem actorSystem = AkkaUtils.createDefaultActorSystem(); + + private static AkkaRpcService akkaRpcService = + new AkkaRpcService(actorSystem, new Timeout(10000, TimeUnit.MILLISECONDS)); + + @AfterClass + public static void shutdown() { + akkaRpcService.stopService(); + actorSystem.shutdown(); + } + + // ------------------------------------------------------------------------ + // tests + // ------------------------------------------------------------------------ + + @Test + public void testScheduleRunnable() throws Exception { + final OneShotLatch latch = new OneShotLatch(); + final long delay = 100; + final long start = System.nanoTime(); + + akkaRpcService.scheduleRunnable(new Runnable() { + @Override + public void run() { + latch.trigger(); + } + }, delay, TimeUnit.MILLISECONDS); + + latch.await(); + final long stop = System.nanoTime(); + + assertTrue("call was not properly delayed", ((stop - start) / 1000000) >= delay); + } + + // ------------------------------------------------------------------------ + // specific component tests - should be moved to the test classes + // for those components + // ------------------------------------------------------------------------ + /** * Tests that the {@link JobMaster} can connect to the {@link ResourceManager} using the * {@link AkkaRpcService}. diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java index c96f4f6cdea9f..9f9bab372f5bb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java @@ -18,93 +18,8 @@ package org.apache.flink.runtime.rpc.taskexecutor; -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.JobID; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.blob.BlobKey; -import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; -import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; -import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; -import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; -import org.apache.flink.runtime.jobgraph.JobVertexID; -import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.rpc.MainThreadExecutor; -import org.apache.flink.runtime.rpc.RpcEndpoint; -import org.apache.flink.runtime.rpc.RpcGateway; -import org.apache.flink.runtime.rpc.RpcService; -import org.apache.flink.runtime.rpc.StartStoppable; -import org.apache.flink.runtime.util.DirectExecutorService; -import org.apache.flink.util.SerializedValue; import org.apache.flink.util.TestLogger; -import org.junit.Test; -import org.mockito.Matchers; -import org.mockito.cglib.proxy.InvocationHandler; -import org.mockito.cglib.proxy.Proxy; -import scala.concurrent.Future; - -import java.net.URL; -import java.util.Collections; - -import static org.junit.Assert.fail; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; public class TaskExecutorTest extends TestLogger { - - /** - * Tests that we can deploy and cancel a task on the TaskExecutor without exceptions - */ - @Test - public void testTaskExecution() throws Exception { - RpcService testingRpcService = mock(RpcService.class); - InvocationHandler invocationHandler = mock(InvocationHandler.class); - Object selfGateway = Proxy.newProxyInstance(ClassLoader.getSystemClassLoader(), new Class[] {TaskExecutorGateway.class, MainThreadExecutor.class, StartStoppable.class}, invocationHandler); - when(testingRpcService.startServer(Matchers.any(RpcEndpoint.class))).thenReturn((RpcGateway)selfGateway); - - DirectExecutorService directExecutorService = new DirectExecutorService(); - TaskExecutor taskExecutor = new TaskExecutor(testingRpcService, directExecutorService); - taskExecutor.start(); - - TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor( - new JobID(), - "Test job", - new JobVertexID(), - new ExecutionAttemptID(), - new SerializedValue(null), - "Test task", - 0, - 1, - 0, - new Configuration(), - new Configuration(), - "Invokable", - Collections.emptyList(), - Collections.emptyList(), - Collections.emptyList(), - Collections.emptyList(), - 0 - ); - - Acknowledge ack = taskExecutor.executeTask(tdd); - - ack = taskExecutor.cancelTask(tdd.getExecutionId()); - } - - /** - * Tests that cancelling a non-existing task will return an exception - */ - @Test(expected=Exception.class) - public void testWrongTaskCancellation() throws Exception { - RpcService testingRpcService = mock(RpcService.class); - InvocationHandler invocationHandler = mock(InvocationHandler.class); - Object selfGateway = Proxy.newProxyInstance(ClassLoader.getSystemClassLoader(), new Class[] {TaskExecutorGateway.class, MainThreadExecutor.class, StartStoppable.class}, invocationHandler); - when(testingRpcService.startServer(Matchers.any(RpcEndpoint.class))).thenReturn((RpcGateway)selfGateway); - DirectExecutorService directExecutorService = null; - TaskExecutor taskExecutor = new TaskExecutor(testingRpcService, directExecutorService); - taskExecutor.start(); - - taskExecutor.cancelTask(new ExecutionAttemptID()); - - fail("The cancellation should have thrown an exception."); - } + } From b8dedf1d2e87991b4ca0d0d3d228a63cd048c8fb Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 16 Aug 2016 21:11:01 +0200 Subject: [PATCH 11/50] [FLINK-4403] [rpc] Use relative classloader for proxies, rather than system class loader. --- .../org/apache/flink/runtime/rpc/akka/AkkaRpcService.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java index b647bbd2bba90..d987c2f0361c0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java @@ -104,9 +104,14 @@ public C apply(Object obj) { InvocationHandler akkaInvocationHandler = new AkkaInvocationHandler(actorRef, timeout, maximumFramesize); + // Rather than using the System ClassLoader directly, we derive the ClassLoader + // from this class . That works better in cases where Flink runs embedded and all Flink + // code is loaded dynamically (for example from an OSGI bundle) through a custom ClassLoader + ClassLoader classLoader = AkkaRpcService.this.getClass().getClassLoader(); + @SuppressWarnings("unchecked") C proxy = (C) Proxy.newProxyInstance( - ClassLoader.getSystemClassLoader(), + classLoader, new Class[] {clazz}, akkaInvocationHandler); From 8d502fa37c10bb9e95906c06e81f35c0f0933e18 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 18 Aug 2016 16:34:47 +0200 Subject: [PATCH 12/50] [FLINK-4414] [cluster] Add getAddress method to RpcGateway The RpcGateway.getAddress method allows to retrieve the fully qualified address of the associated RpcEndpoint. This closes #2392. --- .../apache/flink/runtime/rpc/RpcEndpoint.java | 6 +----- .../apache/flink/runtime/rpc/RpcGateway.java | 7 +++++++ .../apache/flink/runtime/rpc/RpcService.java | 11 ---------- .../rpc/akka/AkkaInvocationHandler.java | 14 +++++++++++-- .../runtime/rpc/akka/AkkaRpcService.java | 21 ++++++------------- .../runtime/rpc/akka/AkkaRpcActorTest.java | 16 ++++++++++++++ 6 files changed, 42 insertions(+), 33 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java index a28bc1437c020..7b3f8a189d5d8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java @@ -69,9 +69,6 @@ public abstract class RpcEndpoint { /** Self gateway which can be used to schedule asynchronous calls on yourself */ private final C self; - /** the fully qualified address of the this RPC endpoint */ - private final String selfAddress; - /** The main thread execution context to be used to execute future callbacks in the main thread * of the executing rpc server. */ private final ExecutionContext mainThreadExecutionContext; @@ -92,7 +89,6 @@ protected RpcEndpoint(final RpcService rpcService) { this.selfGatewayType = ReflectionUtil.getTemplateType1(getClass()); this.self = rpcService.startServer(this); - this.selfAddress = rpcService.getAddress(self); this.mainThreadExecutionContext = new MainThreadExecutionContext((MainThreadExecutor) self); } @@ -156,7 +152,7 @@ public C getSelf() { * @return Fully qualified address of the underlying RPC endpoint */ public String getAddress() { - return selfAddress; + return self.getAddress(); } /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcGateway.java index e3a16b47331ca..81075eebb01e1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcGateway.java @@ -22,4 +22,11 @@ * Rpc gateway interface which has to be implemented by Rpc gateways. */ public interface RpcGateway { + + /** + * Returns the fully qualified address under which the associated rpc endpoint is reachable. + * + * @return Fully qualified address under which the associated rpc endpoint is reachable + */ + String getAddress(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java index fabdb057514c3..bc0f5cb191fee 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java @@ -64,17 +64,6 @@ public interface RpcService { */ void stopService(); - /** - * Get the fully qualified address of the underlying rpc server represented by the self gateway. - * It must be possible to connect from a remote host to the rpc server via the returned fully - * qualified address. - * - * @param selfGateway Self gateway associated with the underlying rpc server - * @param Type of the rpc gateway - * @return Fully qualified address - */ - String getAddress(C selfGateway); - /** * Gets the execution context, provided by this RPC service. This execution * context can be used for example for the {@code onComplete(...)} or {@code onSuccess(...)} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java index 524bf7480097d..bfa04f6bedeb4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java @@ -23,6 +23,7 @@ import akka.util.Timeout; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.rpc.MainThreadExecutor; +import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcTimeout; import org.apache.flink.runtime.rpc.StartStoppable; import org.apache.flink.runtime.rpc.akka.messages.CallAsync; @@ -55,6 +56,8 @@ class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThreadExecutor, StartStoppable { private static final Logger LOG = Logger.getLogger(AkkaInvocationHandler.class); + private final String address; + private final ActorRef rpcEndpoint; // whether the actor ref is local and thus no message serialization is needed @@ -65,7 +68,8 @@ class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThrea private final long maximumFramesize; - AkkaInvocationHandler(ActorRef rpcEndpoint, Timeout timeout, long maximumFramesize) { + AkkaInvocationHandler(String address, ActorRef rpcEndpoint, Timeout timeout, long maximumFramesize) { + this.address = Preconditions.checkNotNull(address); this.rpcEndpoint = Preconditions.checkNotNull(rpcEndpoint); this.isLocal = this.rpcEndpoint.path().address().hasLocalScope(); this.timeout = Preconditions.checkNotNull(timeout); @@ -79,7 +83,8 @@ public Object invoke(Object proxy, Method method, Object[] args) throws Throwabl Object result; if (declaringClass.equals(AkkaGateway.class) || declaringClass.equals(MainThreadExecutor.class) || - declaringClass.equals(Object.class) || declaringClass.equals(StartStoppable.class)) { + declaringClass.equals(Object.class) || declaringClass.equals(StartStoppable.class) || + declaringClass.equals(RpcGateway.class)) { result = method.invoke(this, args); } else { String methodName = method.getName(); @@ -290,4 +295,9 @@ private static boolean isRpcTimeout(Annotation[] annotations) { return false; } + + @Override + public String getAddress() { + return address; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java index d987c2f0361c0..00a693253bf4f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java @@ -102,7 +102,9 @@ public Future connect(final String address, final Clas public C apply(Object obj) { ActorRef actorRef = ((ActorIdentity) obj).getRef(); - InvocationHandler akkaInvocationHandler = new AkkaInvocationHandler(actorRef, timeout, maximumFramesize); + final String address = AkkaUtils.getAkkaURL(actorSystem, actorRef); + + InvocationHandler akkaInvocationHandler = new AkkaInvocationHandler(address, actorRef, timeout, maximumFramesize); // Rather than using the System ClassLoader directly, we derive the ClassLoader // from this class . That works better in cases where Flink runs embedded and all Flink @@ -135,7 +137,9 @@ public > C startServer(S rpcEndpo LOG.info("Starting RPC endpoint for {} at {} .", rpcEndpoint.getClass().getName(), actorRef.path()); - InvocationHandler akkaInvocationHandler = new AkkaInvocationHandler(actorRef, timeout, maximumFramesize); + final String address = AkkaUtils.getAkkaURL(actorSystem, actorRef); + + InvocationHandler akkaInvocationHandler = new AkkaInvocationHandler(address, actorRef, timeout, maximumFramesize); // Rather than using the System ClassLoader directly, we derive the ClassLoader // from this class . That works better in cases where Flink runs embedded and all Flink @@ -196,19 +200,6 @@ public void stopService() { actorSystem.awaitTermination(); } - @Override - public String getAddress(RpcGateway selfGateway) { - checkState(!stopped, "RpcService is stopped"); - - if (selfGateway instanceof AkkaGateway) { - ActorRef actorRef = ((AkkaGateway) selfGateway).getRpcEndpoint(); - return AkkaUtils.getAkkaURL(actorSystem, actorRef); - } else { - String className = AkkaGateway.class.getName(); - throw new IllegalArgumentException("Cannot get address for non " + className + '.'); - } - } - @Override public ExecutionContext getExecutionContext() { return actorSystem.dispatcher(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java index 1653facb492a7..82d13f065c294 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java @@ -34,6 +34,7 @@ import java.util.concurrent.TimeUnit; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; public class AkkaRpcActorTest extends TestLogger { @@ -56,6 +57,21 @@ public static void shutdown() { actorSystem.awaitTermination(); } + /** + * Tests that the rpc endpoint and the associated rpc gateway have the same addresses. + * @throws Exception + */ + @Test + public void testAddressResolution() throws Exception { + DummyRpcEndpoint rpcEndpoint = new DummyRpcEndpoint(akkaRpcService); + + Future futureRpcGateway = akkaRpcService.connect(rpcEndpoint.getAddress(), DummyRpcGateway.class); + + DummyRpcGateway rpcGateway = Await.result(futureRpcGateway, timeout.duration()); + + assertEquals(rpcEndpoint.getAddress(), rpcGateway.getAddress()); + } + /** * Tests that the {@link AkkaRpcActor} stashes messages until the corresponding * {@link RpcEndpoint} has been started. From 9f73a9355de131f9154c598871db12c7fe08dd30 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Fri, 19 Aug 2016 23:29:45 +0200 Subject: [PATCH 13/50] [FLINK-4434] [rpc] Add a testing RPC service. This closes #2394. --- .../runtime/rpc/RpcCompletenessTest.java | 3 + .../flink/runtime/rpc/TestingGatewayBase.java | 85 +++++++++++++ .../flink/runtime/rpc/TestingRpcService.java | 115 ++++++++++++++++++ 3 files changed, 203 insertions(+) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingGatewayBase.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java index 97cf0cb35524e..b8aad62c81b09 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java @@ -41,9 +41,11 @@ import static org.junit.Assert.fail; public class RpcCompletenessTest extends TestLogger { + private static final Class futureClass = Future.class; @Test + @SuppressWarnings({"rawtypes", "unchecked"}) public void testRpcCompleteness() { Reflections reflections = new Reflections("org.apache.flink"); @@ -64,6 +66,7 @@ public void testRpcCompleteness() { } } + @SuppressWarnings("rawtypes") private void checkCompleteness(Class rpcEndpoint, Class rpcGateway) { Method[] gatewayMethods = rpcGateway.getDeclaredMethods(); Method[] serverMethods = rpcEndpoint.getDeclaredMethods(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingGatewayBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingGatewayBase.java new file mode 100644 index 0000000000000..4256135ed2960 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingGatewayBase.java @@ -0,0 +1,85 @@ +/* + * 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.rpc; + +import akka.dispatch.Futures; +import scala.concurrent.Future; +import scala.concurrent.Promise; + +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +/** + * Utility base class for testing gateways + */ +public abstract class TestingGatewayBase implements RpcGateway { + + private final ScheduledExecutorService executor; + + protected TestingGatewayBase() { + this.executor = Executors.newSingleThreadScheduledExecutor(); + } + + // ------------------------------------------------------------------------ + // shutdown + // ------------------------------------------------------------------------ + + public void stop() { + executor.shutdownNow(); + } + + @Override + protected void finalize() throws Throwable { + super.finalize(); + executor.shutdownNow(); + } + + // ------------------------------------------------------------------------ + // utilities + // ------------------------------------------------------------------------ + + public Future futureWithTimeout(long timeoutMillis) { + Promise promise = Futures.promise(); + executor.schedule(new FutureTimeout(promise), timeoutMillis, TimeUnit.MILLISECONDS); + return promise.future(); + } + + // ------------------------------------------------------------------------ + + private static final class FutureTimeout implements Runnable { + + private final Promise promise; + + private FutureTimeout(Promise promise) { + this.promise = promise; + } + + @Override + public void run() { + try { + promise.failure(new TimeoutException()); + } catch (Throwable t) { + System.err.println("CAUGHT AN ERROR IN THE TEST: " + t.getMessage()); + t.printStackTrace(); + } + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java new file mode 100644 index 0000000000000..7e92e8df3d6be --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java @@ -0,0 +1,115 @@ +/* + * 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.rpc; + +import akka.dispatch.Futures; +import akka.util.Timeout; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.rpc.akka.AkkaRpcService; + +import scala.concurrent.Future; +import scala.concurrent.duration.FiniteDuration; + +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeUnit; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * An RPC Service implementation for testing. This RPC service acts as a replacement for + * teh regular RPC service for cases where tests need to return prepared mock gateways instead of + * proper RPC gateways. + * + *

The TestingRpcService can be used for example in the following fashion, + * using Mockito for mocks and verification: + * + *

{@code
+ * TestingRpcService rpc = new TestingRpcService();
+ *
+ * ResourceManagerGateway testGateway = mock(ResourceManagerGateway.class);
+ * rpc.registerGateway("myAddress", testGateway);
+ * 
+ * MyComponentToTest component = new MyComponentToTest();
+ * component.triggerSomethingThatCallsTheGateway();
+ * 
+ * verify(testGateway, timeout(1000)).theTestMethod(any(UUID.class), anyString());
+ * }
+ */ +public class TestingRpcService extends AkkaRpcService { + + /** Map of pre-registered connections */ + private final ConcurrentHashMap registeredConnections; + + /** + * Creates a new {@code TestingRpcService}. + */ + public TestingRpcService() { + this(new Configuration()); + } + + /** + * Creates a new {@code TestingRpcService}, using the given configuration. + */ + public TestingRpcService(Configuration configuration) { + super(AkkaUtils.createLocalActorSystem(configuration), new Timeout(new FiniteDuration(10, TimeUnit.SECONDS))); + + this.registeredConnections = new ConcurrentHashMap<>(); + } + + // ------------------------------------------------------------------------ + + @Override + public void stopService() { + super.stopService(); + registeredConnections.clear(); + } + + // ------------------------------------------------------------------------ + // connections + // ------------------------------------------------------------------------ + + public void registerGateway(String address, RpcGateway gateway) { + checkNotNull(address); + checkNotNull(gateway); + + if (registeredConnections.putIfAbsent(address, gateway) != null) { + throw new IllegalStateException("a gateway is already registered under " + address); + } + } + + @Override + public Future connect(String address, Class clazz) { + RpcGateway gateway = registeredConnections.get(address); + + if (gateway != null) { + if (clazz.isAssignableFrom(gateway.getClass())) { + @SuppressWarnings("unchecked") + C typedGateway = (C) gateway; + return Futures.successful(typedGateway); + } else { + return Futures.failed( + new Exception("Gateway registered under " + address + " is not of type " + clazz)); + } + } else { + return Futures.failed(new Exception("No gateway registered under that name")); + } + } +} \ No newline at end of file From 0a957cd78fe58c05b1d0d31cdb9dcc83904ba202 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Fri, 19 Aug 2016 23:45:54 +0200 Subject: [PATCH 14/50] [FLINK-4355] [cluster management] Add tests for the TaskManager -> ResourceManager registration. This closes #2395. --- .../registration/RetryingRegistration.java | 4 + .../runtime/rpc/taskexecutor/SlotReport.java | 38 -- .../rpc/taskexecutor/TaskExecutor.java | 12 + ...skExecutorToResourceManagerConnection.java | 4 + .../TestingHighAvailabilityServices.java | 53 +++ .../flink/runtime/rpc/TestingGatewayBase.java | 18 +- .../RetryingRegistrationTest.java | 336 ++++++++++++++++++ .../registration/TestRegistrationGateway.java | 85 +++++ .../rpc/taskexecutor/TaskExecutorTest.java | 92 ++++- 9 files changed, 602 insertions(+), 40 deletions(-) delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotReport.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/rpc/registration/RetryingRegistrationTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/rpc/registration/TestRegistrationGateway.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RetryingRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RetryingRegistration.java index 4c93684b9fde4..dcb5011fe0a41 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RetryingRegistration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RetryingRegistration.java @@ -58,12 +58,16 @@ public abstract class RetryingRegistration> future = registration.getFuture(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java new file mode 100644 index 0000000000000..3a9f9437bbd34 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java @@ -0,0 +1,53 @@ +/* + * 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.highavailability; + +import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; + +/** + * A variant of the HighAvailabilityServices for testing. Each individual service can be set + * to an arbitrary implementation, such as a mock or default service. + */ +public class TestingHighAvailabilityServices implements HighAvailabilityServices { + + private volatile LeaderRetrievalService resourceManagerLeaderRetriever; + + + // ------------------------------------------------------------------------ + // Setters for mock / testing implementations + // ------------------------------------------------------------------------ + + public void setResourceManagerLeaderRetriever(LeaderRetrievalService resourceManagerLeaderRetriever) { + this.resourceManagerLeaderRetriever = resourceManagerLeaderRetriever; + } + + // ------------------------------------------------------------------------ + // HA Services Methods + // ------------------------------------------------------------------------ + + @Override + public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception { + LeaderRetrievalService service = this.resourceManagerLeaderRetriever; + if (service != null) { + return service; + } else { + throw new IllegalStateException("ResourceManagerLeaderRetriever has not been set"); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingGatewayBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingGatewayBase.java index 4256135ed2960..8133a87e78b01 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingGatewayBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingGatewayBase.java @@ -34,8 +34,15 @@ public abstract class TestingGatewayBase implements RpcGateway { private final ScheduledExecutorService executor; - protected TestingGatewayBase() { + private final String address; + + protected TestingGatewayBase(final String address) { this.executor = Executors.newSingleThreadScheduledExecutor(); + this.address = address; + } + + protected TestingGatewayBase() { + this("localhost"); } // ------------------------------------------------------------------------ @@ -52,6 +59,15 @@ protected void finalize() throws Throwable { executor.shutdownNow(); } + // ------------------------------------------------------------------------ + // Base class methods + // ------------------------------------------------------------------------ + + @Override + public String getAddress() { + return address; + } + // ------------------------------------------------------------------------ // utilities // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/registration/RetryingRegistrationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/registration/RetryingRegistrationTest.java new file mode 100644 index 0000000000000..9508825f5b7a4 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/registration/RetryingRegistrationTest.java @@ -0,0 +1,336 @@ +/* + * 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.rpc.registration; + +import akka.dispatch.Futures; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.rpc.TestingRpcService; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +import org.slf4j.LoggerFactory; + +import scala.concurrent.Await; +import scala.concurrent.ExecutionContext$; +import scala.concurrent.Future; +import scala.concurrent.Promise; +import scala.concurrent.duration.FiniteDuration; + +import java.util.UUID; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeoutException; + +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; + +/** + * Tests for the generic retrying registration class, validating the failure, retry, and back-off behavior. + */ +public class RetryingRegistrationTest extends TestLogger { + + @Test + public void testSimpleSuccessfulRegistration() throws Exception { + final String testId = "laissez les bon temps roulez"; + final String testEndpointAddress = ""; + final UUID leaderId = UUID.randomUUID(); + + // an endpoint that immediately returns success + TestRegistrationGateway testGateway = new TestRegistrationGateway(new TestRegistrationSuccess(testId)); + TestingRpcService rpc = new TestingRpcService(); + + try { + rpc.registerGateway(testEndpointAddress, testGateway); + + TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId); + registration.startRegistration(); + + Future> future = registration.getFuture(); + assertNotNull(future); + + // multiple accesses return the same future + assertEquals(future, registration.getFuture()); + + Tuple2 success = + Await.result(future, new FiniteDuration(10, SECONDS)); + + // validate correct invocation and result + assertEquals(testId, success.f1.getCorrelationId()); + assertEquals(leaderId, testGateway.getInvocations().take().leaderId()); + } + finally { + testGateway.stop(); + rpc.stopService(); + } + } + + @Test + public void testPropagateFailures() throws Exception { + final String testExceptionMessage = "testExceptionMessage"; + + // RPC service that fails with exception upon the connection + RpcService rpc = mock(RpcService.class); + when(rpc.connect(anyString(), any(Class.class))).thenThrow(new RuntimeException(testExceptionMessage)); + + TestRetryingRegistration registration = new TestRetryingRegistration(rpc, "testaddress", UUID.randomUUID()); + registration.startRegistration(); + + Future future = registration.getFuture(); + assertTrue(future.failed().isCompleted()); + + assertEquals(testExceptionMessage, future.failed().value().get().get().getMessage()); + } + + @Test + public void testRetryConnectOnFailure() throws Exception { + final String testId = "laissez les bon temps roulez"; + final UUID leaderId = UUID.randomUUID(); + + ExecutorService executor = Executors.newCachedThreadPool(); + TestRegistrationGateway testGateway = new TestRegistrationGateway(new TestRegistrationSuccess(testId)); + + try { + // RPC service that fails upon the first connection, but succeeds on the second + RpcService rpc = mock(RpcService.class); + when(rpc.connect(anyString(), any(Class.class))).thenReturn( + Futures.failed(new Exception("test connect failure")), // first connection attempt fails + Futures.successful(testGateway) // second connection attempt succeeds + ); + when(rpc.getExecutionContext()).thenReturn(ExecutionContext$.MODULE$.fromExecutor(executor)); + + TestRetryingRegistration registration = new TestRetryingRegistration(rpc, "foobar address", leaderId); + registration.startRegistration(); + + Tuple2 success = + Await.result(registration.getFuture(), new FiniteDuration(10, SECONDS)); + + // validate correct invocation and result + assertEquals(testId, success.f1.getCorrelationId()); + assertEquals(leaderId, testGateway.getInvocations().take().leaderId()); + } + finally { + testGateway.stop(); + executor.shutdown(); + } + } + + @Test + public void testRetriesOnTimeouts() throws Exception { + final String testId = "rien ne va plus"; + final String testEndpointAddress = ""; + final UUID leaderId = UUID.randomUUID(); + + // an endpoint that immediately returns futures with timeouts before returning a successful future + TestRegistrationGateway testGateway = new TestRegistrationGateway( + null, // timeout + null, // timeout + new TestRegistrationSuccess(testId) // success + ); + + TestingRpcService rpc = new TestingRpcService(); + + try { + rpc.registerGateway(testEndpointAddress, testGateway); + + TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId); + + long started = System.nanoTime(); + registration.startRegistration(); + + Future> future = registration.getFuture(); + Tuple2 success = + Await.result(future, new FiniteDuration(10, SECONDS)); + + long finished = System.nanoTime(); + long elapsedMillis = (finished - started) / 1000000; + + // validate correct invocation and result + assertEquals(testId, success.f1.getCorrelationId()); + assertEquals(leaderId, testGateway.getInvocations().take().leaderId()); + + // validate that some retry-delay / back-off behavior happened + assertTrue("retries did not properly back off", elapsedMillis >= 3 * TestRetryingRegistration.INITIAL_TIMEOUT); + } + finally { + rpc.stopService(); + testGateway.stop(); + } + } + + @Test + public void testDecline() throws Exception { + final String testId = "qui a coupe le fromage"; + final String testEndpointAddress = ""; + final UUID leaderId = UUID.randomUUID(); + + TestingRpcService rpc = new TestingRpcService(); + + TestRegistrationGateway testGateway = new TestRegistrationGateway( + null, // timeout + new RegistrationResponse.Decline("no reason "), + null, // timeout + new TestRegistrationSuccess(testId) // success + ); + + try { + rpc.registerGateway(testEndpointAddress, testGateway); + + TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId); + + long started = System.nanoTime(); + registration.startRegistration(); + + Future> future = registration.getFuture(); + Tuple2 success = + Await.result(future, new FiniteDuration(10, SECONDS)); + + long finished = System.nanoTime(); + long elapsedMillis = (finished - started) / 1000000; + + // validate correct invocation and result + assertEquals(testId, success.f1.getCorrelationId()); + assertEquals(leaderId, testGateway.getInvocations().take().leaderId()); + + // validate that some retry-delay / back-off behavior happened + assertTrue("retries did not properly back off", elapsedMillis >= + 2 * TestRetryingRegistration.INITIAL_TIMEOUT + TestRetryingRegistration.DELAY_ON_DECLINE); + } + finally { + testGateway.stop(); + rpc.stopService(); + } + } + + @Test + @SuppressWarnings("unchecked") + public void testRetryOnError() throws Exception { + final String testId = "Petit a petit, l'oiseau fait son nid"; + final String testEndpointAddress = ""; + final UUID leaderId = UUID.randomUUID(); + + TestingRpcService rpc = new TestingRpcService(); + + try { + // gateway that upon calls first responds with a failure, then with a success + TestRegistrationGateway testGateway = mock(TestRegistrationGateway.class); + + when(testGateway.registrationCall(any(UUID.class), anyLong())).thenReturn( + Futures.failed(new Exception("test exception")), + Futures.successful(new TestRegistrationSuccess(testId))); + + rpc.registerGateway(testEndpointAddress, testGateway); + + TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId); + + long started = System.nanoTime(); + registration.startRegistration(); + + Future> future = registration.getFuture(); + Tuple2 success = + Await.result(future, new FiniteDuration(10, SECONDS)); + + long finished = System.nanoTime(); + long elapsedMillis = (finished - started) / 1000000; + + assertEquals(testId, success.f1.getCorrelationId()); + + // validate that some retry-delay / back-off behavior happened + assertTrue("retries did not properly back off", + elapsedMillis >= TestRetryingRegistration.DELAY_ON_ERROR); + } + finally { + rpc.stopService(); + } + } + + @Test + public void testCancellation() throws Exception { + final String testEndpointAddress = "my-test-address"; + final UUID leaderId = UUID.randomUUID(); + + TestingRpcService rpc = new TestingRpcService(); + + try { + Promise result = Futures.promise(); + + TestRegistrationGateway testGateway = mock(TestRegistrationGateway.class); + when(testGateway.registrationCall(any(UUID.class), anyLong())).thenReturn(result.future()); + + rpc.registerGateway(testEndpointAddress, testGateway); + + TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId); + registration.startRegistration(); + + // cancel and fail the current registration attempt + registration.cancel(); + result.failure(new TimeoutException()); + + // there should not be a second registration attempt + verify(testGateway, atMost(1)).registrationCall(any(UUID.class), anyLong()); + } + finally { + rpc.stopService(); + } + } + + // ------------------------------------------------------------------------ + // test registration + // ------------------------------------------------------------------------ + + private static class TestRegistrationSuccess extends RegistrationResponse.Success { + private static final long serialVersionUID = 5542698790917150604L; + + private final String correlationId; + + private TestRegistrationSuccess(String correlationId) { + this.correlationId = correlationId; + } + + public String getCorrelationId() { + return correlationId; + } + } + + private static class TestRetryingRegistration extends RetryingRegistration { + + // we use shorter timeouts here to speed up the tests + static final long INITIAL_TIMEOUT = 20; + static final long MAX_TIMEOUT = 200; + static final long DELAY_ON_ERROR = 200; + static final long DELAY_ON_DECLINE = 200; + + public TestRetryingRegistration(RpcService rpc, String targetAddress, UUID leaderId) { + super(LoggerFactory.getLogger(RetryingRegistrationTest.class), + rpc, "TestEndpoint", + TestRegistrationGateway.class, + targetAddress, leaderId, + INITIAL_TIMEOUT, MAX_TIMEOUT, DELAY_ON_ERROR, DELAY_ON_DECLINE); + } + + @Override + protected Future invokeRegistration( + TestRegistrationGateway gateway, UUID leaderId, long timeoutMillis) { + return gateway.registrationCall(leaderId, timeoutMillis); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/registration/TestRegistrationGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/registration/TestRegistrationGateway.java new file mode 100644 index 0000000000000..a049e48b5ae33 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/registration/TestRegistrationGateway.java @@ -0,0 +1,85 @@ +/* + * 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.rpc.registration; + +import akka.dispatch.Futures; + +import org.apache.flink.runtime.rpc.TestingGatewayBase; +import org.apache.flink.util.Preconditions; + +import scala.concurrent.Future; + +import java.util.UUID; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; + +public class TestRegistrationGateway extends TestingGatewayBase { + + private final BlockingQueue invocations; + + private final RegistrationResponse[] responses; + + private int pos; + + public TestRegistrationGateway(RegistrationResponse... responses) { + Preconditions.checkArgument(responses != null && responses.length > 0); + + this.invocations = new LinkedBlockingQueue<>(); + this.responses = responses; + + } + + // ------------------------------------------------------------------------ + + public Future registrationCall(UUID leaderId, long timeout) { + invocations.add(new RegistrationCall(leaderId, timeout)); + + RegistrationResponse response = responses[pos]; + if (pos < responses.length - 1) { + pos++; + } + + // return a completed future (for a proper value), or one that never completes and will time out (for null) + return response != null ? Futures.successful(response) : this.futureWithTimeout(timeout); + } + + public BlockingQueue getInvocations() { + return invocations; + } + + // ------------------------------------------------------------------------ + + public static class RegistrationCall { + private final UUID leaderId; + private final long timeout; + + public RegistrationCall(UUID leaderId, long timeout) { + this.leaderId = leaderId; + this.timeout = timeout; + } + + public UUID leaderId() { + return leaderId; + } + + public long timeout() { + return timeout; + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java index 9f9bab372f5bb..b831ead5c9a11 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java @@ -18,8 +18,98 @@ package org.apache.flink.runtime.rpc.taskexecutor; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.highavailability.NonHaServices; +import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; +import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService; +import org.apache.flink.runtime.rpc.TestingRpcService; +import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway; import org.apache.flink.util.TestLogger; +import org.junit.Test; + +import scala.concurrent.duration.FiniteDuration; + +import java.util.UUID; + +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; + public class TaskExecutorTest extends TestLogger { - + + @Test + public void testImmediatelyRegistersIfLeaderIsKnown() throws Exception { + final ResourceID resourceID = ResourceID.generate(); + final String resourceManagerAddress = "/resource/manager/address/one"; + + final TestingRpcService rpc = new TestingRpcService(); + try { + // register a mock resource manager gateway + ResourceManagerGateway rmGateway = mock(ResourceManagerGateway.class); + rpc.registerGateway(resourceManagerAddress, rmGateway); + + NonHaServices haServices = new NonHaServices(resourceManagerAddress); + TaskExecutor taskManager = new TaskExecutor(rpc, haServices, resourceID); + String taskManagerAddress = taskManager.getAddress(); + + taskManager.start(); + + verify(rmGateway, timeout(5000)).registerTaskExecutor( + any(UUID.class), eq(taskManagerAddress), eq(resourceID), any(FiniteDuration.class)); + } + finally { + rpc.stopService(); + } + } + + @Test + public void testTriggerRegistrationOnLeaderChange() throws Exception { + final ResourceID resourceID = ResourceID.generate(); + + final String address1 = "/resource/manager/address/one"; + final String address2 = "/resource/manager/address/two"; + final UUID leaderId1 = UUID.randomUUID(); + final UUID leaderId2 = UUID.randomUUID(); + + final TestingRpcService rpc = new TestingRpcService(); + try { + // register the mock resource manager gateways + ResourceManagerGateway rmGateway1 = mock(ResourceManagerGateway.class); + ResourceManagerGateway rmGateway2 = mock(ResourceManagerGateway.class); + rpc.registerGateway(address1, rmGateway1); + rpc.registerGateway(address2, rmGateway2); + + TestingLeaderRetrievalService testLeaderService = new TestingLeaderRetrievalService(); + + TestingHighAvailabilityServices haServices = new TestingHighAvailabilityServices(); + haServices.setResourceManagerLeaderRetriever(testLeaderService); + + TaskExecutor taskManager = new TaskExecutor(rpc, haServices, resourceID); + String taskManagerAddress = taskManager.getAddress(); + taskManager.start(); + + // no connection initially, since there is no leader + assertNull(taskManager.getResourceManagerConnection()); + + // define a leader and see that a registration happens + testLeaderService.notifyListener(address1, leaderId1); + + verify(rmGateway1, timeout(5000)).registerTaskExecutor( + eq(leaderId1), eq(taskManagerAddress), eq(resourceID), any(FiniteDuration.class)); + assertNotNull(taskManager.getResourceManagerConnection()); + + // cancel the leader + testLeaderService.notifyListener(null, null); + + // set a new leader, see that a registration happens + testLeaderService.notifyListener(address2, leaderId2); + + verify(rmGateway2, timeout(5000)).registerTaskExecutor( + eq(leaderId2), eq(taskManagerAddress), eq(resourceID), any(FiniteDuration.class)); + assertNotNull(taskManager.getResourceManagerConnection()); + } + finally { + rpc.stopService(); + } + } } From cc76c4fbf95baa7c5672d0559641b58813f05d3e Mon Sep 17 00:00:00 2001 From: "xiaogang.sxg" Date: Wed, 17 Aug 2016 13:46:00 +0800 Subject: [PATCH 15/50] [FLINK-4400] [cluster mngmt] Implement leadership election among JobMasters Adapt related components to the changes in HighAvailabilityServices Add comments for getJobMasterElectionService in HighAvailabilityServices This closes #2377. --- .../HighAvailabilityServices.java | 9 + .../highavailability/NonHaServices.java | 8 + .../runtime/rpc/jobmaster/JobMaster.java | 318 +++++++++--------- .../runtime/rpc/akka/AkkaRpcServiceTest.java | 53 +-- 4 files changed, 179 insertions(+), 209 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java index 094d36f0c6d9c..73e4f1fe6d5c6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java @@ -18,6 +18,8 @@ package org.apache.flink.runtime.highavailability; +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; /** @@ -36,4 +38,11 @@ public interface HighAvailabilityServices { * Gets the leader retriever for the cluster's resource manager. */ LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception; + + /** + * Gets the leader election service for the given job. + * + * @param jobID The identifier of the job running the election. + */ + LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java index b8c2ed89adff5..3d2769bab1630 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java @@ -18,6 +18,9 @@ package org.apache.flink.runtime.highavailability; +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.leaderelection.LeaderElectionService; +import org.apache.flink.runtime.leaderelection.StandaloneLeaderElectionService; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.leaderretrieval.StandaloneLeaderRetrievalService; @@ -56,4 +59,9 @@ public NonHaServices(String resourceManagerAddress) { public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception { return new StandaloneLeaderRetrievalService(resourceManagerAddress, new UUID(0, 0)); } + + @Override + public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception { + return new StandaloneLeaderElectionService(); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java index e53cd68f93fdb..49b200be3ad1d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java @@ -18,74 +18,168 @@ package org.apache.flink.runtime.rpc.jobmaster; -import akka.dispatch.Futures; -import akka.dispatch.Mapper; -import akka.dispatch.OnComplete; -import org.apache.flink.runtime.instance.InstanceID; +import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmanager.RecoveryMode; +import org.apache.flink.runtime.leaderelection.LeaderContender; +import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.rpc.RpcMethod; -import org.apache.flink.runtime.rpc.resourcemanager.JobMasterRegistration; -import org.apache.flink.runtime.rpc.resourcemanager.RegistrationResponse; import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.taskmanager.TaskExecutionState; import org.apache.flink.util.Preconditions; -import scala.Tuple2; -import scala.concurrent.ExecutionContext; -import scala.concurrent.ExecutionContext$; -import scala.concurrent.Future; -import scala.concurrent.duration.Deadline; -import scala.concurrent.duration.FiniteDuration; import java.util.UUID; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledThreadPoolExecutor; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; /** * JobMaster implementation. The job master is responsible for the execution of a single * {@link org.apache.flink.runtime.jobgraph.JobGraph}. - * + *

* It offers the following methods as part of its rpc interface to interact with the JobMaster * remotely: *

    - *
  • {@link #registerAtResourceManager(String)} triggers the registration at the resource manager
  • *
  • {@link #updateTaskExecutionState(TaskExecutionState)} updates the task execution state for * given task
  • *
*/ public class JobMaster extends RpcEndpoint { - /** Execution context for future callbacks */ - private final ExecutionContext executionContext; - - /** Execution context for scheduled runnables */ - private final ScheduledExecutorService scheduledExecutorService; - - private final FiniteDuration initialRegistrationTimeout = new FiniteDuration(500, TimeUnit.MILLISECONDS); - private final FiniteDuration maxRegistrationTimeout = new FiniteDuration(30, TimeUnit.SECONDS); - private final FiniteDuration registrationDuration = new FiniteDuration(365, TimeUnit.DAYS); - private final long failedRegistrationDelay = 10000; /** Gateway to connected resource manager, null iff not connected */ private ResourceManagerGateway resourceManager = null; - /** UUID to filter out old registration runs */ - private UUID currentRegistrationRun; + /** Logical representation of the job */ + private final JobGraph jobGraph; + private final JobID jobID; + + /** Configuration of the job */ + private final Configuration configuration; + private final RecoveryMode recoveryMode; + + /** Service to contend for and retrieve the leadership of JM and RM */ + private final HighAvailabilityServices highAvailabilityServices; + + /** Leader Management */ + private LeaderElectionService leaderElectionService = null; + private UUID leaderSessionID; + + /** + * The JM's Constructor + * + * @param jobGraph The representation of the job's execution plan + * @param configuration The job's configuration + * @param rpcService The RPC service at which the JM serves + * @param highAvailabilityService The cluster's HA service from the JM can elect and retrieve leaders. + */ + public JobMaster( + JobGraph jobGraph, + Configuration configuration, + RpcService rpcService, + HighAvailabilityServices highAvailabilityService) { - public JobMaster(RpcService rpcService, ExecutorService executorService) { super(rpcService); - executionContext = ExecutionContext$.MODULE$.fromExecutor( - Preconditions.checkNotNull(executorService)); - scheduledExecutorService = new ScheduledThreadPoolExecutor(1); + + this.jobGraph = Preconditions.checkNotNull(jobGraph); + this.jobID = Preconditions.checkNotNull(jobGraph.getJobID()); + + this.configuration = Preconditions.checkNotNull(configuration); + this.recoveryMode = RecoveryMode.fromConfig(configuration); + + this.highAvailabilityServices = Preconditions.checkNotNull(highAvailabilityService); } public ResourceManagerGateway getResourceManager() { return resourceManager; } + //---------------------------------------------------------------------------------------------- + // Initialization methods + //---------------------------------------------------------------------------------------------- + public void start() { + super.start(); + + // register at the election once the JM starts + registerAtElectionService(); + } + + + //---------------------------------------------------------------------------------------------- + // JobMaster Leadership methods + //---------------------------------------------------------------------------------------------- + + /** + * Retrieves the election service and contend for the leadership. + */ + private void registerAtElectionService() { + try { + leaderElectionService = highAvailabilityServices.getJobMasterLeaderElectionService(jobID); + leaderElectionService.start(new JobMasterLeaderContender()); + } catch (Exception e) { + throw new RuntimeException("Fail to register at the election of JobMaster", e); + } + } + + /** + * Start the execution when the leadership is granted. + * + * @param newLeaderSessionID The identifier of the new leadership session + */ + public void grantJobMasterLeadership(final UUID newLeaderSessionID) { + runAsync(new Runnable() { + @Override + public void run() { + log.info("JobManager {} grants leadership with session id {}.", getAddress(), newLeaderSessionID); + + // The operation may be blocking, but since JM is idle before it grants the leadership, it's okay that + // JM waits here for the operation's completeness. + leaderSessionID = newLeaderSessionID; + leaderElectionService.confirmLeaderSessionID(newLeaderSessionID); + + // TODO:: execute the job when the leadership is granted. + } + }); + } + + /** + * Stop the execution when the leadership is revoked. + */ + public void revokeJobMasterLeadership() { + runAsync(new Runnable() { + @Override + public void run() { + log.info("JobManager {} was revoked leadership.", getAddress()); + + // TODO:: cancel the job's execution and notify all listeners + cancelAndClearEverything(new Exception("JobManager is no longer the leader.")); + + leaderSessionID = null; + } + }); + } + + /** + * Handles error occurring in the leader election service + * + * @param exception Exception thrown in the leader election service + */ + public void onJobMasterElectionError(final Exception exception) { + runAsync(new Runnable() { + @Override + public void run() { + log.error("Received an error from the LeaderElectionService.", exception); + + // TODO:: cancel the job's execution and shutdown the JM + cancelAndClearEverything(exception); + + leaderSessionID = null; + } + }); + + } + //---------------------------------------------------------------------------------------------- // RPC methods //---------------------------------------------------------------------------------------------- @@ -109,18 +203,7 @@ public Acknowledge updateTaskExecutionState(TaskExecutionState taskExecutionStat */ @RpcMethod public void registerAtResourceManager(final String address) { - currentRegistrationRun = UUID.randomUUID(); - - Future resourceManagerFuture = getRpcService().connect(address, ResourceManagerGateway.class); - - handleResourceManagerRegistration( - new JobMasterRegistration(getAddress()), - 1, - resourceManagerFuture, - currentRegistrationRun, - initialRegistrationTimeout, - maxRegistrationTimeout, - registrationDuration.fromNow()); + //TODO:: register at the RM } //---------------------------------------------------------------------------------------------- @@ -128,124 +211,37 @@ public void registerAtResourceManager(final String address) { //---------------------------------------------------------------------------------------------- /** - * Helper method to handle the resource manager registration process. If a registration attempt - * times out, then a new attempt with the doubled time out is initiated. The whole registration - * process has a deadline. Once this deadline is overdue without successful registration, the - * job master shuts down. + * Cancel the current job and notify all listeners the job's cancellation. * - * @param jobMasterRegistration Job master registration info which is sent to the resource - * manager - * @param attemptNumber Registration attempt number - * @param resourceManagerFuture Future of the resource manager gateway - * @param registrationRun UUID describing the current registration run - * @param timeout Timeout of the last registration attempt - * @param maxTimeout Maximum timeout between registration attempts - * @param deadline Deadline for the registration + * @param cause Cause for the cancelling. */ - void handleResourceManagerRegistration( - final JobMasterRegistration jobMasterRegistration, - final int attemptNumber, - final Future resourceManagerFuture, - final UUID registrationRun, - final FiniteDuration timeout, - final FiniteDuration maxTimeout, - final Deadline deadline) { - - // filter out concurrent registration runs - if (registrationRun.equals(currentRegistrationRun)) { - - log.info("Start registration attempt #{}.", attemptNumber); - - if (deadline.isOverdue()) { - // we've exceeded our registration deadline. This means that we have to shutdown the JobMaster - log.error("Exceeded registration deadline without successfully registering at the ResourceManager."); - shutDown(); - } else { - Future> registrationResponseFuture = resourceManagerFuture.flatMap(new Mapper>>() { - @Override - public Future> apply(ResourceManagerGateway resourceManagerGateway) { - return resourceManagerGateway.registerJobMaster(jobMasterRegistration, timeout).zip(Futures.successful(resourceManagerGateway)); - } - }, executionContext); - - registrationResponseFuture.onComplete(new OnComplete>() { - @Override - public void onComplete(Throwable failure, Tuple2 tuple) throws Throwable { - if (failure != null) { - if (failure instanceof TimeoutException) { - // we haven't received an answer in the given timeout interval, - // so increase it and try again. - final FiniteDuration newTimeout = timeout.$times(2L).min(maxTimeout); - - handleResourceManagerRegistration( - jobMasterRegistration, - attemptNumber + 1, - resourceManagerFuture, - registrationRun, - newTimeout, - maxTimeout, - deadline); - } else { - log.error("Received unknown error while registering at the ResourceManager.", failure); - shutDown(); - } - } else { - final RegistrationResponse response = tuple._1(); - final ResourceManagerGateway gateway = tuple._2(); - - if (response.isSuccess()) { - finishResourceManagerRegistration(gateway, response.getInstanceID()); - } else { - log.info("The registration was refused. Try again."); - - scheduledExecutorService.schedule(new Runnable() { - @Override - public void run() { - // we have to execute scheduled runnable in the main thread - // because we need consistency wrt currentRegistrationRun - runAsync(new Runnable() { - @Override - public void run() { - // our registration attempt was refused. Start over. - handleResourceManagerRegistration( - jobMasterRegistration, - 1, - resourceManagerFuture, - registrationRun, - initialRegistrationTimeout, - maxTimeout, - deadline); - } - }); - } - }, failedRegistrationDelay, TimeUnit.MILLISECONDS); - } - } - } - }, getMainThreadExecutionContext()); // use the main thread execution context to execute the call back in the main thread - } - } else { - log.info("Discard out-dated registration run."); - } + private void cancelAndClearEverything(Throwable cause) { + // currently, nothing to do here } - /** - * Finish the resource manager registration by setting the new resource manager gateway. - * - * @param resourceManager New resource manager gateway - * @param instanceID Instance id assigned by the resource manager - */ - void finishResourceManagerRegistration(ResourceManagerGateway resourceManager, InstanceID instanceID) { - log.info("Successfully registered at the ResourceManager under instance id {}.", instanceID); - this.resourceManager = resourceManager; - } + // ------------------------------------------------------------------------ + // Utility classes + // ------------------------------------------------------------------------ + private class JobMasterLeaderContender implements LeaderContender { - /** - * Return if the job master is connected to a resource manager. - * - * @return true if the job master is connected to the resource manager - */ - public boolean isConnected() { - return resourceManager != null; + @Override + public void grantLeadership(UUID leaderSessionID) { + JobMaster.this.grantJobMasterLeadership(leaderSessionID); + } + + @Override + public void revokeLeadership() { + JobMaster.this.revokeJobMasterLeadership(); + } + + @Override + public String getAddress() { + return JobMaster.this.getAddress(); + } + + @Override + public void handleError(Exception exception) { + onJobMasterElectionError(exception); + } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java index 7b4ab89a84c24..2790cf88290b2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java @@ -20,9 +20,12 @@ import akka.actor.ActorSystem; import akka.util.Timeout; - import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.highavailability.NonHaServices; +import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.rpc.jobmaster.JobMaster; import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.rpc.resourcemanager.ResourceManager; @@ -31,6 +34,7 @@ import org.junit.AfterClass; import org.junit.Test; +import org.mockito.Mockito; import scala.concurrent.duration.Deadline; import scala.concurrent.duration.FiniteDuration; @@ -80,51 +84,4 @@ public void run() { assertTrue("call was not properly delayed", ((stop - start) / 1000000) >= delay); } - - // ------------------------------------------------------------------------ - // specific component tests - should be moved to the test classes - // for those components - // ------------------------------------------------------------------------ - - /** - * Tests that the {@link JobMaster} can connect to the {@link ResourceManager} using the - * {@link AkkaRpcService}. - */ - @Test - public void testJobMasterResourceManagerRegistration() throws Exception { - Timeout akkaTimeout = new Timeout(10, TimeUnit.SECONDS); - ActorSystem actorSystem = AkkaUtils.createDefaultActorSystem(); - ActorSystem actorSystem2 = AkkaUtils.createDefaultActorSystem(); - AkkaRpcService akkaRpcService = new AkkaRpcService(actorSystem, akkaTimeout); - AkkaRpcService akkaRpcService2 = new AkkaRpcService(actorSystem2, akkaTimeout); - ExecutorService executorService = new ForkJoinPool(); - - ResourceManager resourceManager = new ResourceManager(akkaRpcService, executorService); - JobMaster jobMaster = new JobMaster(akkaRpcService2, executorService); - - resourceManager.start(); - jobMaster.start(); - - ResourceManagerGateway rm = resourceManager.getSelf(); - - assertTrue(rm instanceof AkkaGateway); - - AkkaGateway akkaClient = (AkkaGateway) rm; - - - jobMaster.registerAtResourceManager(AkkaUtils.getAkkaURL(actorSystem, akkaClient.getRpcEndpoint())); - - // wait for successful registration - FiniteDuration timeout = new FiniteDuration(200, TimeUnit.SECONDS); - Deadline deadline = timeout.fromNow(); - - while (deadline.hasTimeLeft() && !jobMaster.isConnected()) { - Thread.sleep(100); - } - - assertFalse(deadline.isOverdue()); - - jobMaster.shutDown(); - resourceManager.shutDown(); - } } From f8041aba71219262460662058baec867fdc2d522 Mon Sep 17 00:00:00 2001 From: "wenlong.lwl" Date: Sun, 21 Aug 2016 00:46:51 +0800 Subject: [PATCH 16/50] [FLINK-4443] [rpc] Add support for rpc gateway and rpc endpoint inheritance This commit extends the RpcCompletenessTest such that it can now check for inherited remote procedure calls. All methods defined at the RpcGateway are considered native. This means that they need no RpcEndpoint counterpart because they are implemented by the RpcGateway implementation. This closes #2401. update comments remove native method annotation add line break --- .../apache/flink/runtime/rpc/RpcMethod.java | 2 ++ .../TestingHighAvailabilityServices.java | 19 +++++++++++ .../runtime/rpc/RpcCompletenessTest.java | 33 +++++++++++++++++-- 3 files changed, 52 insertions(+), 2 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcMethod.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcMethod.java index 875e557933395..e4b0e94d81a0e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcMethod.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcMethod.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.rpc; import java.lang.annotation.ElementType; +import java.lang.annotation.Inherited; import java.lang.annotation.Retention; import java.lang.annotation.RetentionPolicy; import java.lang.annotation.Target; @@ -29,6 +30,7 @@ * RpcCompletenessTest makes sure that the set of rpc methods in a rpc server and the set of * gateway methods in the corresponding gateway implementation are identical. */ +@Inherited @Target(ElementType.METHOD) @Retention(RetentionPolicy.RUNTIME) public @interface RpcMethod { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java index 3a9f9437bbd34..4d654a390d37f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java @@ -18,6 +18,8 @@ package org.apache.flink.runtime.highavailability; +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; /** @@ -28,6 +30,8 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices private volatile LeaderRetrievalService resourceManagerLeaderRetriever; + private volatile LeaderElectionService jobMasterLeaderElectionService; + // ------------------------------------------------------------------------ // Setters for mock / testing implementations @@ -36,6 +40,10 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices public void setResourceManagerLeaderRetriever(LeaderRetrievalService resourceManagerLeaderRetriever) { this.resourceManagerLeaderRetriever = resourceManagerLeaderRetriever; } + + public void setJobMasterLeaderElectionService(LeaderElectionService leaderElectionService) { + this.jobMasterLeaderElectionService = leaderElectionService; + } // ------------------------------------------------------------------------ // HA Services Methods @@ -50,4 +58,15 @@ public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Excepti throw new IllegalStateException("ResourceManagerLeaderRetriever has not been set"); } } + + @Override + public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception { + LeaderElectionService service = jobMasterLeaderElectionService; + + if (service != null) { + return service; + } else { + throw new IllegalStateException("JobMasterLeaderElectionService has not been set"); + } + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java index b8aad62c81b09..b431eb92be2cd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java @@ -68,8 +68,8 @@ public void testRpcCompleteness() { @SuppressWarnings("rawtypes") private void checkCompleteness(Class rpcEndpoint, Class rpcGateway) { - Method[] gatewayMethods = rpcGateway.getDeclaredMethods(); - Method[] serverMethods = rpcEndpoint.getDeclaredMethods(); + Method[] gatewayMethods = getRpcMethodsFromGateway(rpcGateway).toArray(new Method[0]); + Method[] serverMethods = rpcEndpoint.getMethods(); Map> rpcMethods = new HashMap<>(); Set unmatchedRpcMethods = new HashSet<>(); @@ -340,4 +340,33 @@ private static Class resolvePrimitiveType(Class primitveType) { throw new RuntimeException("Could not retrive basic type information for primitive type " + primitveType + '.'); } } + + /** + * Extract all rpc methods defined by the gateway interface + * + * @param interfaceClass the given rpc gateway interface + * @return all methods defined by the given interface + */ + private List getRpcMethodsFromGateway(Class interfaceClass) { + if(!interfaceClass.isInterface()) { + fail(interfaceClass.getName() + "is not a interface"); + } + + ArrayList allMethods = new ArrayList<>(); + // Methods defined in RpcGateway are native method + if(interfaceClass.equals(RpcGateway.class)) { + return allMethods; + } + + // Get all methods declared in current interface + for(Method method : interfaceClass.getDeclaredMethods()) { + allMethods.add(method); + } + + // Get all method inherited from super interface + for(Class superClass : interfaceClass.getInterfaces()) { + allMethods.addAll(getRpcMethodsFromGateway(superClass)); + } + return allMethods; + } } From 2f7a67326a5dc78c318c6716bc7e7bc7c3cdd6ed Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 25 Aug 2016 20:37:15 +0200 Subject: [PATCH 17/50] [hotfix] Remove RecoveryMode from JobMaster The recovery mode is not used any more by the latest CheckpointCoordinator. All difference in recovery logic between high-availability and non-high-availability is encapsulated in the HighAvailabilityServices. --- .../java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java index 49b200be3ad1d..a046cb8948052 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java @@ -22,7 +22,6 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobmanager.RecoveryMode; import org.apache.flink.runtime.leaderelection.LeaderContender; import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.messages.Acknowledge; @@ -57,7 +56,6 @@ public class JobMaster extends RpcEndpoint { /** Configuration of the job */ private final Configuration configuration; - private final RecoveryMode recoveryMode; /** Service to contend for and retrieve the leadership of JM and RM */ private final HighAvailabilityServices highAvailabilityServices; @@ -86,7 +84,6 @@ public JobMaster( this.jobID = Preconditions.checkNotNull(jobGraph.getJobID()); this.configuration = Preconditions.checkNotNull(configuration); - this.recoveryMode = RecoveryMode.fromConfig(configuration); this.highAvailabilityServices = Preconditions.checkNotNull(highAvailabilityService); } From 86f2fa31f1334a18633eec531303e652bfc4b54b Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 29 Aug 2016 17:40:57 +0200 Subject: [PATCH 18/50] [hotfix] [clustermgnt] Set pending registration properly in TaskExecutorToResourceManagerConnection --- .../TaskExecutorToResourceManagerConnection.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorToResourceManagerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorToResourceManagerConnection.java index f398b7d06d6dc..7ccc8794761e8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorToResourceManagerConnection.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorToResourceManagerConnection.java @@ -55,7 +55,7 @@ public class TaskExecutorToResourceManagerConnection { private final String resourceManagerAddress; - private ResourceManagerRegistration pendingRegistration; + private TaskExecutorToResourceManagerConnection.ResourceManagerRegistration pendingRegistration; private ResourceManagerGateway registeredResourceManager; @@ -86,13 +86,13 @@ public void start() { checkState(!closed, "The connection is already closed"); checkState(!isRegistered() && pendingRegistration == null, "The connection is already started"); - ResourceManagerRegistration registration = new ResourceManagerRegistration( + pendingRegistration = new TaskExecutorToResourceManagerConnection.ResourceManagerRegistration( log, taskExecutor.getRpcService(), resourceManagerAddress, resourceManagerLeaderId, taskExecutor.getAddress(), taskExecutor.getResourceID()); - registration.startRegistration(); + pendingRegistration.startRegistration(); - Future> future = registration.getFuture(); + Future> future = pendingRegistration.getFuture(); future.onSuccess(new OnSuccess>() { @Override @@ -167,14 +167,14 @@ public String toString() { // Utilities // ------------------------------------------------------------------------ - static class ResourceManagerRegistration + private static class ResourceManagerRegistration extends RetryingRegistration { private final String taskExecutorAddress; private final ResourceID resourceID; - public ResourceManagerRegistration( + ResourceManagerRegistration( Logger log, RpcService rpcService, String targetAddress, From 75a688435cec321764170d6870509cdd2ef2e5d6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=B7=98=E6=B1=9F?= Date: Tue, 30 Aug 2016 11:28:14 +0800 Subject: [PATCH 19/50] [FLINK-4363] Implement TaskManager basic startup of all components in java This closes #2400 --- .../rpc/taskexecutor/TaskExecutor.java | 686 +++++++++++++++++- .../TaskExecutorConfiguration.java | 151 ++++ .../rpc/taskexecutor/TaskExecutorTest.java | 8 +- 3 files changed, 822 insertions(+), 23 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorConfiguration.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java index f201e00c31050..36d6310d9c320 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java @@ -18,15 +18,60 @@ package org.apache.flink.runtime.rpc.taskexecutor; +import akka.actor.ActorSystem; +import akka.dispatch.ExecutionContexts$; +import akka.util.Timeout; +import com.typesafe.config.Config; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.IllegalConfigurationException; +import org.apache.flink.core.memory.HeapMemorySegment; +import org.apache.flink.core.memory.HybridMemorySegment; +import org.apache.flink.core.memory.MemorySegmentFactory; +import org.apache.flink.core.memory.MemoryType; +import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.instance.InstanceConnectionInfo; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode; +import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; +import org.apache.flink.runtime.io.network.NetworkEnvironment; +import org.apache.flink.runtime.io.network.netty.NettyConfig; +import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener; +import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; +import org.apache.flink.runtime.memory.MemoryManager; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcMethod; import org.apache.flink.runtime.rpc.RpcService; - +import org.apache.flink.runtime.rpc.akka.AkkaRpcService; +import org.apache.flink.runtime.taskmanager.MemoryLogger; +import org.apache.flink.runtime.util.EnvironmentInformation; +import org.apache.flink.runtime.util.LeaderRetrievalUtils; +import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration; +import org.apache.flink.util.MathUtils; +import org.apache.flink.util.NetUtils; + +import scala.Tuple2; +import scala.Option; +import scala.Some; +import scala.concurrent.ExecutionContext; +import scala.concurrent.duration.Duration; +import scala.concurrent.duration.FiniteDuration; + +import java.io.File; +import java.io.IOException; +import java.net.InetAddress; +import java.net.InetSocketAddress; import java.util.UUID; +import java.util.concurrent.ForkJoinPool; +import java.util.concurrent.TimeUnit; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -36,12 +81,29 @@ */ public class TaskExecutor extends RpcEndpoint { + private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class); + /** The unique resource ID of this TaskExecutor */ private final ResourceID resourceID; /** The access to the leader election and metadata storage services */ private final HighAvailabilityServices haServices; + /** The task manager configuration */ + private final TaskExecutorConfiguration taskExecutorConfig; + + /** The I/O manager component in the task manager */ + private final IOManager ioManager; + + /** The memory manager component in the task manager */ + private final MemoryManager memoryManager; + + /** The network component in the task manager */ + private final NetworkEnvironment networkEnvironment; + + /** The number of slots in the task manager, should be 1 for YARN */ + private final int numberOfSlots; + // --------- resource manager -------- private TaskExecutorToResourceManagerConnection resourceManagerConnection; @@ -49,22 +111,24 @@ public class TaskExecutor extends RpcEndpoint { // ------------------------------------------------------------------------ public TaskExecutor( + TaskExecutorConfiguration taskExecutorConfig, + ResourceID resourceID, + MemoryManager memoryManager, + IOManager ioManager, + NetworkEnvironment networkEnvironment, + int numberOfSlots, RpcService rpcService, - HighAvailabilityServices haServices, - ResourceID resourceID) { + HighAvailabilityServices haServices) { super(rpcService); - this.haServices = checkNotNull(haServices); + this.taskExecutorConfig = checkNotNull(taskExecutorConfig); this.resourceID = checkNotNull(resourceID); - } - - // ------------------------------------------------------------------------ - // Properties - // ------------------------------------------------------------------------ - - public ResourceID getResourceID() { - return resourceID; + this.memoryManager = checkNotNull(memoryManager); + this.ioManager = checkNotNull(ioManager); + this.networkEnvironment = checkNotNull(networkEnvironment); + this.numberOfSlots = checkNotNull(numberOfSlots); + this.haServices = checkNotNull(haServices); } // ------------------------------------------------------------------------ @@ -83,7 +147,6 @@ public void start() { } } - // ------------------------------------------------------------------------ // RPC methods - ResourceManager related // ------------------------------------------------------------------------ @@ -94,12 +157,12 @@ public void notifyOfNewResourceManagerLeader(String newLeaderAddress, UUID newLe if (newLeaderAddress != null) { // the resource manager switched to a new leader log.info("ResourceManager leader changed from {} to {}. Registering at new leader.", - resourceManagerConnection.getResourceManagerAddress(), newLeaderAddress); + resourceManagerConnection.getResourceManagerAddress(), newLeaderAddress); } else { // address null means that the current leader is lost without a new leader being there, yet log.info("Current ResourceManager {} lost leader status. Waiting for new ResourceManager leader.", - resourceManagerConnection.getResourceManagerAddress()); + resourceManagerConnection.getResourceManagerAddress()); } // drop the current connection or connection attempt @@ -112,21 +175,604 @@ public void notifyOfNewResourceManagerLeader(String newLeaderAddress, UUID newLe // establish a connection to the new leader if (newLeaderAddress != null) { log.info("Attempting to register at ResourceManager {}", newLeaderAddress); - resourceManagerConnection = - new TaskExecutorToResourceManagerConnection(log, this, newLeaderAddress, newLeaderId); + resourceManagerConnection = + new TaskExecutorToResourceManagerConnection(log, this, newLeaderAddress, newLeaderId); resourceManagerConnection.start(); } } + /** + * Starts and runs the TaskManager. + *

+ * This method first tries to select the network interface to use for the TaskManager + * communication. The network interface is used both for the actor communication + * (coordination) as well as for the data exchange between task managers. Unless + * the hostname/interface is explicitly configured in the configuration, this + * method will try out various interfaces and methods to connect to the JobManager + * and select the one where the connection attempt is successful. + *

+ * After selecting the network interface, this method brings up an actor system + * for the TaskManager and its actors, starts the TaskManager's services + * (library cache, shuffle network stack, ...), and starts the TaskManager itself. + * + * @param configuration The configuration for the TaskManager. + * @param resourceID The id of the resource which the task manager will run on. + */ + public static void selectNetworkInterfaceAndRunTaskManager( + Configuration configuration, + ResourceID resourceID) throws Exception { + + final InetSocketAddress taskManagerAddress = selectNetworkInterfaceAndPort(configuration); + + runTaskManager(taskManagerAddress.getHostName(), resourceID, taskManagerAddress.getPort(), configuration); + } + + private static InetSocketAddress selectNetworkInterfaceAndPort(Configuration configuration) + throws Exception { + String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null); + if (taskManagerHostname != null) { + LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname); + } else { + LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration); + FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration); + + InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout); + taskManagerHostname = taskManagerAddress.getHostName(); + LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.", + taskManagerHostname, taskManagerAddress.getHostAddress()); + } + + // if no task manager port has been configured, use 0 (system will pick any free port) + final int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0); + if (actorSystemPort < 0 || actorSystemPort > 65535) { + throw new IllegalConfigurationException("Invalid value for '" + + ConfigConstants.TASK_MANAGER_IPC_PORT_KEY + + "' (port for the TaskManager actor system) : " + actorSystemPort + + " - Leave config parameter empty or use 0 to let the system choose a port automatically."); + } + + return new InetSocketAddress(taskManagerHostname, actorSystemPort); + } + + /** + * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its + * actors, starts the TaskManager's services (library cache, shuffle network stack, ...), + * and starts the TaskManager itself. + *

+ * This method will also spawn a process reaper for the TaskManager (kill the process if + * the actor fails) and optionally start the JVM memory logging thread. + * + * @param taskManagerHostname The hostname/address of the interface where the actor system + * will communicate. + * @param resourceID The id of the resource which the task manager will run on. + * @param actorSystemPort The port at which the actor system will communicate. + * @param configuration The configuration for the TaskManager. + */ + private static void runTaskManager( + String taskManagerHostname, + ResourceID resourceID, + int actorSystemPort, + final Configuration configuration) throws Exception { + + LOG.info("Starting TaskManager"); + + // Bring up the TaskManager actor system first, bind it to the given address. + + LOG.info("Starting TaskManager actor system at " + + NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort)); + + final ActorSystem taskManagerSystem; + try { + Tuple2 address = new Tuple2(taskManagerHostname, actorSystemPort); + Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address)); + LOG.debug("Using akka configuration\n " + akkaConfig); + taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig); + } catch (Throwable t) { + if (t instanceof org.jboss.netty.channel.ChannelException) { + Throwable cause = t.getCause(); + if (cause != null && t.getCause() instanceof java.net.BindException) { + String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort); + throw new IOException("Unable to bind TaskManager actor system to address " + + address + " - " + cause.getMessage(), t); + } + } + throw new Exception("Could not create TaskManager actor system", t); + } + + // start akka rpc service based on actor system + final Timeout timeout = new Timeout(AkkaUtils.getTimeout(configuration).toMillis(), TimeUnit.MILLISECONDS); + final AkkaRpcService akkaRpcService = new AkkaRpcService(taskManagerSystem, timeout); + + // start high availability service to implement getResourceManagerLeaderRetriever method only + final HighAvailabilityServices haServices = new HighAvailabilityServices() { + @Override + public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception { + return LeaderRetrievalUtils.createLeaderRetrievalService(configuration); + } + + @Override + public LeaderElectionService getResourceManagerLeaderElectionService() throws Exception { + return null; + } + + @Override + public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception { + return null; + } + }; + + // start all the TaskManager services (network stack, library cache, ...) + // and the TaskManager actor + try { + LOG.info("Starting TaskManager actor"); + TaskExecutor taskExecutor = startTaskManagerComponentsAndActor( + configuration, + resourceID, + akkaRpcService, + taskManagerHostname, + haServices, + false); + + taskExecutor.start(); + + // if desired, start the logging daemon that periodically logs the memory usage information + if (LOG.isInfoEnabled() && configuration.getBoolean( + ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD, + ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) { + LOG.info("Starting periodic memory usage logger"); + + long interval = configuration.getLong( + ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS, + ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS); + + MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem); + logger.start(); + } + + // block until everything is done + taskManagerSystem.awaitTermination(); + } catch (Throwable t) { + LOG.error("Error while starting up taskManager", t); + try { + taskManagerSystem.shutdown(); + } catch (Throwable tt) { + LOG.warn("Could not cleanly shut down actor system", tt); + } + throw t; + } + } + + // -------------------------------------------------------------------------- + // Starting and running the TaskManager + // -------------------------------------------------------------------------- + + /** + * @param configuration The configuration for the TaskManager. + * @param resourceID The id of the resource which the task manager will run on. + * @param rpcService The rpc service which is used to start and connect to the TaskManager RpcEndpoint . + * @param taskManagerHostname The hostname/address that describes the TaskManager's data location. + * @param haServices Optionally, a high availability service can be provided. If none is given, + * then a HighAvailabilityServices is constructed from the configuration. + * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack. + * @return An ActorRef to the TaskManager actor. + * @throws org.apache.flink.configuration.IllegalConfigurationException Thrown, if the given config contains illegal values. + * @throws java.io.IOException Thrown, if any of the I/O components (such as buffer pools, + * I/O manager, ...) cannot be properly started. + * @throws java.lang.Exception Thrown is some other error occurs while parsing the configuration + * or starting the TaskManager components. + */ + public static TaskExecutor startTaskManagerComponentsAndActor( + Configuration configuration, + ResourceID resourceID, + RpcService rpcService, + String taskManagerHostname, + HighAvailabilityServices haServices, + boolean localTaskManagerCommunication) throws Exception { + + final TaskExecutorConfiguration taskExecutorConfig = parseTaskManagerConfiguration( + configuration, taskManagerHostname, localTaskManagerCommunication); + + MemoryType memType = taskExecutorConfig.getNetworkConfig().memoryType(); + + // pre-start checks + checkTempDirs(taskExecutorConfig.getTmpDirPaths()); + + ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool()); + + // we start the network first, to make sure it can allocate its buffers first + final NetworkEnvironment network = new NetworkEnvironment( + executionContext, + taskExecutorConfig.getTimeout(), + taskExecutorConfig.getNetworkConfig(), + taskExecutorConfig.getConnectionInfo()); + + // computing the amount of memory to use depends on how much memory is available + // it strictly needs to happen AFTER the network stack has been initialized + + // check if a value has been configured + long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L); + checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory, + ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, + "MemoryManager needs at least one MB of memory. " + + "If you leave this config parameter empty, the system automatically " + + "pick a fraction of the available memory."); + + final long memorySize; + boolean preAllocateMemory = configuration.getBoolean( + ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY, + ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE); + if (configuredMemory > 0) { + if (preAllocateMemory) { + LOG.info("Using {} MB for managed memory." , configuredMemory); + } else { + LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory); + } + memorySize = configuredMemory << 20; // megabytes to bytes + } else { + float fraction = configuration.getFloat( + ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY, + ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION); + checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction, + ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY, + "MemoryManager fraction of the free memory must be between 0.0 and 1.0"); + + if (memType == MemoryType.HEAP) { + long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction); + if (preAllocateMemory) { + LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." , + fraction , relativeMemSize >> 20); + } else { + LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " + + "memory will be allocated lazily." , fraction , relativeMemSize >> 20); + } + memorySize = relativeMemSize; + } else if (memType == MemoryType.OFF_HEAP) { + // The maximum heap memory has been adjusted according to the fraction + long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory(); + long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction); + if (preAllocateMemory) { + LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." , + fraction, directMemorySize >> 20); + } else { + LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," + + " memory will be allocated lazily.", fraction, directMemorySize >> 20); + } + memorySize = directMemorySize; + } else { + throw new RuntimeException("No supported memory type detected."); + } + } + + // now start the memory manager + final MemoryManager memoryManager; + try { + memoryManager = new MemoryManager( + memorySize, + taskExecutorConfig.getNumberOfSlots(), + taskExecutorConfig.getNetworkConfig().networkBufferSize(), + memType, + preAllocateMemory); + } catch (OutOfMemoryError e) { + if (memType == MemoryType.HEAP) { + throw new Exception("OutOfMemory error (" + e.getMessage() + + ") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e); + } else if (memType == MemoryType.OFF_HEAP) { + throw new Exception("OutOfMemory error (" + e.getMessage() + + ") while allocating the TaskManager off-heap memory (" + memorySize + + " bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e); + } else { + throw e; + } + } + + // start the I/O manager, it will create some temp directories. + final IOManager ioManager = new IOManagerAsync(taskExecutorConfig.getTmpDirPaths()); + + final TaskExecutor taskExecutor = new TaskExecutor( + taskExecutorConfig, + resourceID, + memoryManager, + ioManager, + network, + taskExecutorConfig.getNumberOfSlots(), + rpcService, + haServices); + + return taskExecutor; + } + + // -------------------------------------------------------------------------- + // Parsing and checking the TaskManager Configuration + // -------------------------------------------------------------------------- + + /** + * Utility method to extract TaskManager config parameters from the configuration and to + * sanity check them. + * + * @param configuration The configuration. + * @param taskManagerHostname The host name under which the TaskManager communicates. + * @param localTaskManagerCommunication True, to skip initializing the network stack. + * Use only in cases where only one task manager runs. + * @return TaskExecutorConfiguration that wrappers InstanceConnectionInfo, NetworkEnvironmentConfiguration, etc. + */ + private static TaskExecutorConfiguration parseTaskManagerConfiguration( + Configuration configuration, + String taskManagerHostname, + boolean localTaskManagerCommunication) throws Exception { + + // ------- read values from the config and check them --------- + // (a lot of them) + + // ----> hosts / ports for communication and data exchange + + int dataport = configuration.getInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, + ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT); + if (dataport == 0) { + dataport = NetUtils.getAvailablePort(); + } + checkConfigParameter(dataport > 0, dataport, ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, + "Leave config parameter empty or use 0 to let the system choose a port automatically."); + + InetAddress taskManagerAddress = InetAddress.getByName(taskManagerHostname); + final InstanceConnectionInfo connectionInfo = new InstanceConnectionInfo(taskManagerAddress, dataport); + + // ----> memory / network stack (shuffles/broadcasts), task slots, temp directories + + // we need this because many configs have been written with a "-1" entry + int slots = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1); + if (slots == -1) { + slots = 1; + } + checkConfigParameter(slots >= 1, slots, ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, + "Number of task slots must be at least one."); + + final int numNetworkBuffers = configuration.getInteger( + ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, + ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS); + checkConfigParameter(numNetworkBuffers > 0, numNetworkBuffers, + ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, ""); + + final int pageSize = configuration.getInteger( + ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY, + ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE); + // check page size of for minimum size + checkConfigParameter(pageSize >= MemoryManager.MIN_PAGE_SIZE, pageSize, + ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY, + "Minimum memory segment size is " + MemoryManager.MIN_PAGE_SIZE); + // check page size for power of two + checkConfigParameter(MathUtils.isPowerOf2(pageSize), pageSize, + ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY, + "Memory segment size must be a power of 2."); + + // check whether we use heap or off-heap memory + final MemoryType memType; + if (configuration.getBoolean(ConfigConstants.TASK_MANAGER_MEMORY_OFF_HEAP_KEY, false)) { + memType = MemoryType.OFF_HEAP; + } else { + memType = MemoryType.HEAP; + } + + // initialize the memory segment factory accordingly + if (memType == MemoryType.HEAP) { + if (!MemorySegmentFactory.initializeIfNotInitialized(HeapMemorySegment.FACTORY)) { + throw new Exception("Memory type is set to heap memory, but memory segment " + + "factory has been initialized for off-heap memory segments"); + } + } else { + if (!MemorySegmentFactory.initializeIfNotInitialized(HybridMemorySegment.FACTORY)) { + throw new Exception("Memory type is set to off-heap memory, but memory segment " + + "factory has been initialized for heap memory segments"); + } + } + + final String[] tmpDirs = configuration.getString( + ConfigConstants.TASK_MANAGER_TMP_DIR_KEY, + ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH).split(",|" + File.pathSeparator); + + final NettyConfig nettyConfig; + if (!localTaskManagerCommunication) { + nettyConfig = new NettyConfig(connectionInfo.address(), connectionInfo.dataPort(), pageSize, slots, configuration); + } else { + nettyConfig = null; + } + + // Default spill I/O mode for intermediate results + final String syncOrAsync = configuration.getString( + ConfigConstants.TASK_MANAGER_NETWORK_DEFAULT_IO_MODE, + ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_DEFAULT_IO_MODE); + + final IOMode ioMode; + if (syncOrAsync.equals("async")) { + ioMode = IOManager.IOMode.ASYNC; + } else { + ioMode = IOManager.IOMode.SYNC; + } + + final int queryServerPort = configuration.getInteger( + ConfigConstants.QUERYABLE_STATE_SERVER_PORT, + ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_PORT); + + final int queryServerNetworkThreads = configuration.getInteger( + ConfigConstants.QUERYABLE_STATE_SERVER_NETWORK_THREADS, + ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_NETWORK_THREADS); + + final int queryServerQueryThreads = configuration.getInteger( + ConfigConstants.QUERYABLE_STATE_SERVER_QUERY_THREADS, + ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_QUERY_THREADS); + + final NetworkEnvironmentConfiguration networkConfig = new NetworkEnvironmentConfiguration( + numNetworkBuffers, + pageSize, + memType, + ioMode, + queryServerPort, + queryServerNetworkThreads, + queryServerQueryThreads, + localTaskManagerCommunication ? Option.empty() : new Some<>(nettyConfig), + new Tuple2<>(500, 3000)); + + // ----> timeouts, library caching, profiling + + final FiniteDuration timeout; + try { + timeout = AkkaUtils.getTimeout(configuration); + } catch (Exception e) { + throw new IllegalArgumentException( + "Invalid format for '" + ConfigConstants.AKKA_ASK_TIMEOUT + + "'.Use formats like '50 s' or '1 min' to specify the timeout."); + } + LOG.info("Messages between TaskManager and JobManager have a max timeout of " + timeout); + + final long cleanupInterval = configuration.getLong( + ConfigConstants.LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL, + ConfigConstants.DEFAULT_LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL) * 1000; + + final FiniteDuration finiteRegistrationDuration; + try { + Duration maxRegistrationDuration = Duration.create(configuration.getString( + ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION, + ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_DURATION)); + if (maxRegistrationDuration.isFinite()) { + finiteRegistrationDuration = new FiniteDuration(maxRegistrationDuration.toSeconds(), TimeUnit.SECONDS); + } else { + finiteRegistrationDuration = null; + } + } catch (NumberFormatException e) { + throw new IllegalArgumentException("Invalid format for parameter " + + ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION, e); + } + + final FiniteDuration initialRegistrationPause; + try { + Duration pause = Duration.create(configuration.getString( + ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, + ConfigConstants.DEFAULT_TASK_MANAGER_INITIAL_REGISTRATION_PAUSE)); + if (pause.isFinite()) { + initialRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS); + } else { + throw new IllegalArgumentException("The initial registration pause must be finite: " + pause); + } + } catch (NumberFormatException e) { + throw new IllegalArgumentException("Invalid format for parameter " + + ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e); + } + + final FiniteDuration maxRegistrationPause; + try { + Duration pause = Duration.create(configuration.getString( + ConfigConstants.TASK_MANAGER_MAX_REGISTARTION_PAUSE, + ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_PAUSE)); + if (pause.isFinite()) { + maxRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS); + } else { + throw new IllegalArgumentException("The maximum registration pause must be finite: " + pause); + } + } catch (NumberFormatException e) { + throw new IllegalArgumentException("Invalid format for parameter " + + ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e); + } + + final FiniteDuration refusedRegistrationPause; + try { + Duration pause = Duration.create(configuration.getString( + ConfigConstants.TASK_MANAGER_REFUSED_REGISTRATION_PAUSE, + ConfigConstants.DEFAULT_TASK_MANAGER_REFUSED_REGISTRATION_PAUSE)); + if (pause.isFinite()) { + refusedRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS); + } else { + throw new IllegalArgumentException("The refused registration pause must be finite: " + pause); + } + } catch (NumberFormatException e) { + throw new IllegalArgumentException("Invalid format for parameter " + + ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e); + } + + return new TaskExecutorConfiguration( + tmpDirs, + cleanupInterval, + connectionInfo, + networkConfig, + timeout, + finiteRegistrationDuration, + slots, + configuration, + initialRegistrationPause, + maxRegistrationPause, + refusedRegistrationPause); + } + + /** + * Validates a condition for a config parameter and displays a standard exception, if the + * the condition does not hold. + * + * @param condition The condition that must hold. If the condition is false, an exception is thrown. + * @param parameter The parameter value. Will be shown in the exception message. + * @param name The name of the config parameter. Will be shown in the exception message. + * @param errorMessage The optional custom error message to append to the exception message. + */ + private static void checkConfigParameter( + boolean condition, + Object parameter, + String name, + String errorMessage) { + if (!condition) { + throw new IllegalConfigurationException("Invalid configuration value for " + name + " : " + parameter + " - " + errorMessage); + } + } + + /** + * Validates that all the directories denoted by the strings do actually exist, are proper + * directories (not files), and are writable. + * + * @param tmpDirs The array of directory paths to check. + * @throws Exception Thrown if any of the directories does not exist or is not writable + * or is a file, rather than a directory. + */ + private static void checkTempDirs(String[] tmpDirs) throws IOException { + for (String dir : tmpDirs) { + if (dir != null && !dir.equals("")) { + File file = new File(dir); + if (!file.exists()) { + throw new IOException("Temporary file directory " + file.getAbsolutePath() + " does not exist."); + } + if (!file.isDirectory()) { + throw new IOException("Temporary file directory " + file.getAbsolutePath() + " is not a directory."); + } + if (!file.canWrite()) { + throw new IOException("Temporary file directory " + file.getAbsolutePath() + " is not writable."); + } + + if (LOG.isInfoEnabled()) { + long totalSpaceGb = file.getTotalSpace() >> 30; + long usableSpaceGb = file.getUsableSpace() >> 30; + double usablePercentage = (double)usableSpaceGb / totalSpaceGb * 100; + String path = file.getAbsolutePath(); + LOG.info(String.format("Temporary file directory '%s': total %d GB, " + "usable %d GB (%.2f%% usable)", + path, totalSpaceGb, usableSpaceGb, usablePercentage)); + } + } else { + throw new IllegalArgumentException("Temporary file directory #$id is null."); + } + } + } + + // ------------------------------------------------------------------------ + // Properties + // ------------------------------------------------------------------------ + + public ResourceID getResourceID() { + return resourceID; + } + // ------------------------------------------------------------------------ - // Error handling + // Error Handling // ------------------------------------------------------------------------ /** * Notifies the TaskExecutor that a fatal error has occurred and it cannot proceed. * This method should be used when asynchronous threads want to notify the * TaskExecutor of a fatal error. - * + * * @param t The exception describing the fatal error */ void onFatalErrorAsync(final Throwable t) { @@ -141,7 +787,7 @@ public void run() { /** * Notifies the TaskExecutor that a fatal error has occurred and it cannot proceed. * This method must only be called from within the TaskExecutor's main thread. - * + * * @param t The exception describing the fatal error */ void onFatalError(Throwable t) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorConfiguration.java new file mode 100644 index 0000000000000..32484e114748d --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorConfiguration.java @@ -0,0 +1,151 @@ +/* + * 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.rpc.taskexecutor; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.instance.InstanceConnectionInfo; +import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration; + +import scala.concurrent.duration.FiniteDuration; + +import java.io.Serializable; +import java.util.concurrent.TimeUnit; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * {@link TaskExecutor} Configuration + */ +public class TaskExecutorConfiguration implements Serializable { + + private static final long serialVersionUID = 1L; + + private final String[] tmpDirPaths; + + private final long cleanupInterval; + + private final int numberOfSlots; + + private final Configuration configuration; + + private final FiniteDuration timeout; + private final FiniteDuration maxRegistrationDuration; + private final FiniteDuration initialRegistrationPause; + private final FiniteDuration maxRegistrationPause; + private final FiniteDuration refusedRegistrationPause; + + private final NetworkEnvironmentConfiguration networkConfig; + + private final InstanceConnectionInfo connectionInfo; + + public TaskExecutorConfiguration( + String[] tmpDirPaths, + long cleanupInterval, + InstanceConnectionInfo connectionInfo, + NetworkEnvironmentConfiguration networkConfig, + FiniteDuration timeout, + FiniteDuration maxRegistrationDuration, + int numberOfSlots, + Configuration configuration) { + + this (tmpDirPaths, + cleanupInterval, + connectionInfo, + networkConfig, + timeout, + maxRegistrationDuration, + numberOfSlots, + configuration, + new FiniteDuration(500, TimeUnit.MILLISECONDS), + new FiniteDuration(30, TimeUnit.SECONDS), + new FiniteDuration(10, TimeUnit.SECONDS)); + } + + public TaskExecutorConfiguration( + String[] tmpDirPaths, + long cleanupInterval, + InstanceConnectionInfo connectionInfo, + NetworkEnvironmentConfiguration networkConfig, + FiniteDuration timeout, + FiniteDuration maxRegistrationDuration, + int numberOfSlots, + Configuration configuration, + FiniteDuration initialRegistrationPause, + FiniteDuration maxRegistrationPause, + FiniteDuration refusedRegistrationPause) { + + this.tmpDirPaths = checkNotNull(tmpDirPaths); + this.cleanupInterval = checkNotNull(cleanupInterval); + this.connectionInfo = checkNotNull(connectionInfo); + this.networkConfig = checkNotNull(networkConfig); + this.timeout = checkNotNull(timeout); + this.maxRegistrationDuration = maxRegistrationDuration; + this.numberOfSlots = checkNotNull(numberOfSlots); + this.configuration = checkNotNull(configuration); + this.initialRegistrationPause = checkNotNull(initialRegistrationPause); + this.maxRegistrationPause = checkNotNull(maxRegistrationPause); + this.refusedRegistrationPause = checkNotNull(refusedRegistrationPause); + } + + // -------------------------------------------------------------------------------------------- + // Properties + // -------------------------------------------------------------------------------------------- + + public String[] getTmpDirPaths() { + return tmpDirPaths; + } + + public long getCleanupInterval() { + return cleanupInterval; + } + + public InstanceConnectionInfo getConnectionInfo() { return connectionInfo; } + + public NetworkEnvironmentConfiguration getNetworkConfig() { return networkConfig; } + + public FiniteDuration getTimeout() { + return timeout; + } + + public FiniteDuration getMaxRegistrationDuration() { + return maxRegistrationDuration; + } + + public int getNumberOfSlots() { + return numberOfSlots; + } + + public Configuration getConfiguration() { + return configuration; + } + + public FiniteDuration getInitialRegistrationPause() { + return initialRegistrationPause; + } + + public FiniteDuration getMaxRegistrationPause() { + return maxRegistrationPause; + } + + public FiniteDuration getRefusedRegistrationPause() { + return refusedRegistrationPause; + } + +} + diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java index b831ead5c9a11..25a670c405a5d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.rpc.taskexecutor; +import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.highavailability.NonHaServices; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; @@ -49,9 +50,9 @@ public void testImmediatelyRegistersIfLeaderIsKnown() throws Exception { rpc.registerGateway(resourceManagerAddress, rmGateway); NonHaServices haServices = new NonHaServices(resourceManagerAddress); - TaskExecutor taskManager = new TaskExecutor(rpc, haServices, resourceID); + TaskExecutor taskManager = TaskExecutor.startTaskManagerComponentsAndActor( + new Configuration(), resourceID, rpc, "localhost", haServices, true); String taskManagerAddress = taskManager.getAddress(); - taskManager.start(); verify(rmGateway, timeout(5000)).registerTaskExecutor( @@ -84,7 +85,8 @@ public void testTriggerRegistrationOnLeaderChange() throws Exception { TestingHighAvailabilityServices haServices = new TestingHighAvailabilityServices(); haServices.setResourceManagerLeaderRetriever(testLeaderService); - TaskExecutor taskManager = new TaskExecutor(rpc, haServices, resourceID); + TaskExecutor taskManager = TaskExecutor.startTaskManagerComponentsAndActor( + new Configuration(), resourceID, rpc, "localhost", haServices, true); String taskManagerAddress = taskManager.getAddress(); taskManager.start(); From 278f60923321ba2084f0ccfb345520411ada33df Mon Sep 17 00:00:00 2001 From: Kurt Young Date: Thu, 18 Aug 2016 15:48:30 +0800 Subject: [PATCH 20/50] [FLINK-4347][cluster management] Implement SlotManager core This closes #2388 --- .../runtime/clusterframework/SlotManager.java | 525 +++++++++++++++++ .../clusterframework/types/ResourceID.java | 4 +- .../types/ResourceProfile.java | 5 + .../clusterframework/types/ResourceSlot.java | 66 +++ .../clusterframework/types/SlotID.java | 14 +- .../rpc/resourcemanager/SlotRequest.java | 51 +- .../runtime/rpc/taskexecutor/SlotReport.java | 56 ++ .../runtime/rpc/taskexecutor/SlotStatus.java | 129 +++++ .../clusterframework/SlotManagerTest.java | 540 ++++++++++++++++++ 9 files changed, 1382 insertions(+), 8 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/SlotManager.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotReport.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotStatus.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/SlotManagerTest.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/SlotManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/SlotManager.java new file mode 100644 index 0000000000000..cc140a1abd287 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/SlotManager.java @@ -0,0 +1,525 @@ +/* + * 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.clusterframework; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.clusterframework.types.ResourceSlot; +import org.apache.flink.runtime.clusterframework.types.SlotID; +import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.rpc.resourcemanager.SlotRequest; +import org.apache.flink.runtime.rpc.taskexecutor.SlotReport; +import org.apache.flink.runtime.rpc.taskexecutor.SlotStatus; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.Map; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * SlotManager is responsible for receiving slot requests and do slot allocations. It allows to request + * slots from registered TaskManagers and issues container allocation requests in case of there are not + * enough available slots. Besides, it should sync its slot allocation with TaskManager's heartbeat. + *

+ * The main operation principle of SlotManager is: + *

    + *
  • 1. All slot allocation status should be synced with TaskManager, which is the ground truth.
  • + *
  • 2. All slots that have registered must be tracked, either by free pool or allocated pool.
  • + *
  • 3. All slot requests will be handled by best efforts, there is no guarantee that one request will be + * fulfilled in time or correctly allocated. Conflicts or timeout or some special error will happen, it should + * be handled outside SlotManager. SlotManager will make each decision based on the information it currently + * holds.
  • + *
+ * IMPORTANT: This class is Not Thread-safe. + */ +public abstract class SlotManager { + + private static final Logger LOG = LoggerFactory.getLogger(SlotManager.class); + + /** Gateway to communicate with ResourceManager */ + private final ResourceManagerGateway resourceManagerGateway; + + /** All registered slots, including free and allocated slots */ + private final Map> registeredSlots; + + /** All pending slot requests, waiting available slots to fulfil */ + private final Map pendingSlotRequests; + + /** All free slots that can be used to be allocated */ + private final Map freeSlots; + + /** All allocations, we can lookup allocations either by SlotID or AllocationID */ + private final AllocationMap allocationMap; + + public SlotManager(ResourceManagerGateway resourceManagerGateway) { + this.resourceManagerGateway = checkNotNull(resourceManagerGateway); + this.registeredSlots = new HashMap<>(16); + this.pendingSlotRequests = new LinkedHashMap<>(16); + this.freeSlots = new HashMap<>(16); + this.allocationMap = new AllocationMap(); + } + + // ------------------------------------------------------------------------ + // slot managements + // ------------------------------------------------------------------------ + + /** + * Request a slot with requirements, we may either fulfill the request or pending it. Trigger container + * allocation if we don't have enough resource. If we have free slot which can match the request, record + * this allocation and forward the request to TaskManager through ResourceManager (we want this done by + * RPC's main thread to avoid race condition). + * + * @param request The detailed request of the slot + */ + public void requestSlot(final SlotRequest request) { + if (isRequestDuplicated(request)) { + LOG.warn("Duplicated slot request, AllocationID:{}", request.getAllocationId()); + return; + } + + // try to fulfil the request with current free slots + ResourceSlot slot = chooseSlotToUse(request, freeSlots); + if (slot != null) { + LOG.info("Assigning SlotID({}) to AllocationID({}), JobID:{}", slot.getSlotId(), + request.getAllocationId(), request.getJobId()); + + // record this allocation in bookkeeping + allocationMap.addAllocation(slot.getSlotId(), request.getAllocationId()); + + // remove selected slot from free pool + freeSlots.remove(slot.getSlotId()); + + // TODO: send slot request to TaskManager + } else { + LOG.info("Cannot fulfil slot request, try to allocate a new container for it, " + + "AllocationID:{}, JobID:{}", request.getAllocationId(), request.getJobId()); + allocateContainer(request.getResourceProfile()); + pendingSlotRequests.put(request.getAllocationId(), request); + } + } + + /** + * Sync slot status with TaskManager's SlotReport. + */ + public void updateSlotStatus(final SlotReport slotReport) { + for (SlotStatus slotStatus : slotReport.getSlotsStatus()) { + updateSlotStatus(slotStatus); + } + } + + /** + * The slot request to TaskManager may be either failed by rpc communication (timeout, network error, etc.) + * or really rejected by TaskManager. We shall retry this request by: + *
    + *
  • 1. verify and clear all the previous allocate information for this request + *
  • 2. try to request slot again + *
+ *

+ * This may cause some duplicate allocation, e.g. the slot request to TaskManager is successful but the response + * is lost somehow, so we may request a slot in another TaskManager, this causes two slots assigned to one request, + * but it can be taken care of by rejecting registration at JobManager. + * + * @param originalRequest The original slot request + * @param slotId The target SlotID + */ + public void handleSlotRequestFailedAtTaskManager(final SlotRequest originalRequest, final SlotID slotId) { + final AllocationID originalAllocationId = originalRequest.getAllocationId(); + LOG.info("Slot request failed at TaskManager, SlotID:{}, AllocationID:{}, JobID:{}", + slotId, originalAllocationId, originalRequest.getJobId()); + + // verify the allocation info before we do anything + if (freeSlots.containsKey(slotId)) { + // this slot is currently empty, no need to de-allocate it from our allocations + LOG.info("Original slot is somehow empty, retrying this request"); + + // before retry, we should double check whether this request was allocated by some other ways + if (!allocationMap.isAllocated(originalAllocationId)) { + requestSlot(originalRequest); + } else { + LOG.info("The failed request has somehow been allocated, SlotID:{}", + allocationMap.getSlotID(originalAllocationId)); + } + } else if (allocationMap.isAllocated(slotId)) { + final AllocationID currentAllocationId = allocationMap.getAllocationID(slotId); + + // check whether we have an agreement on whom this slot belongs to + if (originalAllocationId.equals(currentAllocationId)) { + LOG.info("De-allocate this request and retry"); + allocationMap.removeAllocation(currentAllocationId); + + // put this slot back to free pool + ResourceSlot slot = checkNotNull(getRegisteredSlot(slotId)); + freeSlots.put(slotId, slot); + + // retry the request + requestSlot(originalRequest); + } else { + // the slot is taken by someone else, no need to de-allocate it from our allocations + LOG.info("Original slot is taken by someone else, current AllocationID:{}", currentAllocationId); + + // before retry, we should double check whether this request was allocated by some other ways + if (!allocationMap.isAllocated(originalAllocationId)) { + requestSlot(originalRequest); + } else { + LOG.info("The failed request is somehow been allocated, SlotID:{}", + allocationMap.getSlotID(originalAllocationId)); + } + } + } else { + LOG.error("BUG! {} is neither in free pool nor in allocated pool", slotId); + } + } + + /** + * Callback for TaskManager failures. In case that a TaskManager fails, we have to clean up all its slots. + * + * @param resourceId The ResourceID of the TaskManager + */ + public void notifyTaskManagerFailure(final ResourceID resourceId) { + LOG.info("Resource:{} been notified failure", resourceId); + final Map slotIdsToRemove = registeredSlots.remove(resourceId); + if (slotIdsToRemove != null) { + for (SlotID slotId : slotIdsToRemove.keySet()) { + LOG.info("Removing Slot:{} upon resource failure", slotId); + if (freeSlots.containsKey(slotId)) { + freeSlots.remove(slotId); + } else if (allocationMap.isAllocated(slotId)) { + allocationMap.removeAllocation(slotId); + } else { + LOG.error("BUG! {} is neither in free pool nor in allocated pool", slotId); + } + } + } + } + + // ------------------------------------------------------------------------ + // internal behaviors + // ------------------------------------------------------------------------ + + /** + * Update slot status based on TaskManager's report. There are mainly two situations when we receive the report: + *

    + *
  • 1. The slot is newly registered.
  • + *
  • 2. The slot has registered, it contains its current status.
  • + *
+ *

+ * Regarding 1: It's fairly simple, we just record this slot's status, and trigger schedule if slot is empty. + *

+ * Regarding 2: It will cause some weird situation since we may have some time-gap on how the slot's status really + * is. We may have some updates on the slot's allocation, but it doesn't reflected by TaskManager's heartbeat yet, + * and we may make some wrong decision if we cannot guarantee we have the exact status about all the slots. So + * the principle here is: We always trust TaskManager's heartbeat, we will correct our information based on that + * and take next action based on the diff between our information and heartbeat status. + * + * @param reportedStatus Reported slot status + */ + void updateSlotStatus(final SlotStatus reportedStatus) { + final SlotID slotId = reportedStatus.getSlotID(); + final ResourceSlot slot = new ResourceSlot(slotId, reportedStatus.getProfiler()); + + if (registerNewSlot(slot)) { + // we have a newly registered slot + LOG.info("New slot appeared, SlotID:{}, AllocationID:{}", slotId, reportedStatus.getAllocationID()); + + if (reportedStatus.getAllocationID() != null) { + // slot in use, record this in bookkeeping + allocationMap.addAllocation(slotId, reportedStatus.getAllocationID()); + } else { + handleFreeSlot(new ResourceSlot(slotId, reportedStatus.getProfiler())); + } + } else { + // slot exists, update current information + if (reportedStatus.getAllocationID() != null) { + // slot is reported in use + final AllocationID reportedAllocationId = reportedStatus.getAllocationID(); + + // check whether we also thought this slot is in use + if (allocationMap.isAllocated(slotId)) { + // we also think that slot is in use, check whether the AllocationID matches + final AllocationID currentAllocationId = allocationMap.getAllocationID(slotId); + + if (!reportedAllocationId.equals(currentAllocationId)) { + LOG.info("Slot allocation info mismatch! SlotID:{}, current:{}, reported:{}", + slotId, currentAllocationId, reportedAllocationId); + + // seems we have a disagreement about the slot assignments, need to correct it + allocationMap.removeAllocation(slotId); + allocationMap.addAllocation(slotId, reportedAllocationId); + } + } else { + LOG.info("Slot allocation info mismatch! SlotID:{}, current:null, reported:{}", + slotId, reportedAllocationId); + + // we thought the slot is free, should correct this information + allocationMap.addAllocation(slotId, reportedStatus.getAllocationID()); + + // remove this slot from free slots pool + freeSlots.remove(slotId); + } + } else { + // slot is reported empty + + // check whether we also thought this slot is empty + if (allocationMap.isAllocated(slotId)) { + LOG.info("Slot allocation info mismatch! SlotID:{}, current:{}, reported:null", + slotId, allocationMap.getAllocationID(slotId)); + + // we thought the slot is in use, correct it + allocationMap.removeAllocation(slotId); + + // we have a free slot! + handleFreeSlot(new ResourceSlot(slotId, reportedStatus.getProfiler())); + } + } + } + } + + /** + * When we have a free slot, try to fulfill the pending request first. If any request can be fulfilled, + * record this allocation in bookkeeping and send slot request to TaskManager, else we just add this slot + * to the free pool. + * + * @param freeSlot The free slot + */ + private void handleFreeSlot(final ResourceSlot freeSlot) { + SlotRequest chosenRequest = chooseRequestToFulfill(freeSlot, pendingSlotRequests); + + if (chosenRequest != null) { + pendingSlotRequests.remove(chosenRequest.getAllocationId()); + + LOG.info("Assigning SlotID({}) to AllocationID({}), JobID:{}", freeSlot.getSlotId(), + chosenRequest.getAllocationId(), chosenRequest.getJobId()); + allocationMap.addAllocation(freeSlot.getSlotId(), chosenRequest.getAllocationId()); + + // TODO: send slot request to TaskManager + } else { + freeSlots.put(freeSlot.getSlotId(), freeSlot); + } + } + + /** + * Check whether the request is duplicated. We use AllocationID to identify slot request, for each + * formerly received slot request, it is either in pending list or already been allocated. + * + * @param request The slot request + * @return true if the request is duplicated + */ + private boolean isRequestDuplicated(final SlotRequest request) { + final AllocationID allocationId = request.getAllocationId(); + return pendingSlotRequests.containsKey(allocationId) + || allocationMap.isAllocated(allocationId); + } + + /** + * Try to register slot, and tell if this slot is newly registered. + * + * @param slot The ResourceSlot which will be checked and registered + * @return true if we meet a new slot + */ + private boolean registerNewSlot(final ResourceSlot slot) { + final SlotID slotId = slot.getSlotId(); + final ResourceID resourceId = slotId.getResourceID(); + if (!registeredSlots.containsKey(resourceId)) { + registeredSlots.put(resourceId, new HashMap()); + } + return registeredSlots.get(resourceId).put(slotId, slot) == null; + } + + private ResourceSlot getRegisteredSlot(final SlotID slotId) { + final ResourceID resourceId = slotId.getResourceID(); + if (!registeredSlots.containsKey(resourceId)) { + return null; + } + return registeredSlots.get(resourceId).get(slotId); + } + + // ------------------------------------------------------------------------ + // Framework specific behavior + // ------------------------------------------------------------------------ + + /** + * Choose a slot to use among all free slots, the behavior is framework specified. + * + * @param request The slot request + * @param freeSlots All slots which can be used + * @return The slot we choose to use, null if we did not find a match + */ + protected abstract ResourceSlot chooseSlotToUse(final SlotRequest request, + final Map freeSlots); + + /** + * Choose a pending request to fulfill when we have a free slot, the behavior is framework specified. + * + * @param offeredSlot The free slot + * @param pendingRequests All the pending slot requests + * @return The chosen SlotRequest, null if we did not find a match + */ + protected abstract SlotRequest chooseRequestToFulfill(final ResourceSlot offeredSlot, + final Map pendingRequests); + + /** + * The framework specific code for allocating a container for specified resource profile. + * + * @param resourceProfile The resource profile + */ + protected abstract void allocateContainer(final ResourceProfile resourceProfile); + + + // ------------------------------------------------------------------------ + // Helper classes + // ------------------------------------------------------------------------ + + /** + * We maintain all the allocations with SlotID and AllocationID. We are able to get or remove the allocation info + * either by SlotID or AllocationID. + */ + private static class AllocationMap { + + /** All allocated slots (by SlotID) */ + private final Map allocatedSlots; + + /** All allocated slots (by AllocationID), it'a a inverse view of allocatedSlots */ + private final Map allocatedSlotsByAllocationId; + + AllocationMap() { + this.allocatedSlots = new HashMap<>(16); + this.allocatedSlotsByAllocationId = new HashMap<>(16); + } + + /** + * Add a allocation + * + * @param slotId The slot id + * @param allocationId The allocation id + */ + void addAllocation(final SlotID slotId, final AllocationID allocationId) { + allocatedSlots.put(slotId, allocationId); + allocatedSlotsByAllocationId.put(allocationId, slotId); + } + + /** + * De-allocation with slot id + * + * @param slotId The slot id + */ + void removeAllocation(final SlotID slotId) { + if (allocatedSlots.containsKey(slotId)) { + final AllocationID allocationId = allocatedSlots.get(slotId); + allocatedSlots.remove(slotId); + allocatedSlotsByAllocationId.remove(allocationId); + } + } + + /** + * De-allocation with allocation id + * + * @param allocationId The allocation id + */ + void removeAllocation(final AllocationID allocationId) { + if (allocatedSlotsByAllocationId.containsKey(allocationId)) { + SlotID slotId = allocatedSlotsByAllocationId.get(allocationId); + allocatedSlotsByAllocationId.remove(allocationId); + allocatedSlots.remove(slotId); + } + } + + /** + * Check whether allocation exists by slot id + * + * @param slotId The slot id + * @return true if the allocation exists + */ + boolean isAllocated(final SlotID slotId) { + return allocatedSlots.containsKey(slotId); + } + + /** + * Check whether allocation exists by allocation id + * + * @param allocationId The allocation id + * @return true if the allocation exists + */ + boolean isAllocated(final AllocationID allocationId) { + return allocatedSlotsByAllocationId.containsKey(allocationId); + } + + AllocationID getAllocationID(final SlotID slotId) { + return allocatedSlots.get(slotId); + } + + SlotID getSlotID(final AllocationID allocationId) { + return allocatedSlotsByAllocationId.get(allocationId); + } + + public int size() { + return allocatedSlots.size(); + } + } + + // ------------------------------------------------------------------------ + // Testing utilities + // ------------------------------------------------------------------------ + + @VisibleForTesting + boolean isAllocated(final SlotID slotId) { + return allocationMap.isAllocated(slotId); + } + + @VisibleForTesting + boolean isAllocated(final AllocationID allocationId) { + return allocationMap.isAllocated(allocationId); + } + + /** + * Add free slots directly to the free pool, this will not trigger pending requests allocation + * + * @param slot The resource slot + */ + @VisibleForTesting + void addFreeSlot(final ResourceSlot slot) { + final ResourceID resourceId = slot.getResourceID(); + final SlotID slotId = slot.getSlotId(); + + if (!registeredSlots.containsKey(resourceId)) { + registeredSlots.put(resourceId, new HashMap()); + } + registeredSlots.get(resourceId).put(slot.getSlotId(), slot); + freeSlots.put(slotId, slot); + } + + @VisibleForTesting + int getAllocatedSlotCount() { + return allocationMap.size(); + } + + @VisibleForTesting + int getFreeSlotCount() { + return freeSlots.size(); + } + + @VisibleForTesting + int getPendingRequestCount() { + return pendingSlotRequests.size(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceID.java index 8cf9ccb1fa53a..6b8a03720845b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceID.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceID.java @@ -63,9 +63,7 @@ public final int hashCode() { @Override public String toString() { - return "ResourceID{" + - "resourceId='" + resourceId + '\'' + - '}'; + return resourceId; } /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java index cbe709f946a77..ff1c4bfd3b8b3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java @@ -40,6 +40,11 @@ public ResourceProfile(double cpuCores, long memoryInMB) { this.memoryInMB = memoryInMB; } + public ResourceProfile(ResourceProfile other) { + this.cpuCores = other.cpuCores; + this.memoryInMB = other.memoryInMB; + } + /** * Get the cpu cores needed * @return The cpu cores, 1.0 means a full cpu thread diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java new file mode 100644 index 0000000000000..8a6db5f5cfc8a --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java @@ -0,0 +1,66 @@ +/* + * 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.clusterframework.types; + +import java.io.Serializable; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A ResourceSlot represents a slot located in TaskManager from ResourceManager's view. It has a unique + * identification and resource profile which we can compare to the resource request. + */ +public class ResourceSlot implements ResourceIDRetrievable, Serializable { + + private static final long serialVersionUID = -5853720153136840674L; + + /** The unique identification of this slot */ + private final SlotID slotId; + + /** The resource profile of this slot */ + private final ResourceProfile resourceProfile; + + public ResourceSlot(SlotID slotId, ResourceProfile resourceProfile) { + this.slotId = checkNotNull(slotId); + this.resourceProfile = checkNotNull(resourceProfile); + } + + @Override + public ResourceID getResourceID() { + return slotId.getResourceID(); + } + + public SlotID getSlotId() { + return slotId; + } + + public ResourceProfile getResourceProfile() { + return resourceProfile; + } + + /** + * Check whether required resource profile can be matched by this slot. + * + * @param required The required resource profile + * @return true if requirement can be matched + */ + public boolean isMatchingRequirement(ResourceProfile required) { + return resourceProfile.isMatching(required); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/SlotID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/SlotID.java index d1b072defda76..e831a5dd184be 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/SlotID.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/SlotID.java @@ -75,9 +75,15 @@ public int hashCode() { @Override public String toString() { - return "SlotID{" + - "resourceId=" + resourceId + - ", slotId=" + slotId + - '}'; + return resourceId + "_" + slotId; + } + + /** + * Generate a random slot id. + * + * @return A random slot id. + */ + public static SlotID generate() { + return new SlotID(ResourceID.generate(), 0); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotRequest.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotRequest.java index d8fe2689644ea..74c7c3900784a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotRequest.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotRequest.java @@ -18,8 +18,57 @@ package org.apache.flink.runtime.rpc.resourcemanager; +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; + import java.io.Serializable; -public class SlotRequest implements Serializable{ +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * This describes the requirement of the slot, mainly used by JobManager requesting slot from ResourceManager. + */ +public class SlotRequest implements Serializable { + private static final long serialVersionUID = -6586877187990445986L; + + /** The JobID of the slot requested for */ + private final JobID jobId; + + /** The unique identification of this request */ + private final AllocationID allocationId; + + /** The resource profile of the required slot */ + private final ResourceProfile resourceProfile; + + public SlotRequest(JobID jobId, AllocationID allocationId, ResourceProfile resourceProfile) { + this.jobId = checkNotNull(jobId); + this.allocationId = checkNotNull(allocationId); + this.resourceProfile = checkNotNull(resourceProfile); + } + + /** + * Get the JobID of the slot requested for. + * @return The job id + */ + public JobID getJobId() { + return jobId; + } + + /** + * Get the unique identification of this request + * @return the allocation id + */ + public AllocationID getAllocationId() { + return allocationId; + } + + /** + * Get the resource profile of the desired slot + * @return The resource profile + */ + public ResourceProfile getResourceProfile() { + return resourceProfile; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotReport.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotReport.java new file mode 100644 index 0000000000000..c372ecbadcaa9 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotReport.java @@ -0,0 +1,56 @@ +/* + * 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.rpc.taskexecutor; + +import org.apache.flink.runtime.clusterframework.types.ResourceID; + +import java.io.Serializable; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A report about the current status of all slots of the TaskExecutor, describing + * which slots are available and allocated, and what jobs (JobManagers) the allocated slots + * have been allocated to. + */ +public class SlotReport implements Serializable { + + private static final long serialVersionUID = -3150175198722481689L; + + /** The slots status of the TaskManager */ + private final List slotsStatus; + + /** The resource id which identifies the TaskManager */ + private final ResourceID resourceID; + + public SlotReport(final List slotsStatus, final ResourceID resourceID) { + this.slotsStatus = checkNotNull(slotsStatus); + this.resourceID = checkNotNull(resourceID); + } + + public List getSlotsStatus() { + return slotsStatus; + } + + public ResourceID getResourceID() { + return resourceID; + } + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotStatus.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotStatus.java new file mode 100644 index 0000000000000..e8e208448c9c0 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotStatus.java @@ -0,0 +1,129 @@ +/* + * 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.rpc.taskexecutor; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.clusterframework.types.SlotID; + +import java.io.Serializable; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * This describes the slot current status which located in TaskManager. + */ +public class SlotStatus implements Serializable { + + private static final long serialVersionUID = 5099191707339664493L; + + /** slotID to identify a slot */ + private final SlotID slotID; + + /** the resource profile of the slot */ + private final ResourceProfile profiler; + + /** if the slot is allocated, allocationId identify its allocation; else, allocationId is null */ + private final AllocationID allocationID; + + /** if the slot is allocated, jobId identify which job this slot is allocated to; else, jobId is null */ + private final JobID jobID; + + public SlotStatus(SlotID slotID, ResourceProfile profiler) { + this(slotID, profiler, null, null); + } + + public SlotStatus(SlotID slotID, ResourceProfile profiler, AllocationID allocationID, JobID jobID) { + this.slotID = checkNotNull(slotID, "slotID cannot be null"); + this.profiler = checkNotNull(profiler, "profile cannot be null"); + this.allocationID = allocationID; + this.jobID = jobID; + } + + /** + * Get the unique identification of this slot + * + * @return The slot id + */ + public SlotID getSlotID() { + return slotID; + } + + /** + * Get the resource profile of this slot + * + * @return The resource profile + */ + public ResourceProfile getProfiler() { + return profiler; + } + + /** + * Get the allocation id of this slot + * + * @return The allocation id if this slot is allocated, otherwise null + */ + public AllocationID getAllocationID() { + return allocationID; + } + + /** + * Get the job id of the slot allocated for + * + * @return The job id if this slot is allocated, otherwise null + */ + public JobID getJobID() { + return jobID; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + SlotStatus that = (SlotStatus) o; + + if (!slotID.equals(that.slotID)) { + return false; + } + if (!profiler.equals(that.profiler)) { + return false; + } + if (allocationID != null ? !allocationID.equals(that.allocationID) : that.allocationID != null) { + return false; + } + return jobID != null ? jobID.equals(that.jobID) : that.jobID == null; + + } + + @Override + public int hashCode() { + int result = slotID.hashCode(); + result = 31 * result + profiler.hashCode(); + result = 31 * result + (allocationID != null ? allocationID.hashCode() : 0); + result = 31 * result + (jobID != null ? jobID.hashCode() : 0); + return result; + } + +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/SlotManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/SlotManagerTest.java new file mode 100644 index 0000000000000..2ee280f9034ec --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/SlotManagerTest.java @@ -0,0 +1,540 @@ +/* + * 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.clusterframework; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.clusterframework.types.ResourceSlot; +import org.apache.flink.runtime.clusterframework.types.SlotID; +import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.rpc.resourcemanager.SlotRequest; +import org.apache.flink.runtime.rpc.taskexecutor.SlotStatus; +import org.junit.Before; +import org.junit.Test; + +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; + +public class SlotManagerTest { + + private static final double DEFAULT_TESTING_CPU_CORES = 1.0; + + private static final long DEFAULT_TESTING_MEMORY = 512; + + private static final ResourceProfile DEFAULT_TESTING_PROFILE = + new ResourceProfile(DEFAULT_TESTING_CPU_CORES, DEFAULT_TESTING_MEMORY); + + private static final ResourceProfile DEFAULT_TESTING_BIG_PROFILE = + new ResourceProfile(DEFAULT_TESTING_CPU_CORES * 2, DEFAULT_TESTING_MEMORY * 2); + + private ResourceManagerGateway resourceManagerGateway; + + @Before + public void setUp() { + resourceManagerGateway = mock(ResourceManagerGateway.class); + } + + /** + * Tests that there are no free slots when we request, need to allocate from cluster manager master + */ + @Test + public void testRequestSlotWithoutFreeSlot() { + TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); + slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE)); + + assertEquals(0, slotManager.getAllocatedSlotCount()); + assertEquals(0, slotManager.getFreeSlotCount()); + assertEquals(1, slotManager.getPendingRequestCount()); + assertEquals(1, slotManager.getAllocatedContainers().size()); + assertEquals(DEFAULT_TESTING_PROFILE, slotManager.getAllocatedContainers().get(0)); + } + + /** + * Tests that there are some free slots when we request, and the request is fulfilled immediately + */ + @Test + public void testRequestSlotWithFreeSlot() { + TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); + + directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 1); + assertEquals(1, slotManager.getFreeSlotCount()); + + slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE)); + assertEquals(1, slotManager.getAllocatedSlotCount()); + assertEquals(0, slotManager.getFreeSlotCount()); + assertEquals(0, slotManager.getPendingRequestCount()); + assertEquals(0, slotManager.getAllocatedContainers().size()); + } + + /** + * Tests that there are some free slots when we request, but none of them are suitable + */ + @Test + public void testRequestSlotWithoutSuitableSlot() { + TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); + + directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 2); + assertEquals(2, slotManager.getFreeSlotCount()); + + slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_BIG_PROFILE)); + assertEquals(0, slotManager.getAllocatedSlotCount()); + assertEquals(2, slotManager.getFreeSlotCount()); + assertEquals(1, slotManager.getPendingRequestCount()); + assertEquals(1, slotManager.getAllocatedContainers().size()); + assertEquals(DEFAULT_TESTING_BIG_PROFILE, slotManager.getAllocatedContainers().get(0)); + } + + /** + * Tests that we send duplicated slot request + */ + @Test + public void testDuplicatedSlotRequest() { + TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); + directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 1); + + SlotRequest request1 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE); + SlotRequest request2 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_BIG_PROFILE); + + slotManager.requestSlot(request1); + slotManager.requestSlot(request2); + slotManager.requestSlot(request2); + slotManager.requestSlot(request1); + + assertEquals(1, slotManager.getAllocatedSlotCount()); + assertEquals(0, slotManager.getFreeSlotCount()); + assertEquals(1, slotManager.getPendingRequestCount()); + assertEquals(1, slotManager.getAllocatedContainers().size()); + assertEquals(DEFAULT_TESTING_BIG_PROFILE, slotManager.getAllocatedContainers().get(0)); + } + + /** + * Tests that we send multiple slot requests + */ + @Test + public void testRequestMultipleSlots() { + TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); + directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 5); + + // request 3 normal slots + for (int i = 0; i < 3; ++i) { + slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE)); + } + + // request 2 big slots + for (int i = 0; i < 2; ++i) { + slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_BIG_PROFILE)); + } + + // request 1 normal slot again + slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE)); + + assertEquals(4, slotManager.getAllocatedSlotCount()); + assertEquals(1, slotManager.getFreeSlotCount()); + assertEquals(2, slotManager.getPendingRequestCount()); + assertEquals(2, slotManager.getAllocatedContainers().size()); + assertEquals(DEFAULT_TESTING_BIG_PROFILE, slotManager.getAllocatedContainers().get(0)); + assertEquals(DEFAULT_TESTING_BIG_PROFILE, slotManager.getAllocatedContainers().get(1)); + } + + /** + * Tests that a new slot appeared in SlotReport, and we used it to fulfill a pending request + */ + @Test + public void testNewlyAppearedFreeSlotFulfillPendingRequest() { + TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); + slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE)); + assertEquals(1, slotManager.getPendingRequestCount()); + + SlotID slotId = SlotID.generate(); + SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE); + slotManager.updateSlotStatus(slotStatus); + + assertEquals(1, slotManager.getAllocatedSlotCount()); + assertEquals(0, slotManager.getFreeSlotCount()); + assertEquals(0, slotManager.getPendingRequestCount()); + assertTrue(slotManager.isAllocated(slotId)); + } + + /** + * Tests that a new slot appeared in SlotReport, but we have no pending request + */ + @Test + public void testNewlyAppearedFreeSlot() { + TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); + SlotID slotId = SlotID.generate(); + SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE); + slotManager.updateSlotStatus(slotStatus); + + assertEquals(0, slotManager.getAllocatedSlotCount()); + assertEquals(1, slotManager.getFreeSlotCount()); + } + + /** + * Tests that a new slot appeared in SlotReport, but it't not suitable for all the pending requests + */ + @Test + public void testNewlyAppearedFreeSlotNotMatchPendingRequests() { + TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); + slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_BIG_PROFILE)); + assertEquals(1, slotManager.getPendingRequestCount()); + + SlotID slotId = SlotID.generate(); + SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE); + slotManager.updateSlotStatus(slotStatus); + + assertEquals(0, slotManager.getAllocatedSlotCount()); + assertEquals(1, slotManager.getFreeSlotCount()); + assertEquals(1, slotManager.getPendingRequestCount()); + assertFalse(slotManager.isAllocated(slotId)); + } + + /** + * Tests that a new slot appeared in SlotReport, and it's been reported using by some job + */ + @Test + public void testNewlyAppearedInUseSlot() { + TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); + + SlotID slotId = SlotID.generate(); + SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE, new AllocationID(), new JobID()); + slotManager.updateSlotStatus(slotStatus); + + assertEquals(1, slotManager.getAllocatedSlotCount()); + assertEquals(0, slotManager.getFreeSlotCount()); + assertTrue(slotManager.isAllocated(slotId)); + } + + /** + * Tests that we had a slot in-use, and it's confirmed by SlotReport + */ + @Test + public void testExistingInUseSlotUpdateStatus() { + TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); + SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE); + slotManager.requestSlot(request); + + // make this slot in use + SlotID slotId = SlotID.generate(); + SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE); + slotManager.updateSlotStatus(slotStatus); + + assertEquals(1, slotManager.getAllocatedSlotCount()); + assertEquals(0, slotManager.getFreeSlotCount()); + assertTrue(slotManager.isAllocated(slotId)); + + // slot status is confirmed + SlotStatus slotStatus2 = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE, + request.getAllocationId(), request.getJobId()); + slotManager.updateSlotStatus(slotStatus2); + + assertEquals(1, slotManager.getAllocatedSlotCount()); + assertEquals(0, slotManager.getFreeSlotCount()); + assertTrue(slotManager.isAllocated(slotId)); + } + + /** + * Tests that we had a slot in-use, but it's empty according to the SlotReport + */ + @Test + public void testExistingInUseSlotAdjustedToEmpty() { + TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); + SlotRequest request1 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE); + slotManager.requestSlot(request1); + + // make this slot in use + SlotID slotId = SlotID.generate(); + SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE); + slotManager.updateSlotStatus(slotStatus); + + // another request pending + SlotRequest request2 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE); + slotManager.requestSlot(request2); + + assertEquals(1, slotManager.getAllocatedSlotCount()); + assertEquals(0, slotManager.getFreeSlotCount()); + assertEquals(1, slotManager.getPendingRequestCount()); + assertTrue(slotManager.isAllocated(slotId)); + assertTrue(slotManager.isAllocated(request1.getAllocationId())); + + + // but slot is reported empty again, request2 will be fulfilled, request1 will be missing + slotManager.updateSlotStatus(slotStatus); + + assertEquals(1, slotManager.getAllocatedSlotCount()); + assertEquals(0, slotManager.getFreeSlotCount()); + assertEquals(0, slotManager.getPendingRequestCount()); + assertTrue(slotManager.isAllocated(slotId)); + assertTrue(slotManager.isAllocated(request2.getAllocationId())); + } + + /** + * Tests that we had a slot in use, and it's also reported in use by TaskManager, but the allocation + * information didn't match. + */ + @Test + public void testExistingInUseSlotWithDifferentAllocationInfo() { + TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); + SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE); + slotManager.requestSlot(request); + + // make this slot in use + SlotID slotId = SlotID.generate(); + SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE); + slotManager.updateSlotStatus(slotStatus); + + assertEquals(1, slotManager.getAllocatedSlotCount()); + assertEquals(0, slotManager.getFreeSlotCount()); + assertEquals(0, slotManager.getPendingRequestCount()); + assertTrue(slotManager.isAllocated(slotId)); + assertTrue(slotManager.isAllocated(request.getAllocationId())); + + SlotStatus slotStatus2 = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE, new AllocationID(), new JobID()); + // update slot status with different allocation info + slotManager.updateSlotStatus(slotStatus2); + + // original request is missing and won't be allocated + assertEquals(1, slotManager.getAllocatedSlotCount()); + assertEquals(0, slotManager.getFreeSlotCount()); + assertEquals(0, slotManager.getPendingRequestCount()); + assertTrue(slotManager.isAllocated(slotId)); + assertFalse(slotManager.isAllocated(request.getAllocationId())); + assertTrue(slotManager.isAllocated(slotStatus2.getAllocationID())); + } + + /** + * Tests that we had a free slot, and it's confirmed by SlotReport + */ + @Test + public void testExistingEmptySlotUpdateStatus() { + TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); + ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE); + slotManager.addFreeSlot(slot); + + SlotStatus slotStatus = new SlotStatus(slot.getSlotId(), DEFAULT_TESTING_PROFILE); + slotManager.updateSlotStatus(slotStatus); + + assertEquals(0, slotManager.getAllocatedSlotCount()); + assertEquals(1, slotManager.getFreeSlotCount()); + assertEquals(0, slotManager.getPendingRequestCount()); + } + + /** + * Tests that we had a free slot, and it's reported in-use by TaskManager + */ + @Test + public void testExistingEmptySlotAdjustedToInUse() { + TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); + ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE); + slotManager.addFreeSlot(slot); + + SlotStatus slotStatus = new SlotStatus(slot.getSlotId(), DEFAULT_TESTING_PROFILE, + new AllocationID(), new JobID()); + slotManager.updateSlotStatus(slotStatus); + + assertEquals(1, slotManager.getAllocatedSlotCount()); + assertEquals(0, slotManager.getFreeSlotCount()); + assertEquals(0, slotManager.getPendingRequestCount()); + assertTrue(slotManager.isAllocated(slot.getSlotId())); + } + + /** + * Tests that we did some allocation but failed / rejected by TaskManager, request will retry + */ + @Test + public void testSlotAllocationFailedAtTaskManager() { + TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); + ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE); + slotManager.addFreeSlot(slot); + + SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE); + slotManager.requestSlot(request); + + assertEquals(1, slotManager.getAllocatedSlotCount()); + assertEquals(0, slotManager.getFreeSlotCount()); + assertEquals(0, slotManager.getPendingRequestCount()); + assertTrue(slotManager.isAllocated(slot.getSlotId())); + + slotManager.handleSlotRequestFailedAtTaskManager(request, slot.getSlotId()); + + assertEquals(1, slotManager.getAllocatedSlotCount()); + assertEquals(0, slotManager.getFreeSlotCount()); + assertEquals(0, slotManager.getPendingRequestCount()); + } + + + /** + * Tests that we did some allocation but failed / rejected by TaskManager, and slot is occupied by another request + */ + @Test + public void testSlotAllocationFailedAtTaskManagerOccupiedByOther() { + TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); + ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE); + slotManager.addFreeSlot(slot); + + SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE); + slotManager.requestSlot(request); + + // slot is set empty by heartbeat + SlotStatus slotStatus = new SlotStatus(slot.getSlotId(), slot.getResourceProfile()); + slotManager.updateSlotStatus(slotStatus); + + // another request took this slot + SlotRequest request2 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE); + slotManager.requestSlot(request2); + + assertEquals(1, slotManager.getAllocatedSlotCount()); + assertEquals(0, slotManager.getFreeSlotCount()); + assertEquals(0, slotManager.getPendingRequestCount()); + assertFalse(slotManager.isAllocated(request.getAllocationId())); + assertTrue(slotManager.isAllocated(request2.getAllocationId())); + + // original request should be pended + slotManager.handleSlotRequestFailedAtTaskManager(request, slot.getSlotId()); + + assertEquals(1, slotManager.getAllocatedSlotCount()); + assertEquals(0, slotManager.getFreeSlotCount()); + assertEquals(1, slotManager.getPendingRequestCount()); + assertFalse(slotManager.isAllocated(request.getAllocationId())); + assertTrue(slotManager.isAllocated(request2.getAllocationId())); + } + + @Test + public void testNotifyTaskManagerFailure() { + TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); + + ResourceID resource1 = ResourceID.generate(); + ResourceID resource2 = ResourceID.generate(); + + ResourceSlot slot11 = new ResourceSlot(new SlotID(resource1, 1), DEFAULT_TESTING_PROFILE); + ResourceSlot slot12 = new ResourceSlot(new SlotID(resource1, 2), DEFAULT_TESTING_PROFILE); + ResourceSlot slot21 = new ResourceSlot(new SlotID(resource2, 1), DEFAULT_TESTING_PROFILE); + ResourceSlot slot22 = new ResourceSlot(new SlotID(resource2, 2), DEFAULT_TESTING_PROFILE); + + slotManager.addFreeSlot(slot11); + slotManager.addFreeSlot(slot21); + + slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE)); + slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE)); + + assertEquals(2, slotManager.getAllocatedSlotCount()); + assertEquals(0, slotManager.getFreeSlotCount()); + assertEquals(0, slotManager.getPendingRequestCount()); + + slotManager.addFreeSlot(slot12); + slotManager.addFreeSlot(slot22); + + assertEquals(2, slotManager.getAllocatedSlotCount()); + assertEquals(2, slotManager.getFreeSlotCount()); + assertEquals(0, slotManager.getPendingRequestCount()); + + slotManager.notifyTaskManagerFailure(resource2); + + assertEquals(1, slotManager.getAllocatedSlotCount()); + assertEquals(1, slotManager.getFreeSlotCount()); + assertEquals(0, slotManager.getPendingRequestCount()); + + // notify an not exist resource failure + slotManager.notifyTaskManagerFailure(ResourceID.generate()); + + assertEquals(1, slotManager.getAllocatedSlotCount()); + assertEquals(1, slotManager.getFreeSlotCount()); + assertEquals(0, slotManager.getPendingRequestCount()); + } + + // ------------------------------------------------------------------------ + // testing utilities + // ------------------------------------------------------------------------ + + private void directlyProvideFreeSlots( + final SlotManager slotManager, + final ResourceProfile resourceProfile, + final int freeSlotNum) + { + for (int i = 0; i < freeSlotNum; ++i) { + slotManager.addFreeSlot(new ResourceSlot(SlotID.generate(), new ResourceProfile(resourceProfile))); + } + } + + // ------------------------------------------------------------------------ + // testing classes + // ------------------------------------------------------------------------ + + private static class TestingSlotManager extends SlotManager { + + private final List allocatedContainers; + + TestingSlotManager(ResourceManagerGateway resourceManagerGateway) { + super(resourceManagerGateway); + this.allocatedContainers = new LinkedList<>(); + } + + /** + * Choose slot randomly if it matches requirement + * + * @param request The slot request + * @param freeSlots All slots which can be used + * @return The chosen slot or null if cannot find a match + */ + @Override + protected ResourceSlot chooseSlotToUse(SlotRequest request, Map freeSlots) { + for (ResourceSlot slot : freeSlots.values()) { + if (slot.isMatchingRequirement(request.getResourceProfile())) { + return slot; + } + } + return null; + } + + /** + * Choose request randomly if offered slot can match its requirement + * + * @param offeredSlot The free slot + * @param pendingRequests All the pending slot requests + * @return The chosen request's AllocationID or null if cannot find a match + */ + @Override + protected SlotRequest chooseRequestToFulfill(ResourceSlot offeredSlot, + Map pendingRequests) + { + for (Map.Entry pendingRequest : pendingRequests.entrySet()) { + if (offeredSlot.isMatchingRequirement(pendingRequest.getValue().getResourceProfile())) { + return pendingRequest.getValue(); + } + } + return null; + } + + @Override + protected void allocateContainer(ResourceProfile resourceProfile) { + allocatedContainers.add(resourceProfile); + } + + List getAllocatedContainers() { + return allocatedContainers; + } + } +} From 700136c8c867562aa899fc95121e7c800b2881c2 Mon Sep 17 00:00:00 2001 From: beyond1920 Date: Sat, 27 Aug 2016 14:14:28 +0800 Subject: [PATCH 21/50] [FLINK-4516] leader election of resourcemanager - add serial rpc service - add a special rpcService implementation which directly executes the asynchronous calls serially one by one, it is just for testcase - Change ResourceManagerLeaderContender code and TestingSerialRpcService code - override shutdown logic to stop leadershipService - use a mocked RpcService rather than TestingSerialRpcService for resourceManager HA test This closes #2427 --- .../HighAvailabilityServices.java | 7 + .../highavailability/NonHaServices.java | 5 + .../rpc/resourcemanager/ResourceManager.java | 111 +++++- .../TestingHighAvailabilityServices.java | 19 +- .../runtime/rpc/TestingSerialRpcService.java | 369 ++++++++++++++++++ .../ResourceManagerHATest.java | 76 ++++ 6 files changed, 578 insertions(+), 9 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerHATest.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java index 73e4f1fe6d5c6..298147cf1dce1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java @@ -39,6 +39,13 @@ public interface HighAvailabilityServices { */ LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception; + /** + * Gets the leader election service for the cluster's resource manager. + * @return + * @throws Exception + */ + LeaderElectionService getResourceManagerLeaderElectionService() throws Exception; + /** * Gets the leader election service for the given job. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java index 3d2769bab1630..292a404118665 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java @@ -60,6 +60,11 @@ public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Excepti return new StandaloneLeaderRetrievalService(resourceManagerAddress, new UUID(0, 0)); } + @Override + public LeaderElectionService getResourceManagerLeaderElectionService() throws Exception { + return new StandaloneLeaderElectionService(); + } + @Override public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception { return new StandaloneLeaderElectionService(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java index 6f34465740760..f7147c977651b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java @@ -20,24 +20,26 @@ import akka.dispatch.Mapper; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.instance.InstanceID; +import org.apache.flink.runtime.leaderelection.LeaderContender; +import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.rpc.RpcMethod; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.jobmaster.JobMaster; import org.apache.flink.runtime.rpc.jobmaster.JobMasterGateway; import org.apache.flink.runtime.rpc.taskexecutor.TaskExecutorRegistrationSuccess; -import org.apache.flink.util.Preconditions; -import scala.concurrent.ExecutionContext; -import scala.concurrent.ExecutionContext$; import scala.concurrent.Future; import java.util.HashMap; import java.util.Map; import java.util.UUID; -import java.util.concurrent.ExecutorService; + +import static org.apache.flink.util.Preconditions.checkNotNull; /** * ResourceManager implementation. The resource manager is responsible for resource de-/allocation @@ -50,16 +52,51 @@ * */ public class ResourceManager extends RpcEndpoint { - private final ExecutionContext executionContext; private final Map jobMasterGateways; + private final HighAvailabilityServices highAvailabilityServices; + private LeaderElectionService leaderElectionService = null; + private UUID leaderSessionID = null; - public ResourceManager(RpcService rpcService, ExecutorService executorService) { + public ResourceManager(RpcService rpcService, HighAvailabilityServices highAvailabilityServices) { super(rpcService); - this.executionContext = ExecutionContext$.MODULE$.fromExecutor( - Preconditions.checkNotNull(executorService)); + this.highAvailabilityServices = checkNotNull(highAvailabilityServices); this.jobMasterGateways = new HashMap<>(); } + @Override + public void start() { + // start a leader + try { + super.start(); + leaderElectionService = highAvailabilityServices.getResourceManagerLeaderElectionService(); + leaderElectionService.start(new ResourceManagerLeaderContender()); + } catch (Throwable e) { + log.error("A fatal error happened when starting the ResourceManager", e); + throw new RuntimeException("A fatal error happened when starting the ResourceManager", e); + } + } + + @Override + public void shutDown() { + try { + leaderElectionService.stop(); + super.shutDown(); + } catch(Throwable e) { + log.error("A fatal error happened when shutdown the ResourceManager", e); + throw new RuntimeException("A fatal error happened when shutdown the ResourceManager", e); + } + } + + /** + * Gets the leader session id of current resourceManager. + * + * @return return the leaderSessionId of current resourceManager, this returns null until the current resourceManager is granted leadership. + */ + @VisibleForTesting + UUID getLeaderSessionID() { + return leaderSessionID; + } + /** * Register a {@link JobMaster} at the resource manager. * @@ -116,4 +153,62 @@ public org.apache.flink.runtime.rpc.registration.RegistrationResponse registerTa return new TaskExecutorRegistrationSuccess(new InstanceID(), 5000); } + + private class ResourceManagerLeaderContender implements LeaderContender { + + /** + * Callback method when current resourceManager is granted leadership + * + * @param leaderSessionID unique leadershipID + */ + @Override + public void grantLeadership(final UUID leaderSessionID) { + runAsync(new Runnable() { + @Override + public void run() { + log.info("ResourceManager {} was granted leadership with leader session ID {}", getAddress(), leaderSessionID); + ResourceManager.this.leaderSessionID = leaderSessionID; + // confirming the leader session ID might be blocking, + leaderElectionService.confirmLeaderSessionID(leaderSessionID); + } + }); + } + + /** + * Callback method when current resourceManager lose leadership. + */ + @Override + public void revokeLeadership() { + runAsync(new Runnable() { + @Override + public void run() { + log.info("ResourceManager {} was revoked leadership.", getAddress()); + jobMasterGateways.clear(); + leaderSessionID = null; + } + }); + } + + @Override + public String getAddress() { + return ResourceManager.this.getAddress(); + } + + /** + * Handles error occurring in the leader election service + * + * @param exception Exception being thrown in the leader election service + */ + @Override + public void handleError(final Exception exception) { + runAsync(new Runnable() { + @Override + public void run() { + log.error("ResourceManager received an error from the LeaderElectionService.", exception); + // terminate ResourceManager in case of an error + shutDown(); + } + }); + } + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java index 4d654a390d37f..3162f40391ea5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java @@ -32,6 +32,8 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices private volatile LeaderElectionService jobMasterLeaderElectionService; + private volatile LeaderElectionService resourceManagerLeaderElectionService; + // ------------------------------------------------------------------------ // Setters for mock / testing implementations @@ -44,7 +46,11 @@ public void setResourceManagerLeaderRetriever(LeaderRetrievalService resourceMan public void setJobMasterLeaderElectionService(LeaderElectionService leaderElectionService) { this.jobMasterLeaderElectionService = leaderElectionService; } - + + public void setResourceManagerLeaderElectionService(LeaderElectionService leaderElectionService) { + this.resourceManagerLeaderElectionService = leaderElectionService; + } + // ------------------------------------------------------------------------ // HA Services Methods // ------------------------------------------------------------------------ @@ -69,4 +75,15 @@ public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) thro throw new IllegalStateException("JobMasterLeaderElectionService has not been set"); } } + + @Override + public LeaderElectionService getResourceManagerLeaderElectionService() throws Exception { + LeaderElectionService service = resourceManagerLeaderElectionService; + + if (service != null) { + return service; + } else { + throw new IllegalStateException("ResourceManagerLeaderElectionService has not been set"); + } + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java new file mode 100644 index 0000000000000..7bdbb99da8aac --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java @@ -0,0 +1,369 @@ +/* + * 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.rpc; + +import akka.dispatch.ExecutionContexts; +import akka.dispatch.Futures; +import akka.util.Timeout; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.util.DirectExecutorService; +import org.apache.flink.util.Preconditions; +import scala.concurrent.Await; +import scala.concurrent.ExecutionContext; +import scala.concurrent.Future; +import scala.concurrent.duration.FiniteDuration; + +import java.lang.annotation.Annotation; +import java.lang.reflect.InvocationHandler; +import java.lang.reflect.Method; +import java.lang.reflect.Proxy; +import java.util.BitSet; +import java.util.UUID; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeUnit; + +import static org.apache.flink.util.Preconditions.checkNotNull; + + +/** + * An RPC Service implementation for testing. This RPC service directly executes all asynchronous calls one by one in the main thread. + */ +public class TestingSerialRpcService implements RpcService { + + private final DirectExecutorService executorService; + private final ConcurrentHashMap registeredConnections; + + public TestingSerialRpcService() { + executorService = new DirectExecutorService(); + this.registeredConnections = new ConcurrentHashMap<>(); + } + + @Override + public void scheduleRunnable(final Runnable runnable, final long delay, final TimeUnit unit) { + try { + unit.sleep(delay); + runnable.run(); + } catch (Throwable e) { + throw new RuntimeException(e); + } + } + + @Override + public ExecutionContext getExecutionContext() { + return ExecutionContexts.fromExecutorService(executorService); + } + + @Override + public void stopService() { + executorService.shutdown(); + registeredConnections.clear(); + } + + @Override + public void stopServer(RpcGateway selfGateway) { + + } + + @Override + public > C startServer(S rpcEndpoint) { + final String address = UUID.randomUUID().toString(); + + InvocationHandler akkaInvocationHandler = new TestingSerialInvocationHandler(address, rpcEndpoint); + ClassLoader classLoader = getClass().getClassLoader(); + + @SuppressWarnings("unchecked") + C self = (C) Proxy.newProxyInstance( + classLoader, + new Class[]{ + rpcEndpoint.getSelfGatewayType(), + MainThreadExecutor.class, + StartStoppable.class, + RpcGateway.class + }, + akkaInvocationHandler); + + return self; + } + + @Override + public Future connect(String address, Class clazz) { + RpcGateway gateway = registeredConnections.get(address); + + if (gateway != null) { + if (clazz.isAssignableFrom(gateway.getClass())) { + @SuppressWarnings("unchecked") + C typedGateway = (C) gateway; + return Futures.successful(typedGateway); + } else { + return Futures.failed( + new Exception("Gateway registered under " + address + " is not of type " + clazz)); + } + } else { + return Futures.failed(new Exception("No gateway registered under that name")); + } + } + + // ------------------------------------------------------------------------ + // connections + // ------------------------------------------------------------------------ + + public void registerGateway(String address, RpcGateway gateway) { + checkNotNull(address); + checkNotNull(gateway); + + if (registeredConnections.putIfAbsent(address, gateway) != null) { + throw new IllegalStateException("a gateway is already registered under " + address); + } + } + + private static class TestingSerialInvocationHandler> implements InvocationHandler, RpcGateway, MainThreadExecutor, StartStoppable { + + private final T rpcEndpoint; + + /** default timeout for asks */ + private final Timeout timeout; + + private final String address; + + private TestingSerialInvocationHandler(String address, T rpcEndpoint) { + this(address, rpcEndpoint, new Timeout(new FiniteDuration(10, TimeUnit.SECONDS))); + } + + private TestingSerialInvocationHandler(String address, T rpcEndpoint, Timeout timeout) { + this.rpcEndpoint = rpcEndpoint; + this.timeout = timeout; + this.address = address; + } + + @Override + public Object invoke(Object proxy, Method method, Object[] args) throws Throwable { + Class declaringClass = method.getDeclaringClass(); + if (declaringClass.equals(MainThreadExecutor.class) || + declaringClass.equals(Object.class) || declaringClass.equals(StartStoppable.class) || + declaringClass.equals(RpcGateway.class)) { + return method.invoke(this, args); + } else { + final String methodName = method.getName(); + Class[] parameterTypes = method.getParameterTypes(); + Annotation[][] parameterAnnotations = method.getParameterAnnotations(); + Timeout futureTimeout = extractRpcTimeout(parameterAnnotations, args, timeout); + + final Tuple2[], Object[]> filteredArguments = filterArguments( + parameterTypes, + parameterAnnotations, + args); + + Class returnType = method.getReturnType(); + + if (returnType.equals(Future.class)) { + try { + Object result = handleRpcInvocationSync(methodName, filteredArguments.f0, filteredArguments.f1, futureTimeout); + return Futures.successful(result); + } catch (Throwable e) { + return Futures.failed(e); + } + } else { + return handleRpcInvocationSync(methodName, filteredArguments.f0, filteredArguments.f1, futureTimeout); + } + } + } + + /** + * Handle rpc invocations by looking up the rpc method on the rpc endpoint and calling this + * method with the provided method arguments. If the method has a return value, it is returned + * to the sender of the call. + */ + private Object handleRpcInvocationSync(final String methodName, + final Class[] parameterTypes, + final Object[] args, + final Timeout futureTimeout) throws Exception { + final Method rpcMethod = lookupRpcMethod(methodName, parameterTypes); + Object result = rpcMethod.invoke(rpcEndpoint, args); + + if (result != null && result instanceof Future) { + Future future = (Future) result; + return Await.result(future, futureTimeout.duration()); + } else { + return result; + } + } + + @Override + public void runAsync(Runnable runnable) { + runnable.run(); + } + + @Override + public Future callAsync(Callable callable, Timeout callTimeout) { + try { + return Futures.successful(callable.call()); + } catch (Throwable e) { + return Futures.failed(e); + } + } + + @Override + public void scheduleRunAsync(final Runnable runnable, final long delay) { + try { + TimeUnit.MILLISECONDS.sleep(delay); + runnable.run(); + } catch (Throwable e) { + throw new RuntimeException(e); + } + } + + @Override + public String getAddress() { + return address; + } + + @Override + public void start() { + // do nothing + } + + @Override + public void stop() { + // do nothing + } + + /** + * Look up the rpc method on the given {@link RpcEndpoint} instance. + * + * @param methodName Name of the method + * @param parameterTypes Parameter types of the method + * @return Method of the rpc endpoint + * @throws NoSuchMethodException Thrown if the method with the given name and parameter types + * cannot be found at the rpc endpoint + */ + private Method lookupRpcMethod(final String methodName, + final Class[] parameterTypes) throws NoSuchMethodException { + return rpcEndpoint.getClass().getMethod(methodName, parameterTypes); + } + + // ------------------------------------------------------------------------ + // Helper methods + // ------------------------------------------------------------------------ + + /** + * Extracts the {@link RpcTimeout} annotated rpc timeout value from the list of given method + * arguments. If no {@link RpcTimeout} annotated parameter could be found, then the default + * timeout is returned. + * + * @param parameterAnnotations Parameter annotations + * @param args Array of arguments + * @param defaultTimeout Default timeout to return if no {@link RpcTimeout} annotated parameter + * has been found + * @return Timeout extracted from the array of arguments or the default timeout + */ + private static Timeout extractRpcTimeout(Annotation[][] parameterAnnotations, Object[] args, + Timeout defaultTimeout) { + if (args != null) { + Preconditions.checkArgument(parameterAnnotations.length == args.length); + + for (int i = 0; i < parameterAnnotations.length; i++) { + if (isRpcTimeout(parameterAnnotations[i])) { + if (args[i] instanceof FiniteDuration) { + return new Timeout((FiniteDuration) args[i]); + } else { + throw new RuntimeException("The rpc timeout parameter must be of type " + + FiniteDuration.class.getName() + ". The type " + args[i].getClass().getName() + + " is not supported."); + } + } + } + } + + return defaultTimeout; + } + + /** + * Removes all {@link RpcTimeout} annotated parameters from the parameter type and argument + * list. + * + * @param parameterTypes Array of parameter types + * @param parameterAnnotations Array of parameter annotations + * @param args Arary of arguments + * @return Tuple of filtered parameter types and arguments which no longer contain the + * {@link RpcTimeout} annotated parameter types and arguments + */ + private static Tuple2[], Object[]> filterArguments( + Class[] parameterTypes, + Annotation[][] parameterAnnotations, + Object[] args) { + + Class[] filteredParameterTypes; + Object[] filteredArgs; + + if (args == null) { + filteredParameterTypes = parameterTypes; + filteredArgs = null; + } else { + Preconditions.checkArgument(parameterTypes.length == parameterAnnotations.length); + Preconditions.checkArgument(parameterAnnotations.length == args.length); + + BitSet isRpcTimeoutParameter = new BitSet(parameterTypes.length); + int numberRpcParameters = parameterTypes.length; + + for (int i = 0; i < parameterTypes.length; i++) { + if (isRpcTimeout(parameterAnnotations[i])) { + isRpcTimeoutParameter.set(i); + numberRpcParameters--; + } + } + + if (numberRpcParameters == parameterTypes.length) { + filteredParameterTypes = parameterTypes; + filteredArgs = args; + } else { + filteredParameterTypes = new Class[numberRpcParameters]; + filteredArgs = new Object[numberRpcParameters]; + int counter = 0; + + for (int i = 0; i < parameterTypes.length; i++) { + if (!isRpcTimeoutParameter.get(i)) { + filteredParameterTypes[counter] = parameterTypes[i]; + filteredArgs[counter] = args[i]; + counter++; + } + } + } + } + return Tuple2.of(filteredParameterTypes, filteredArgs); + } + + /** + * Checks whether any of the annotations is of type {@link RpcTimeout} + * + * @param annotations Array of annotations + * @return True if {@link RpcTimeout} was found; otherwise false + */ + private static boolean isRpcTimeout(Annotation[] annotations) { + for (Annotation annotation : annotations) { + if (annotation.annotationType().equals(RpcTimeout.class)) { + return true; + } + } + + return false; + } + + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerHATest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerHATest.java new file mode 100644 index 0000000000000..dfffeda291760 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerHATest.java @@ -0,0 +1,76 @@ +/* + * 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.rpc.resourcemanager; + +import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; +import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; +import org.apache.flink.runtime.rpc.MainThreadExecutor; +import org.apache.flink.runtime.rpc.RpcEndpoint; +import org.apache.flink.runtime.rpc.RpcGateway; +import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.rpc.StartStoppable; +import org.junit.Assert; +import org.junit.Test; + +import java.util.UUID; + +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.doCallRealMethod; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +/** + * resourceManager HA test, including grant leadership and revoke leadership + */ +public class ResourceManagerHATest { + + @Test + public void testGrantAndRevokeLeadership() throws Exception { + // mock a RpcService which will return a special RpcGateway when call its startServer method, the returned RpcGateway directly execute runAsync call + TestingResourceManagerGatewayProxy gateway = mock(TestingResourceManagerGatewayProxy.class); + doCallRealMethod().when(gateway).runAsync(any(Runnable.class)); + + RpcService rpcService = mock(RpcService.class); + when(rpcService.startServer(any(RpcEndpoint.class))).thenReturn(gateway); + + TestingLeaderElectionService leaderElectionService = new TestingLeaderElectionService(); + TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices(); + highAvailabilityServices.setResourceManagerLeaderElectionService(leaderElectionService); + + final ResourceManager resourceManager = new ResourceManager(rpcService, highAvailabilityServices); + resourceManager.start(); + // before grant leadership, resourceManager's leaderId is null + Assert.assertNull(resourceManager.getLeaderSessionID()); + final UUID leaderId = UUID.randomUUID(); + leaderElectionService.isLeader(leaderId); + // after grant leadership, resourceManager's leaderId has value + Assert.assertEquals(leaderId, resourceManager.getLeaderSessionID()); + // then revoke leadership, resourceManager's leaderId is null again + leaderElectionService.notLeader(); + Assert.assertNull(resourceManager.getLeaderSessionID()); + } + + private static abstract class TestingResourceManagerGatewayProxy implements MainThreadExecutor, StartStoppable, RpcGateway { + @Override + public void runAsync(Runnable runnable) { + runnable.run(); + } + } + +} From 171cfd30f4cb1dfb90ba25358fbad836d3105839 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 29 Aug 2016 16:35:29 +0200 Subject: [PATCH 22/50] [FLINK-4529] [flip-6] Move TaskExecutor, JobMaster and ResourceManager out of the rpc package The TaskExecutor, the JobMaster and the ResourceManager were still contained in the rpc package. With this commit, they will be moved out of this package. Now they are contained in dedicated packages on the o.a.f.runtime level. This closes #2438. --- .../runtime/{rpc => }/jobmaster/JobMaster.java | 4 ++-- .../{rpc => }/jobmaster/JobMasterGateway.java | 2 +- .../registration/RegistrationResponse.java | 2 +- .../registration/RetryingRegistration.java | 2 +- .../resourcemanager/JobMasterRegistration.java | 2 +- .../resourcemanager/RegistrationResponse.java | 2 +- .../{rpc => }/resourcemanager/ResourceManager.java | 10 +++++----- .../resourcemanager/ResourceManagerGateway.java | 6 +++--- .../{rpc => }/resourcemanager/SlotAssignment.java | 2 +- .../SlotManager.java | 8 +++----- .../{rpc => }/resourcemanager/SlotRequest.java | 2 +- .../runtime/{rpc => }/taskexecutor/SlotReport.java | 2 +- .../runtime/{rpc => }/taskexecutor/SlotStatus.java | 2 +- .../{rpc => }/taskexecutor/TaskExecutor.java | 2 +- .../taskexecutor/TaskExecutorConfiguration.java | 2 +- .../taskexecutor/TaskExecutorGateway.java | 2 +- .../TaskExecutorRegistrationSuccess.java | 4 ++-- .../TaskExecutorToResourceManagerConnection.java | 8 ++++---- .../ClusterShutdownITCase.java | 2 +- .../ResourceManagerITCase.java | 2 +- .../ResourceManagerTest.java | 2 +- .../registration/RetryingRegistrationTest.java | 2 +- .../registration/TestRegistrationGateway.java | 2 +- .../resourcemanager/ResourceManagerHATest.java | 2 +- .../SlotManagerTest.java | 6 ++---- .../flink/runtime/rpc/akka/AkkaRpcServiceTest.java | 14 -------------- .../{rpc => }/taskexecutor/TaskExecutorTest.java | 4 ++-- 27 files changed, 41 insertions(+), 59 deletions(-) rename flink-runtime/src/main/java/org/apache/flink/runtime/{rpc => }/jobmaster/JobMaster.java (98%) rename flink-runtime/src/main/java/org/apache/flink/runtime/{rpc => }/jobmaster/JobMasterGateway.java (97%) rename flink-runtime/src/main/java/org/apache/flink/runtime/{rpc => }/registration/RegistrationResponse.java (97%) rename flink-runtime/src/main/java/org/apache/flink/runtime/{rpc => }/registration/RetryingRegistration.java (99%) rename flink-runtime/src/main/java/org/apache/flink/runtime/{rpc => }/resourcemanager/JobMasterRegistration.java (95%) rename flink-runtime/src/main/java/org/apache/flink/runtime/{rpc => }/resourcemanager/RegistrationResponse.java (96%) rename flink-runtime/src/main/java/org/apache/flink/runtime/{rpc => }/resourcemanager/ResourceManager.java (95%) rename flink-runtime/src/main/java/org/apache/flink/runtime/{rpc => }/resourcemanager/ResourceManagerGateway.java (92%) rename flink-runtime/src/main/java/org/apache/flink/runtime/{rpc => }/resourcemanager/SlotAssignment.java (94%) rename flink-runtime/src/main/java/org/apache/flink/runtime/{clusterframework => resourcemanager}/SlotManager.java (98%) rename flink-runtime/src/main/java/org/apache/flink/runtime/{rpc => }/resourcemanager/SlotRequest.java (97%) rename flink-runtime/src/main/java/org/apache/flink/runtime/{rpc => }/taskexecutor/SlotReport.java (97%) rename flink-runtime/src/main/java/org/apache/flink/runtime/{rpc => }/taskexecutor/SlotStatus.java (98%) rename flink-runtime/src/main/java/org/apache/flink/runtime/{rpc => }/taskexecutor/TaskExecutor.java (99%) rename flink-runtime/src/main/java/org/apache/flink/runtime/{rpc => }/taskexecutor/TaskExecutorConfiguration.java (98%) rename flink-runtime/src/main/java/org/apache/flink/runtime/{rpc => }/taskexecutor/TaskExecutorGateway.java (96%) rename flink-runtime/src/main/java/org/apache/flink/runtime/{rpc => }/taskexecutor/TaskExecutorRegistrationSuccess.java (94%) rename flink-runtime/src/main/java/org/apache/flink/runtime/{rpc => }/taskexecutor/TaskExecutorToResourceManagerConnection.java (95%) rename flink-runtime/src/test/java/org/apache/flink/runtime/{resourcemanager => clusterframework}/ClusterShutdownITCase.java (99%) rename flink-runtime/src/test/java/org/apache/flink/runtime/{resourcemanager => clusterframework}/ResourceManagerITCase.java (99%) rename flink-runtime/src/test/java/org/apache/flink/runtime/{resourcemanager => clusterframework}/ResourceManagerTest.java (99%) rename flink-runtime/src/test/java/org/apache/flink/runtime/{rpc => }/registration/RetryingRegistrationTest.java (99%) rename flink-runtime/src/test/java/org/apache/flink/runtime/{rpc => }/registration/TestRegistrationGateway.java (98%) rename flink-runtime/src/test/java/org/apache/flink/runtime/{rpc => }/resourcemanager/ResourceManagerHATest.java (98%) rename flink-runtime/src/test/java/org/apache/flink/runtime/{clusterframework => resourcemanager}/SlotManagerTest.java (98%) rename flink-runtime/src/test/java/org/apache/flink/runtime/{rpc => }/taskexecutor/TaskExecutorTest.java (97%) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java similarity index 98% rename from flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java index a046cb8948052..0a6a7ef0938ec 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.jobmaster; +package org.apache.flink.runtime.jobmaster; import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; @@ -26,7 +26,7 @@ import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.rpc.RpcMethod; -import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.taskmanager.TaskExecutionState; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMasterGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java similarity index 97% rename from flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMasterGateway.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java index 17a4c3ab75eda..a53e383942744 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMasterGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.jobmaster; +package org.apache.flink.runtime.jobmaster; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.rpc.RpcGateway; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RegistrationResponse.java b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RegistrationResponse.java similarity index 97% rename from flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RegistrationResponse.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/registration/RegistrationResponse.java index 2de560a25bce9..fefcc78db5f30 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RegistrationResponse.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RegistrationResponse.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.registration; +package org.apache.flink.runtime.registration; import java.io.Serializable; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RetryingRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java similarity index 99% rename from flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RetryingRegistration.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java index dcb5011fe0a41..88fe9b58d6ac2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RetryingRegistration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.registration; +package org.apache.flink.runtime.registration; import akka.dispatch.OnFailure; import akka.dispatch.OnSuccess; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/JobMasterRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java similarity index 95% rename from flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/JobMasterRegistration.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java index 7a2deae20b0b7..309dcc1762619 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/JobMasterRegistration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.resourcemanager; +package org.apache.flink.runtime.resourcemanager; import java.io.Serializable; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/RegistrationResponse.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/RegistrationResponse.java similarity index 96% rename from flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/RegistrationResponse.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/RegistrationResponse.java index 8ac9e49ecf1aa..fb6c401a1241e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/RegistrationResponse.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/RegistrationResponse.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.resourcemanager; +package org.apache.flink.runtime.resourcemanager; import org.apache.flink.runtime.instance.InstanceID; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java similarity index 95% rename from flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java index f7147c977651b..44c022b748c61 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.resourcemanager; +package org.apache.flink.runtime.resourcemanager; import akka.dispatch.Mapper; @@ -29,9 +29,9 @@ import org.apache.flink.runtime.rpc.RpcMethod; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcService; -import org.apache.flink.runtime.rpc.jobmaster.JobMaster; -import org.apache.flink.runtime.rpc.jobmaster.JobMasterGateway; -import org.apache.flink.runtime.rpc.taskexecutor.TaskExecutorRegistrationSuccess; +import org.apache.flink.runtime.jobmaster.JobMaster; +import org.apache.flink.runtime.jobmaster.JobMasterGateway; +import org.apache.flink.runtime.taskexecutor.TaskExecutorRegistrationSuccess; import scala.concurrent.Future; @@ -146,7 +146,7 @@ public SlotAssignment requestSlot(SlotRequest slotRequest) { * @return The response by the ResourceManager. */ @RpcMethod - public org.apache.flink.runtime.rpc.registration.RegistrationResponse registerTaskExecutor( + public org.apache.flink.runtime.registration.RegistrationResponse registerTaskExecutor( UUID resourceManagerLeaderId, String taskExecutorAddress, ResourceID resourceID) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java similarity index 92% rename from flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerGateway.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java index afddb01d6b128..b5782b06f1819 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java @@ -16,12 +16,12 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.resourcemanager; +package org.apache.flink.runtime.resourcemanager; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcTimeout; -import org.apache.flink.runtime.rpc.jobmaster.JobMaster; +import org.apache.flink.runtime.jobmaster.JobMaster; import scala.concurrent.Future; import scala.concurrent.duration.FiniteDuration; @@ -69,7 +69,7 @@ Future registerJobMaster( * * @return The future to the response by the ResourceManager. */ - Future registerTaskExecutor( + Future registerTaskExecutor( UUID resourceManagerLeaderId, String taskExecutorAddress, ResourceID resourceID, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotAssignment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotAssignment.java similarity index 94% rename from flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotAssignment.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotAssignment.java index 86cd8b76a8cc3..695204d751c6e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotAssignment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotAssignment.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.resourcemanager; +package org.apache.flink.runtime.resourcemanager; import java.io.Serializable; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/SlotManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotManager.java similarity index 98% rename from flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/SlotManager.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotManager.java index cc140a1abd287..5c066480aa4eb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/SlotManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotManager.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.clusterframework; +package org.apache.flink.runtime.resourcemanager; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.runtime.clusterframework.types.AllocationID; @@ -24,10 +24,8 @@ import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.clusterframework.types.ResourceSlot; import org.apache.flink.runtime.clusterframework.types.SlotID; -import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway; -import org.apache.flink.runtime.rpc.resourcemanager.SlotRequest; -import org.apache.flink.runtime.rpc.taskexecutor.SlotReport; -import org.apache.flink.runtime.rpc.taskexecutor.SlotStatus; +import org.apache.flink.runtime.taskexecutor.SlotReport; +import org.apache.flink.runtime.taskexecutor.SlotStatus; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotRequest.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequest.java similarity index 97% rename from flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotRequest.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequest.java index 74c7c3900784a..896421ba7f4ac 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotRequest.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.resourcemanager; +package org.apache.flink.runtime.resourcemanager; import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.clusterframework.types.AllocationID; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotReport.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotReport.java similarity index 97% rename from flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotReport.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotReport.java index c372ecbadcaa9..a5de2d55236fb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotReport.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotReport.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.taskexecutor; +package org.apache.flink.runtime.taskexecutor; import org.apache.flink.runtime.clusterframework.types.ResourceID; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotStatus.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotStatus.java similarity index 98% rename from flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotStatus.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotStatus.java index e8e208448c9c0..744b674a3af7b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotStatus.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotStatus.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.taskexecutor; +package org.apache.flink.runtime.taskexecutor; import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.clusterframework.types.AllocationID; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java similarity index 99% rename from flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java index 36d6310d9c320..4871b96810314 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.taskexecutor; +package org.apache.flink.runtime.taskexecutor; import akka.actor.ActorSystem; import akka.dispatch.ExecutionContexts$; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorConfiguration.java similarity index 98% rename from flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorConfiguration.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorConfiguration.java index 32484e114748d..3707a470c7b6b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorConfiguration.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.taskexecutor; +package org.apache.flink.runtime.taskexecutor; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.instance.InstanceConnectionInfo; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java similarity index 96% rename from flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorGateway.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java index b0b21b00eb2c3..6c99706c6f677 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.taskexecutor; +package org.apache.flink.runtime.taskexecutor; import org.apache.flink.runtime.rpc.RpcGateway; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorRegistrationSuccess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorRegistrationSuccess.java similarity index 94% rename from flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorRegistrationSuccess.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorRegistrationSuccess.java index 641102dda817a..b357f52a4b000 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorRegistrationSuccess.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorRegistrationSuccess.java @@ -16,10 +16,10 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.taskexecutor; +package org.apache.flink.runtime.taskexecutor; import org.apache.flink.runtime.instance.InstanceID; -import org.apache.flink.runtime.rpc.registration.RegistrationResponse; +import org.apache.flink.runtime.registration.RegistrationResponse; import java.io.Serializable; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorToResourceManagerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java similarity index 95% rename from flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorToResourceManagerConnection.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java index 7ccc8794761e8..25332a0108365 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorToResourceManagerConnection.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.taskexecutor; +package org.apache.flink.runtime.taskexecutor; import akka.dispatch.OnFailure; import akka.dispatch.OnSuccess; @@ -25,9 +25,9 @@ import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.rpc.RpcService; -import org.apache.flink.runtime.rpc.registration.RegistrationResponse; -import org.apache.flink.runtime.rpc.registration.RetryingRegistration; -import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.registration.RegistrationResponse; +import org.apache.flink.runtime.registration.RetryingRegistration; +import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.slf4j.Logger; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ClusterShutdownITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ClusterShutdownITCase.java similarity index 99% rename from flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ClusterShutdownITCase.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ClusterShutdownITCase.java index 32c6cac3a9a43..744308cbd2e5a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ClusterShutdownITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ClusterShutdownITCase.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.resourcemanager; +package org.apache.flink.runtime.clusterframework; import akka.actor.ActorSystem; import akka.testkit.JavaTestKit; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerITCase.java similarity index 99% rename from flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerITCase.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerITCase.java index 0c2ca1a0aeef8..1565dc3146a2b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerITCase.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.resourcemanager; +package org.apache.flink.runtime.clusterframework; import akka.actor.ActorSystem; import akka.testkit.JavaTestKit; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java similarity index 99% rename from flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java index 043c81c19b6c7..ca8a07ab1c0c3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.resourcemanager; +package org.apache.flink.runtime.clusterframework; import akka.actor.ActorSystem; import akka.testkit.JavaTestKit; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/registration/RetryingRegistrationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java similarity index 99% rename from flink-runtime/src/test/java/org/apache/flink/runtime/rpc/registration/RetryingRegistrationTest.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java index 9508825f5b7a4..80fa19cc67bc7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/registration/RetryingRegistrationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.registration; +package org.apache.flink.runtime.registration; import akka.dispatch.Futures; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/registration/TestRegistrationGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/TestRegistrationGateway.java similarity index 98% rename from flink-runtime/src/test/java/org/apache/flink/runtime/rpc/registration/TestRegistrationGateway.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/registration/TestRegistrationGateway.java index a049e48b5ae33..431fbe8284cb0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/registration/TestRegistrationGateway.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/TestRegistrationGateway.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.registration; +package org.apache.flink.runtime.registration; import akka.dispatch.Futures; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerHATest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java similarity index 98% rename from flink-runtime/src/test/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerHATest.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java index dfffeda291760..5799e62686101 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerHATest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.resourcemanager; +package org.apache.flink.runtime.resourcemanager; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/SlotManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/SlotManagerTest.java similarity index 98% rename from flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/SlotManagerTest.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/SlotManagerTest.java index 2ee280f9034ec..52d9d06400488 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/SlotManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/SlotManagerTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.clusterframework; +package org.apache.flink.runtime.resourcemanager; import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.clusterframework.types.AllocationID; @@ -24,9 +24,7 @@ import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.clusterframework.types.ResourceSlot; import org.apache.flink.runtime.clusterframework.types.SlotID; -import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway; -import org.apache.flink.runtime.rpc.resourcemanager.SlotRequest; -import org.apache.flink.runtime.rpc.taskexecutor.SlotStatus; +import org.apache.flink.runtime.taskexecutor.SlotStatus; import org.junit.Before; import org.junit.Test; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java index 2790cf88290b2..f55069e44e5b1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java @@ -21,28 +21,14 @@ import akka.actor.ActorSystem; import akka.util.Timeout; import org.apache.flink.core.testutils.OneShotLatch; -import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.akka.AkkaUtils; -import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.highavailability.NonHaServices; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.rpc.jobmaster.JobMaster; -import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway; -import org.apache.flink.runtime.rpc.resourcemanager.ResourceManager; import org.apache.flink.util.TestLogger; import org.junit.AfterClass; import org.junit.Test; -import org.mockito.Mockito; -import scala.concurrent.duration.Deadline; -import scala.concurrent.duration.FiniteDuration; - -import java.util.concurrent.ExecutorService; -import java.util.concurrent.ForkJoinPool; import java.util.concurrent.TimeUnit; -import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; public class AkkaRpcServiceTest extends TestLogger { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java similarity index 97% rename from flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java index 25a670c405a5d..a8d5bd76cf829 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.taskexecutor; +package org.apache.flink.runtime.taskexecutor; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.clusterframework.types.ResourceID; @@ -24,7 +24,7 @@ import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService; import org.apache.flink.runtime.rpc.TestingRpcService; -import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.util.TestLogger; import org.junit.Test; From 156f0ff5403b86be6dba63d38117952a1b08350f Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 29 Aug 2016 15:49:59 +0200 Subject: [PATCH 23/50] [FLINK-4528] [rpc] Marks main thread execution methods in RpcEndpoint as protected Give main thread execution context into the TaskExecutorToResourceManagerConnection --- .../apache/flink/runtime/rpc/RpcEndpoint.java | 8 +++--- .../runtime/taskexecutor/TaskExecutor.java | 7 ++++- ...skExecutorToResourceManagerConnection.java | 26 ++++++++++++------- .../rpc/{akka => }/AsyncCallsTest.java | 11 +++----- 4 files changed, 30 insertions(+), 22 deletions(-) rename flink-runtime/src/test/java/org/apache/flink/runtime/rpc/{akka => }/AsyncCallsTest.java (94%) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java index 7b3f8a189d5d8..e9e2b2c40d3de 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java @@ -161,7 +161,7 @@ public String getAddress() { * * @return Main thread execution context */ - public ExecutionContext getMainThreadExecutionContext() { + protected ExecutionContext getMainThreadExecutionContext() { return mainThreadExecutionContext; } @@ -184,7 +184,7 @@ public RpcService getRpcService() { * * @param runnable Runnable to be executed in the main thread of the underlying RPC endpoint */ - public void runAsync(Runnable runnable) { + protected void runAsync(Runnable runnable) { ((MainThreadExecutor) self).runAsync(runnable); } @@ -195,7 +195,7 @@ public void runAsync(Runnable runnable) { * @param runnable Runnable to be executed * @param delay The delay after which the runnable will be executed */ - public void scheduleRunAsync(Runnable runnable, long delay, TimeUnit unit) { + protected void scheduleRunAsync(Runnable runnable, long delay, TimeUnit unit) { ((MainThreadExecutor) self).scheduleRunAsync(runnable, unit.toMillis(delay)); } @@ -209,7 +209,7 @@ public void scheduleRunAsync(Runnable runnable, long delay, TimeUnit unit) { * @param Return type of the callable * @return Future for the result of the callable. */ - public Future callAsync(Callable callable, Timeout timeout) { + protected Future callAsync(Callable callable, Timeout timeout) { return ((MainThreadExecutor) self).callAsync(callable, timeout); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java index 4871b96810314..735730b3562ba 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java @@ -176,7 +176,12 @@ public void notifyOfNewResourceManagerLeader(String newLeaderAddress, UUID newLe if (newLeaderAddress != null) { log.info("Attempting to register at ResourceManager {}", newLeaderAddress); resourceManagerConnection = - new TaskExecutorToResourceManagerConnection(log, this, newLeaderAddress, newLeaderId); + new TaskExecutorToResourceManagerConnection( + log, + this, + newLeaderAddress, + newLeaderId, + getMainThreadExecutionContext()); resourceManagerConnection.start(); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java index 25332a0108365..28062b6e2496a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java @@ -31,6 +31,7 @@ import org.slf4j.Logger; +import scala.concurrent.ExecutionContext; import scala.concurrent.Future; import scala.concurrent.duration.FiniteDuration; @@ -55,9 +56,12 @@ public class TaskExecutorToResourceManagerConnection { private final String resourceManagerAddress; + /** Execution context to be used to execute the on complete action of the ResourceManagerRegistration */ + private final ExecutionContext executionContext; + private TaskExecutorToResourceManagerConnection.ResourceManagerRegistration pendingRegistration; - private ResourceManagerGateway registeredResourceManager; + private volatile ResourceManagerGateway registeredResourceManager; private InstanceID registrationId; @@ -66,15 +70,17 @@ public class TaskExecutorToResourceManagerConnection { public TaskExecutorToResourceManagerConnection( - Logger log, - TaskExecutor taskExecutor, - String resourceManagerAddress, - UUID resourceManagerLeaderId) { + Logger log, + TaskExecutor taskExecutor, + String resourceManagerAddress, + UUID resourceManagerLeaderId, + ExecutionContext executionContext) { this.log = checkNotNull(log); this.taskExecutor = checkNotNull(taskExecutor); this.resourceManagerAddress = checkNotNull(resourceManagerAddress); this.resourceManagerLeaderId = checkNotNull(resourceManagerLeaderId); + this.executionContext = checkNotNull(executionContext); } // ------------------------------------------------------------------------ @@ -93,22 +99,22 @@ public void start() { pendingRegistration.startRegistration(); Future> future = pendingRegistration.getFuture(); - + future.onSuccess(new OnSuccess>() { @Override public void onSuccess(Tuple2 result) { - registeredResourceManager = result.f0; registrationId = result.f1.getRegistrationId(); + registeredResourceManager = result.f0; } - }, taskExecutor.getMainThreadExecutionContext()); + }, executionContext); // this future should only ever fail if there is a bug, not if the registration is declined future.onFailure(new OnFailure() { @Override public void onFailure(Throwable failure) { - taskExecutor.onFatalError(failure); + taskExecutor.onFatalErrorAsync(failure); } - }, taskExecutor.getMainThreadExecutionContext()); + }, executionContext); } public void close() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AsyncCallsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java similarity index 94% rename from flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AsyncCallsTest.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java index d33987ccf6d69..1791056424e55 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AsyncCallsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java @@ -16,18 +16,15 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka; +package org.apache.flink.runtime.rpc; import akka.actor.ActorSystem; import akka.util.Timeout; import org.apache.flink.core.testutils.OneShotLatch; import org.apache.flink.runtime.akka.AkkaUtils; -import org.apache.flink.runtime.rpc.RpcEndpoint; -import org.apache.flink.runtime.rpc.RpcGateway; -import org.apache.flink.runtime.rpc.RpcMethod; -import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.rpc.akka.AkkaRpcService; import org.apache.flink.util.TestLogger; import org.junit.AfterClass; import org.junit.Test; @@ -51,7 +48,7 @@ public class AsyncCallsTest extends TestLogger { private static ActorSystem actorSystem = AkkaUtils.createDefaultActorSystem(); - private static AkkaRpcService akkaRpcService = + private static AkkaRpcService akkaRpcService = new AkkaRpcService(actorSystem, new Timeout(10000, TimeUnit.MILLISECONDS)); @AfterClass @@ -173,7 +170,7 @@ public void run() { // test RPC endpoint // ------------------------------------------------------------------------ - interface TestGateway extends RpcGateway { + public interface TestGateway extends RpcGateway { void someCall(); From ab566ebfa0ffb30dbd20d15ab1cbc59dae4d8f06 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 2 Sep 2016 14:51:16 +0200 Subject: [PATCH 24/50] [hotfix] Add self rpc gateway registration to TestingSerialRpcService --- .../runtime/rpc/TestingSerialRpcService.java | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java index 7bdbb99da8aac..955edcc8660f3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java @@ -43,7 +43,8 @@ /** - * An RPC Service implementation for testing. This RPC service directly executes all asynchronous calls one by one in the main thread. + * An RPC Service implementation for testing. This RPC service directly executes all asynchronous + * calls one by one in the calling thread. */ public class TestingSerialRpcService implements RpcService { @@ -52,7 +53,7 @@ public class TestingSerialRpcService implements RpcService { public TestingSerialRpcService() { executorService = new DirectExecutorService(); - this.registeredConnections = new ConcurrentHashMap<>(); + this.registeredConnections = new ConcurrentHashMap<>(16); } @Override @@ -78,14 +79,14 @@ public void stopService() { @Override public void stopServer(RpcGateway selfGateway) { - + registeredConnections.remove(selfGateway.getAddress()); } @Override public > C startServer(S rpcEndpoint) { final String address = UUID.randomUUID().toString(); - InvocationHandler akkaInvocationHandler = new TestingSerialInvocationHandler(address, rpcEndpoint); + InvocationHandler akkaInvocationHandler = new TestingSerialRpcService.TestingSerialInvocationHandler<>(address, rpcEndpoint); ClassLoader classLoader = getClass().getClassLoader(); @SuppressWarnings("unchecked") @@ -99,6 +100,9 @@ public > C startServer(S rpcEndpo }, akkaInvocationHandler); + // register self + registeredConnections.putIfAbsent(self.getAddress(), self); + return self; } @@ -133,7 +137,7 @@ public void registerGateway(String address, RpcGateway gateway) { } } - private static class TestingSerialInvocationHandler> implements InvocationHandler, RpcGateway, MainThreadExecutor, StartStoppable { + private static final class TestingSerialInvocationHandler> implements InvocationHandler, RpcGateway, MainThreadExecutor, StartStoppable { private final T rpcEndpoint; @@ -197,7 +201,7 @@ private Object handleRpcInvocationSync(final String methodName, final Method rpcMethod = lookupRpcMethod(methodName, parameterTypes); Object result = rpcMethod.invoke(rpcEndpoint, args); - if (result != null && result instanceof Future) { + if (result instanceof Future) { Future future = (Future) result; return Await.result(future, futureTimeout.duration()); } else { From 004b8ef9edd686d3566093a478a1bbb094299564 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 23 Aug 2016 17:59:54 +0200 Subject: [PATCH 25/50] [FLINK-4451] [rpc] Throw RpcConnectionException when rpc endpoint is not reachable This PR introduces a RpcConnectionException which is thrown if the rpc endpoint is not reachable when calling RpcService.connect. This closes #2405. --- .../registration/RetryingRegistration.java | 2 +- .../apache/flink/runtime/rpc/RpcService.java | 7 +++- .../runtime/rpc/akka/AkkaRpcService.java | 38 ++++++++++------- .../exceptions/RpcConnectionException.java | 41 +++++++++++++++++++ .../runtime/rpc/akka/AkkaRpcActorTest.java | 18 ++++++++ 5 files changed, 88 insertions(+), 18 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/RpcConnectionException.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java index 88fe9b58d6ac2..ea49e42d87826 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java @@ -197,7 +197,7 @@ public void onSuccess(Gateway result) { @Override public void onFailure(Throwable failure) { if (!isCanceled()) { - log.warn("Could not resolve {} address {}, retrying...", targetName, targetAddress); + log.warn("Could not resolve {} address {}, retrying...", targetName, targetAddress, failure); startRegistration(); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java index bc0f5cb191fee..78c1cec33bc9a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.rpc; +import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException; import scala.concurrent.ExecutionContext; import scala.concurrent.Future; @@ -32,12 +33,14 @@ public interface RpcService { /** * Connect to a remote rpc server under the provided address. Returns a rpc gateway which can - * be used to communicate with the rpc server. + * be used to communicate with the rpc server. If the connection failed, then the returned + * future is failed with a {@link RpcConnectionException}. * * @param address Address of the remote rpc server * @param clazz Class of the rpc gateway to return * @param Type of the rpc gateway to return - * @return Future containing the rpc gateway + * @return Future containing the rpc gateway or an {@link RpcConnectionException} if the + * connection attempt failed */ Future connect(String address, Class clazz); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java index 00a693253bf4f..060a1ef499cb0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java @@ -35,6 +35,7 @@ import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.StartStoppable; +import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -99,25 +100,32 @@ public Future connect(final String address, final Clas final Future identify = asker.ask(new Identify(42), timeout); return identify.map(new Mapper(){ @Override - public C apply(Object obj) { - ActorRef actorRef = ((ActorIdentity) obj).getRef(); + public C checkedApply(Object obj) throws Exception { - final String address = AkkaUtils.getAkkaURL(actorSystem, actorRef); + ActorIdentity actorIdentity = (ActorIdentity) obj; - InvocationHandler akkaInvocationHandler = new AkkaInvocationHandler(address, actorRef, timeout, maximumFramesize); + if (actorIdentity.getRef() == null) { + throw new RpcConnectionException("Could not connect to rpc endpoint under address " + address + '.'); + } else { + ActorRef actorRef = actorIdentity.getRef(); + + final String address = AkkaUtils.getAkkaURL(actorSystem, actorRef); + + InvocationHandler akkaInvocationHandler = new AkkaInvocationHandler(address, actorRef, timeout, maximumFramesize); - // Rather than using the System ClassLoader directly, we derive the ClassLoader - // from this class . That works better in cases where Flink runs embedded and all Flink - // code is loaded dynamically (for example from an OSGI bundle) through a custom ClassLoader - ClassLoader classLoader = AkkaRpcService.this.getClass().getClassLoader(); - - @SuppressWarnings("unchecked") - C proxy = (C) Proxy.newProxyInstance( - classLoader, - new Class[] {clazz}, - akkaInvocationHandler); + // Rather than using the System ClassLoader directly, we derive the ClassLoader + // from this class . That works better in cases where Flink runs embedded and all Flink + // code is loaded dynamically (for example from an OSGI bundle) through a custom ClassLoader + ClassLoader classLoader = AkkaRpcService.this.getClass().getClassLoader(); - return proxy; + @SuppressWarnings("unchecked") + C proxy = (C) Proxy.newProxyInstance( + classLoader, + new Class[]{clazz}, + akkaInvocationHandler); + + return proxy; + } } }, actorSystem.dispatcher()); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/RpcConnectionException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/RpcConnectionException.java new file mode 100644 index 0000000000000..a22ebe74b7a06 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/RpcConnectionException.java @@ -0,0 +1,41 @@ +/* + * 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.rpc.exceptions; + +import java.util.concurrent.ExecutionException; + +/** + * Exception class which is thrown if a rpc connection failed. Usually this happens if the remote + * host cannot be reached. + */ +public class RpcConnectionException extends ExecutionException { + private static final long serialVersionUID = -5500560405481142472L; + + public RpcConnectionException(String message) { + super(message); + } + + public RpcConnectionException(String message, Throwable cause) { + super(message, cause); + } + + public RpcConnectionException(Throwable cause) { + super(cause); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java index 82d13f065c294..a6ceb9104a1ef 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java @@ -25,6 +25,7 @@ import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcMethod; import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException; import org.apache.flink.util.TestLogger; import org.hamcrest.core.Is; import org.junit.AfterClass; @@ -36,6 +37,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; public class AkkaRpcActorTest extends TestLogger { @@ -72,6 +74,22 @@ public void testAddressResolution() throws Exception { assertEquals(rpcEndpoint.getAddress(), rpcGateway.getAddress()); } + /** + * Tests that a {@link RpcConnectionException} is thrown if the rpc endpoint cannot be connected to. + */ + @Test + public void testFailingAddressResolution() throws Exception { + Future futureRpcGateway = akkaRpcService.connect("foobar", DummyRpcGateway.class); + + try { + DummyRpcGateway gateway = Await.result(futureRpcGateway, timeout.duration()); + + fail("The rpc connection resolution should have failed."); + } catch (RpcConnectionException exception) { + // we're expecting a RpcConnectionException + } + } + /** * Tests that the {@link AkkaRpcActor} stashes messages until the corresponding * {@link RpcEndpoint} has been started. From f9e2dc05c4369a8c7c08ef620bc60324b08346bb Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 8 Sep 2016 18:43:15 +0200 Subject: [PATCH 26/50] [hotfix] [taskmanager] Fixes TaskManager component creation at startup --- .../runtime/taskexecutor/TaskExecutor.java | 189 +++++++++++++++--- .../TaskExecutorConfiguration.java | 9 - 2 files changed, 159 insertions(+), 39 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java index 735730b3562ba..a455fe2dec72d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java @@ -19,9 +19,19 @@ package org.apache.flink.runtime.taskexecutor; import akka.actor.ActorSystem; -import akka.dispatch.ExecutionContexts$; import akka.util.Timeout; import com.typesafe.config.Config; +import org.apache.flink.runtime.io.network.ConnectionManager; +import org.apache.flink.runtime.io.network.LocalConnectionManager; +import org.apache.flink.runtime.io.network.TaskEventDispatcher; +import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; +import org.apache.flink.runtime.io.network.netty.NettyConnectionManager; +import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; +import org.apache.flink.runtime.query.KvStateRegistry; +import org.apache.flink.runtime.query.netty.DisabledKvStateRequestStats; +import org.apache.flink.runtime.query.netty.KvStateServer; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; +import org.apache.flink.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -37,7 +47,6 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; @@ -61,7 +70,6 @@ import scala.Tuple2; import scala.Option; import scala.Some; -import scala.concurrent.ExecutionContext; import scala.concurrent.duration.Duration; import scala.concurrent.duration.FiniteDuration; @@ -70,9 +78,9 @@ import java.net.InetAddress; import java.net.InetSocketAddress; import java.util.UUID; -import java.util.concurrent.ForkJoinPool; import java.util.concurrent.TimeUnit; +import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; /** @@ -86,6 +94,8 @@ public class TaskExecutor extends RpcEndpoint { /** The unique resource ID of this TaskExecutor */ private final ResourceID resourceID; + private final TaskManagerLocation taskManagerLocation; + /** The access to the leader election and metadata storage services */ private final HighAvailabilityServices haServices; @@ -113,22 +123,26 @@ public class TaskExecutor extends RpcEndpoint { public TaskExecutor( TaskExecutorConfiguration taskExecutorConfig, ResourceID resourceID, + TaskManagerLocation taskManagerLocation, MemoryManager memoryManager, IOManager ioManager, NetworkEnvironment networkEnvironment, - int numberOfSlots, RpcService rpcService, HighAvailabilityServices haServices) { super(rpcService); + checkArgument(taskExecutorConfig.getNumberOfSlots() > 0, "The number of slots has to be larger than 0."); + this.taskExecutorConfig = checkNotNull(taskExecutorConfig); this.resourceID = checkNotNull(resourceID); + this.taskManagerLocation = checkNotNull(taskManagerLocation); this.memoryManager = checkNotNull(memoryManager); this.ioManager = checkNotNull(ioManager); this.networkEnvironment = checkNotNull(networkEnvironment); - this.numberOfSlots = checkNotNull(numberOfSlots); this.haServices = checkNotNull(haServices); + + this.numberOfSlots = taskExecutorConfig.getNumberOfSlots(); } // ------------------------------------------------------------------------ @@ -360,10 +374,10 @@ public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) thro * then a HighAvailabilityServices is constructed from the configuration. * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack. * @return An ActorRef to the TaskManager actor. - * @throws org.apache.flink.configuration.IllegalConfigurationException Thrown, if the given config contains illegal values. - * @throws java.io.IOException Thrown, if any of the I/O components (such as buffer pools, + * @throws IllegalConfigurationException Thrown, if the given config contains illegal values. + * @throws IOException Thrown, if any of the I/O components (such as buffer pools, * I/O manager, ...) cannot be properly started. - * @throws java.lang.Exception Thrown is some other error occurs while parsing the configuration + * @throws Exception Thrown is some other error occurs while parsing the configuration * or starting the TaskManager components. */ public static TaskExecutor startTaskManagerComponentsAndActor( @@ -377,19 +391,105 @@ public static TaskExecutor startTaskManagerComponentsAndActor( final TaskExecutorConfiguration taskExecutorConfig = parseTaskManagerConfiguration( configuration, taskManagerHostname, localTaskManagerCommunication); + TaskManagerComponents taskManagerComponents = createTaskManagerComponents( + resourceID, + InetAddress.getByName(taskManagerHostname), + taskExecutorConfig, + configuration); + + final TaskExecutor taskExecutor = new TaskExecutor( + taskExecutorConfig, + resourceID, + taskManagerComponents.getTaskManagerLocation(), + taskManagerComponents.getMemoryManager(), + taskManagerComponents.getIOManager(), + taskManagerComponents.getNetworkEnvironment(), + rpcService, + haServices); + + return taskExecutor; + } + + /** + * Creates and returns the task manager components. + * + * @param resourceID resource ID of the task manager + * @param taskManagerAddress address of the task manager + * @param taskExecutorConfig task manager configuration + * @param configuration of Flink + * @return task manager components + * @throws Exception + */ + private static TaskExecutor.TaskManagerComponents createTaskManagerComponents( + ResourceID resourceID, + InetAddress taskManagerAddress, + TaskExecutorConfiguration taskExecutorConfig, + Configuration configuration) throws Exception { MemoryType memType = taskExecutorConfig.getNetworkConfig().memoryType(); // pre-start checks checkTempDirs(taskExecutorConfig.getTmpDirPaths()); - ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool()); + NetworkEnvironmentConfiguration networkEnvironmentConfiguration = taskExecutorConfig.getNetworkConfig(); + + NetworkBufferPool networkBufferPool = new NetworkBufferPool( + networkEnvironmentConfiguration.numNetworkBuffers(), + networkEnvironmentConfiguration.networkBufferSize(), + networkEnvironmentConfiguration.memoryType()); + + ConnectionManager connectionManager; + + if (networkEnvironmentConfiguration.nettyConfig().isDefined()) { + connectionManager = new NettyConnectionManager(networkEnvironmentConfiguration.nettyConfig().get()); + } else { + connectionManager = new LocalConnectionManager(); + } + + ResultPartitionManager resultPartitionManager = new ResultPartitionManager(); + TaskEventDispatcher taskEventDispatcher = new TaskEventDispatcher(); + + KvStateRegistry kvStateRegistry = new KvStateRegistry(); + + KvStateServer kvStateServer; + + if (networkEnvironmentConfiguration.nettyConfig().isDefined()) { + NettyConfig nettyConfig = networkEnvironmentConfiguration.nettyConfig().get(); + + int numNetworkThreads = networkEnvironmentConfiguration.queryServerNetworkThreads() == 0 ? + nettyConfig.getNumberOfSlots() : networkEnvironmentConfiguration.queryServerNetworkThreads(); + + int numQueryThreads = networkEnvironmentConfiguration.queryServerQueryThreads() == 0 ? + nettyConfig.getNumberOfSlots() : networkEnvironmentConfiguration.queryServerQueryThreads(); + + kvStateServer = new KvStateServer( + taskManagerAddress, + networkEnvironmentConfiguration.queryServerPort(), + numNetworkThreads, + numQueryThreads, + kvStateRegistry, + new DisabledKvStateRequestStats()); + } else { + kvStateServer = null; + } // we start the network first, to make sure it can allocate its buffers first final NetworkEnvironment network = new NetworkEnvironment( - executionContext, - taskExecutorConfig.getTimeout(), - taskExecutorConfig.getNetworkConfig(), - taskExecutorConfig.getConnectionInfo()); + networkBufferPool, + connectionManager, + resultPartitionManager, + taskEventDispatcher, + kvStateRegistry, + kvStateServer, + networkEnvironmentConfiguration.ioMode(), + networkEnvironmentConfiguration.partitionRequestInitialBackoff(), + networkEnvironmentConfiguration.partitinRequestMaxBackoff()); + + network.start(); + + TaskManagerLocation taskManagerLocation = new TaskManagerLocation( + resourceID, + taskManagerAddress, + network.getConnectionManager().getDataPort()); // computing the amount of memory to use depends on how much memory is available // it strictly needs to happen AFTER the network stack has been initialized @@ -473,17 +573,7 @@ public static TaskExecutor startTaskManagerComponentsAndActor( // start the I/O manager, it will create some temp directories. final IOManager ioManager = new IOManagerAsync(taskExecutorConfig.getTmpDirPaths()); - final TaskExecutor taskExecutor = new TaskExecutor( - taskExecutorConfig, - resourceID, - memoryManager, - ioManager, - network, - taskExecutorConfig.getNumberOfSlots(), - rpcService, - haServices); - - return taskExecutor; + return new TaskExecutor.TaskManagerComponents(taskManagerLocation, memoryManager, ioManager, network); } // -------------------------------------------------------------------------- @@ -519,7 +609,7 @@ private static TaskExecutorConfiguration parseTaskManagerConfiguration( "Leave config parameter empty or use 0 to let the system choose a port automatically."); InetAddress taskManagerAddress = InetAddress.getByName(taskManagerHostname); - final InstanceConnectionInfo connectionInfo = new InstanceConnectionInfo(taskManagerAddress, dataport); + final InetSocketAddress taskManagerInetSocketAddress = new InetSocketAddress(taskManagerAddress, dataport); // ----> memory / network stack (shuffles/broadcasts), task slots, temp directories @@ -576,7 +666,12 @@ private static TaskExecutorConfiguration parseTaskManagerConfiguration( final NettyConfig nettyConfig; if (!localTaskManagerCommunication) { - nettyConfig = new NettyConfig(connectionInfo.address(), connectionInfo.dataPort(), pageSize, slots, configuration); + nettyConfig = new NettyConfig( + taskManagerInetSocketAddress.getAddress(), + taskManagerInetSocketAddress.getPort(), + pageSize, + slots, + configuration); } else { nettyConfig = null; } @@ -613,8 +708,9 @@ private static TaskExecutorConfiguration parseTaskManagerConfiguration( queryServerPort, queryServerNetworkThreads, queryServerQueryThreads, - localTaskManagerCommunication ? Option.empty() : new Some<>(nettyConfig), - new Tuple2<>(500, 3000)); + Option.apply(nettyConfig), + 500, + 30000); // ----> timeouts, library caching, profiling @@ -695,7 +791,6 @@ private static TaskExecutorConfiguration parseTaskManagerConfiguration( return new TaskExecutorConfiguration( tmpDirs, cleanupInterval, - connectionInfo, networkConfig, timeout, finiteRegistrationDuration, @@ -829,4 +924,38 @@ public void handleError(Exception exception) { onFatalErrorAsync(exception); } } + + private static class TaskManagerComponents { + private final TaskManagerLocation taskManagerLocation; + private final MemoryManager memoryManager; + private final IOManager ioManager; + private final NetworkEnvironment networkEnvironment; + + private TaskManagerComponents( + TaskManagerLocation taskManagerLocation, + MemoryManager memoryManager, + IOManager ioManager, + NetworkEnvironment networkEnvironment) { + this.taskManagerLocation = Preconditions.checkNotNull(taskManagerLocation); + this.memoryManager = Preconditions.checkNotNull(memoryManager); + this.ioManager = Preconditions.checkNotNull(ioManager); + this.networkEnvironment = Preconditions.checkNotNull(networkEnvironment); + } + + public MemoryManager getMemoryManager() { + return memoryManager; + } + + public IOManager getIOManager() { + return ioManager; + } + + public NetworkEnvironment getNetworkEnvironment() { + return networkEnvironment; + } + + public TaskManagerLocation getTaskManagerLocation() { + return taskManagerLocation; + } + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorConfiguration.java index 3707a470c7b6b..c97c893530a45 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorConfiguration.java @@ -19,7 +19,6 @@ package org.apache.flink.runtime.taskexecutor; import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration; import scala.concurrent.duration.FiniteDuration; @@ -52,12 +51,9 @@ public class TaskExecutorConfiguration implements Serializable { private final NetworkEnvironmentConfiguration networkConfig; - private final InstanceConnectionInfo connectionInfo; - public TaskExecutorConfiguration( String[] tmpDirPaths, long cleanupInterval, - InstanceConnectionInfo connectionInfo, NetworkEnvironmentConfiguration networkConfig, FiniteDuration timeout, FiniteDuration maxRegistrationDuration, @@ -66,7 +62,6 @@ public TaskExecutorConfiguration( this (tmpDirPaths, cleanupInterval, - connectionInfo, networkConfig, timeout, maxRegistrationDuration, @@ -80,7 +75,6 @@ public TaskExecutorConfiguration( public TaskExecutorConfiguration( String[] tmpDirPaths, long cleanupInterval, - InstanceConnectionInfo connectionInfo, NetworkEnvironmentConfiguration networkConfig, FiniteDuration timeout, FiniteDuration maxRegistrationDuration, @@ -92,7 +86,6 @@ public TaskExecutorConfiguration( this.tmpDirPaths = checkNotNull(tmpDirPaths); this.cleanupInterval = checkNotNull(cleanupInterval); - this.connectionInfo = checkNotNull(connectionInfo); this.networkConfig = checkNotNull(networkConfig); this.timeout = checkNotNull(timeout); this.maxRegistrationDuration = maxRegistrationDuration; @@ -115,8 +108,6 @@ public long getCleanupInterval() { return cleanupInterval; } - public InstanceConnectionInfo getConnectionInfo() { return connectionInfo; } - public NetworkEnvironmentConfiguration getNetworkConfig() { return networkConfig; } public FiniteDuration getTimeout() { From 532877557f92f7f1809f5440a642ba090c337f5b Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Thu, 1 Sep 2016 16:53:31 +0200 Subject: [PATCH 27/50] [FLINK-4538][FLINK-4348] ResourceManager slot allocation protcol - associates JobMasters with JobID instead of InstanceID - adds TaskExecutorGateway to slot - adds SlotManager as RM constructor parameter - adds LeaderRetrievalListener to SlotManager to keep track of the leader id - tests the interaction JM->RM requestSlot - tests the interaction RM->TM requestSlot This closes #2463 --- .../types/ResourceProfile.java | 2 +- .../clusterframework/types/ResourceSlot.java | 14 +- .../JobMasterRegistration.java | 10 +- .../resourcemanager/RegistrationResponse.java | 9 +- .../resourcemanager/ResourceManager.java | 167 +++++++------ .../ResourceManagerGateway.java | 2 +- ...gnment.java => SlotRequestRegistered.java} | 12 +- .../resourcemanager/SlotRequestRejected.java | 34 +++ .../resourcemanager/SlotRequestReply.java | 41 ++++ .../slotmanager/SimpleSlotManager.java | 59 +++++ .../{ => slotmanager}/SlotManager.java | 106 +++++++-- .../runtime/taskexecutor/SlotStatus.java | 5 +- .../taskexecutor/TaskExecutorGateway.java | 17 ++ .../ResourceManagerHATest.java | 4 +- .../{ => slotmanager}/SlotManagerTest.java | 92 ++++--- .../slotmanager/SlotProtocolTest.java | 225 ++++++++++++++++++ .../flink/runtime/rpc/TestingRpcService.java | 6 +- .../runtime/rpc/TestingSerialRpcService.java | 4 + 18 files changed, 656 insertions(+), 153 deletions(-) rename flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/{SlotAssignment.java => SlotRequestRegistered.java} (74%) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRejected.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestReply.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SimpleSlotManager.java rename flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/{ => slotmanager}/SlotManager.java (84%) rename flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/{ => slotmanager}/SlotManagerTest.java (86%) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java index ff1c4bfd3b8b3..fa3aabc458ab9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java @@ -68,6 +68,6 @@ public long getMemoryInMB() { * @return true if the requirement is matched, otherwise false */ public boolean isMatching(ResourceProfile required) { - return Double.compare(cpuCores, required.getCpuCores()) >= 0 && memoryInMB >= required.getMemoryInMB(); + return cpuCores >= required.getCpuCores() && memoryInMB >= required.getMemoryInMB(); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java index 8a6db5f5cfc8a..5fb8aee8c9238 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java @@ -18,6 +18,8 @@ package org.apache.flink.runtime.clusterframework.types; +import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; + import java.io.Serializable; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -26,7 +28,7 @@ * A ResourceSlot represents a slot located in TaskManager from ResourceManager's view. It has a unique * identification and resource profile which we can compare to the resource request. */ -public class ResourceSlot implements ResourceIDRetrievable, Serializable { +public class ResourceSlot implements ResourceIDRetrievable { private static final long serialVersionUID = -5853720153136840674L; @@ -36,9 +38,13 @@ public class ResourceSlot implements ResourceIDRetrievable, Serializable { /** The resource profile of this slot */ private final ResourceProfile resourceProfile; - public ResourceSlot(SlotID slotId, ResourceProfile resourceProfile) { + /** Gateway to the TaskExecutor which owns the slot */ + private final TaskExecutorGateway taskExecutorGateway; + + public ResourceSlot(SlotID slotId, ResourceProfile resourceProfile, TaskExecutorGateway taskExecutorGateway) { this.slotId = checkNotNull(slotId); this.resourceProfile = checkNotNull(resourceProfile); + this.taskExecutorGateway = taskExecutorGateway; } @Override @@ -54,6 +60,10 @@ public ResourceProfile getResourceProfile() { return resourceProfile; } + public TaskExecutorGateway getTaskExecutorGateway() { + return taskExecutorGateway; + } + /** * Check whether required resource profile can be matched by this slot. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java index 309dcc1762619..439e56b0402d8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java @@ -18,18 +18,26 @@ package org.apache.flink.runtime.resourcemanager; +import org.apache.flink.api.common.JobID; + import java.io.Serializable; public class JobMasterRegistration implements Serializable { private static final long serialVersionUID = 8411214999193765202L; private final String address; + private final JobID jobID; - public JobMasterRegistration(String address) { + public JobMasterRegistration(String address, JobID jobID) { this.address = address; + this.jobID = jobID; } public String getAddress() { return address; } + + public JobID getJobID() { + return jobID; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/RegistrationResponse.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/RegistrationResponse.java index fb6c401a1241e..796e634ed37a4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/RegistrationResponse.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/RegistrationResponse.java @@ -18,26 +18,19 @@ package org.apache.flink.runtime.resourcemanager; -import org.apache.flink.runtime.instance.InstanceID; - import java.io.Serializable; public class RegistrationResponse implements Serializable { private static final long serialVersionUID = -2379003255993119993L; private final boolean isSuccess; - private final InstanceID instanceID; - public RegistrationResponse(boolean isSuccess, InstanceID instanceID) { + public RegistrationResponse(boolean isSuccess) { this.isSuccess = isSuccess; - this.instanceID = instanceID; } public boolean isSuccess() { return isSuccess; } - public InstanceID getInstanceID() { - return instanceID; - } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java index 44c022b748c61..29aba1ad3de03 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java @@ -21,11 +21,13 @@ import akka.dispatch.Mapper; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.leaderelection.LeaderContender; import org.apache.flink.runtime.leaderelection.LeaderElectionService; +import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; import org.apache.flink.runtime.rpc.RpcMethod; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcService; @@ -33,6 +35,8 @@ import org.apache.flink.runtime.jobmaster.JobMasterGateway; import org.apache.flink.runtime.taskexecutor.TaskExecutorRegistrationSuccess; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import scala.concurrent.Future; import java.util.HashMap; @@ -51,16 +55,28 @@ *
  • {@link #requestSlot(SlotRequest)} requests a slot from the resource manager
  • * */ -public class ResourceManager extends RpcEndpoint { - private final Map jobMasterGateways; +public class ResourceManager extends RpcEndpoint implements LeaderContender { + + private final Logger LOG = LoggerFactory.getLogger(getClass()); + + private final Map jobMasterGateways; + private final HighAvailabilityServices highAvailabilityServices; - private LeaderElectionService leaderElectionService = null; - private UUID leaderSessionID = null; - public ResourceManager(RpcService rpcService, HighAvailabilityServices highAvailabilityServices) { + private LeaderElectionService leaderElectionService; + + private final SlotManager slotManager; + + private UUID leaderSessionID; + + public ResourceManager( + RpcService rpcService, + HighAvailabilityServices highAvailabilityServices, + SlotManager slotManager) { super(rpcService); this.highAvailabilityServices = checkNotNull(highAvailabilityServices); this.jobMasterGateways = new HashMap<>(); + this.slotManager = slotManager; } @Override @@ -69,7 +85,7 @@ public void start() { try { super.start(); leaderElectionService = highAvailabilityServices.getResourceManagerLeaderElectionService(); - leaderElectionService.start(new ResourceManagerLeaderContender()); + leaderElectionService.start(this); } catch (Throwable e) { log.error("A fatal error happened when starting the ResourceManager", e); throw new RuntimeException("A fatal error happened when starting the ResourceManager", e); @@ -94,7 +110,7 @@ public void shutDown() { */ @VisibleForTesting UUID getLeaderSessionID() { - return leaderSessionID; + return this.leaderSessionID; } /** @@ -105,21 +121,20 @@ UUID getLeaderSessionID() { */ @RpcMethod public Future registerJobMaster(JobMasterRegistration jobMasterRegistration) { - Future jobMasterFuture = getRpcService().connect(jobMasterRegistration.getAddress(), JobMasterGateway.class); + final Future jobMasterFuture = + getRpcService().connect(jobMasterRegistration.getAddress(), JobMasterGateway.class); + final JobID jobID = jobMasterRegistration.getJobID(); return jobMasterFuture.map(new Mapper() { @Override public RegistrationResponse apply(final JobMasterGateway jobMasterGateway) { - InstanceID instanceID; - if (jobMasterGateways.containsKey(jobMasterGateway)) { - instanceID = jobMasterGateways.get(jobMasterGateway); - } else { - instanceID = new InstanceID(); - jobMasterGateways.put(jobMasterGateway, instanceID); + final JobMasterGateway existingGateway = jobMasterGateways.put(jobID, jobMasterGateway); + if (existingGateway != null) { + LOG.info("Replacing existing gateway {} for JobID {} with {}.", + existingGateway, jobID, jobMasterGateway); } - - return new RegistrationResponse(true, instanceID); + return new RegistrationResponse(true); } }, getMainThreadExecutionContext()); } @@ -131,9 +146,16 @@ public RegistrationResponse apply(final JobMasterGateway jobMasterGateway) { * @return Slot assignment */ @RpcMethod - public SlotAssignment requestSlot(SlotRequest slotRequest) { - System.out.println("SlotRequest: " + slotRequest); - return new SlotAssignment(); + public SlotRequestReply requestSlot(SlotRequest slotRequest) { + final JobID jobId = slotRequest.getJobId(); + final JobMasterGateway jobMasterGateway = jobMasterGateways.get(jobId); + + if (jobMasterGateway != null) { + return slotManager.requestSlot(slotRequest); + } else { + LOG.info("Ignoring slot request for unknown JobMaster with JobID {}", jobId); + return new SlotRequestRejected(slotRequest.getAllocationId()); + } } @@ -154,61 +176,62 @@ public org.apache.flink.runtime.registration.RegistrationResponse registerTaskEx return new TaskExecutorRegistrationSuccess(new InstanceID(), 5000); } - private class ResourceManagerLeaderContender implements LeaderContender { - - /** - * Callback method when current resourceManager is granted leadership - * - * @param leaderSessionID unique leadershipID - */ - @Override - public void grantLeadership(final UUID leaderSessionID) { - runAsync(new Runnable() { - @Override - public void run() { - log.info("ResourceManager {} was granted leadership with leader session ID {}", getAddress(), leaderSessionID); - ResourceManager.this.leaderSessionID = leaderSessionID; - // confirming the leader session ID might be blocking, - leaderElectionService.confirmLeaderSessionID(leaderSessionID); - } - }); - } - /** - * Callback method when current resourceManager lose leadership. - */ - @Override - public void revokeLeadership() { - runAsync(new Runnable() { - @Override - public void run() { - log.info("ResourceManager {} was revoked leadership.", getAddress()); - jobMasterGateways.clear(); - leaderSessionID = null; - } - }); - } + // ------------------------------------------------------------------------ + // Leader Contender + // ------------------------------------------------------------------------ - @Override - public String getAddress() { - return ResourceManager.this.getAddress(); - } + /** + * Callback method when current resourceManager is granted leadership + * + * @param leaderSessionID unique leadershipID + */ + @Override + public void grantLeadership(final UUID leaderSessionID) { + runAsync(new Runnable() { + @Override + public void run() { + log.info("ResourceManager {} was granted leadership with leader session ID {}", getAddress(), leaderSessionID); + // confirming the leader session ID might be blocking, + leaderElectionService.confirmLeaderSessionID(leaderSessionID); + // notify SlotManager + slotManager.notifyLeaderAddress(getAddress(), leaderSessionID); + ResourceManager.this.leaderSessionID = leaderSessionID; + } + }); + } - /** - * Handles error occurring in the leader election service - * - * @param exception Exception being thrown in the leader election service - */ - @Override - public void handleError(final Exception exception) { - runAsync(new Runnable() { - @Override - public void run() { - log.error("ResourceManager received an error from the LeaderElectionService.", exception); - // terminate ResourceManager in case of an error - shutDown(); - } - }); - } + /** + * Callback method when current resourceManager lose leadership. + */ + @Override + public void revokeLeadership() { + runAsync(new Runnable() { + @Override + public void run() { + log.info("ResourceManager {} was revoked leadership.", getAddress()); + jobMasterGateways.clear(); + ResourceManager.this.leaderSessionID = null; + } + }); + } + + /** + * Handles error occurring in the leader election service + * + * @param exception Exception being thrown in the leader election service + */ + @Override + public void handleError(final Exception exception) { + runAsync(new Runnable() { + @Override + public void run() { + log.error("ResourceManager received an error from the LeaderElectionService.", exception); + // notify SlotManager + slotManager.handleError(exception); + // terminate ResourceManager in case of an error + shutDown(); + } + }); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java index b5782b06f1819..e5c8b64060994 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java @@ -58,7 +58,7 @@ Future registerJobMaster( * @param slotRequest Slot request * @return Future slot assignment */ - Future requestSlot(SlotRequest slotRequest); + Future requestSlot(SlotRequest slotRequest); /** * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotAssignment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRegistered.java similarity index 74% rename from flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotAssignment.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRegistered.java index 695204d751c6e..6b7f6dc20fdf6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotAssignment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRegistered.java @@ -18,8 +18,16 @@ package org.apache.flink.runtime.resourcemanager; +import org.apache.flink.runtime.clusterframework.types.AllocationID; + import java.io.Serializable; -public class SlotAssignment implements Serializable{ - private static final long serialVersionUID = -6990813455942742322L; +/** + * Acknowledgment by the ResourceManager for a SlotRequest from the JobManager + */ +public class SlotRequestRegistered extends SlotRequestReply { + + public SlotRequestRegistered(AllocationID allocationID) { + super(allocationID); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRejected.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRejected.java new file mode 100644 index 0000000000000..cb3ec72b6dff2 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRejected.java @@ -0,0 +1,34 @@ +/* + * 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.resourcemanager; + +import org.apache.flink.runtime.clusterframework.types.AllocationID; + +import java.io.Serializable; + +/** + * Rejection message by the ResourceManager for a SlotRequest from the JobManager + */ +public class SlotRequestRejected extends SlotRequestReply { + + public SlotRequestRejected(AllocationID allocationID) { + super(allocationID); + } + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestReply.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestReply.java new file mode 100644 index 0000000000000..1b85d0c907033 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestReply.java @@ -0,0 +1,41 @@ +/* + * 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.resourcemanager; + +import org.apache.flink.runtime.clusterframework.types.AllocationID; + +import java.io.Serializable; + +/** + * Acknowledgment by the ResourceManager for a SlotRequest from the JobManager + */ +public abstract class SlotRequestReply implements Serializable { + + private static final long serialVersionUID = 42; + + private final AllocationID allocationID; + + public SlotRequestReply(AllocationID allocationID) { + this.allocationID = allocationID; + } + + public AllocationID getAllocationID() { + return allocationID; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SimpleSlotManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SimpleSlotManager.java new file mode 100644 index 0000000000000..ef5ce3128d85a --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SimpleSlotManager.java @@ -0,0 +1,59 @@ +/* + * 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.resourcemanager.slotmanager; + +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.clusterframework.types.ResourceSlot; +import org.apache.flink.runtime.clusterframework.types.SlotID; +import org.apache.flink.runtime.resourcemanager.SlotRequest; + +import java.util.Iterator; +import java.util.Map; + +/** + * A simple SlotManager which ignores resource profiles. + */ +public class SimpleSlotManager extends SlotManager { + + @Override + protected ResourceSlot chooseSlotToUse(SlotRequest request, Map freeSlots) { + final Iterator slotIterator = freeSlots.values().iterator(); + if (slotIterator.hasNext()) { + return slotIterator.next(); + } else { + return null; + } + } + + @Override + protected SlotRequest chooseRequestToFulfill(ResourceSlot offeredSlot, Map pendingRequests) { + final Iterator requestIterator = pendingRequests.values().iterator(); + if (requestIterator.hasNext()) { + return requestIterator.next(); + } else { + return null; + } + } + + @Override + protected void allocateContainer(ResourceProfile resourceProfile) { + // TODO + } + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java similarity index 84% rename from flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotManager.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java index 5c066480aa4eb..96fde7d1fe7c2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.resourcemanager; +package org.apache.flink.runtime.resourcemanager.slotmanager; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.runtime.clusterframework.types.AllocationID; @@ -24,14 +24,23 @@ import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.clusterframework.types.ResourceSlot; import org.apache.flink.runtime.clusterframework.types.SlotID; +import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener; +import org.apache.flink.runtime.resourcemanager.SlotRequest; +import org.apache.flink.runtime.resourcemanager.SlotRequestRegistered; +import org.apache.flink.runtime.resourcemanager.SlotRequestReply; import org.apache.flink.runtime.taskexecutor.SlotReport; import org.apache.flink.runtime.taskexecutor.SlotStatus; +import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import scala.concurrent.Future; +import scala.concurrent.duration.FiniteDuration; import java.util.HashMap; import java.util.LinkedHashMap; import java.util.Map; +import java.util.UUID; +import java.util.concurrent.TimeUnit; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -51,12 +60,12 @@ * * IMPORTANT: This class is Not Thread-safe. */ -public abstract class SlotManager { +public abstract class SlotManager implements LeaderRetrievalListener { - private static final Logger LOG = LoggerFactory.getLogger(SlotManager.class); + protected final Logger LOG = LoggerFactory.getLogger(getClass()); - /** Gateway to communicate with ResourceManager */ - private final ResourceManagerGateway resourceManagerGateway; + /** All registered task managers with ResourceID and gateway. */ + private final Map taskManagerGateways; /** All registered slots, including free and allocated slots */ private final Map> registeredSlots; @@ -70,14 +79,21 @@ public abstract class SlotManager { /** All allocations, we can lookup allocations either by SlotID or AllocationID */ private final AllocationMap allocationMap; - public SlotManager(ResourceManagerGateway resourceManagerGateway) { - this.resourceManagerGateway = checkNotNull(resourceManagerGateway); + private final FiniteDuration timeout; + + /** The current leader id set by the ResourceManager */ + private UUID leaderID; + + public SlotManager() { this.registeredSlots = new HashMap<>(16); this.pendingSlotRequests = new LinkedHashMap<>(16); this.freeSlots = new HashMap<>(16); this.allocationMap = new AllocationMap(); + this.taskManagerGateways = new HashMap<>(); + this.timeout = new FiniteDuration(10, TimeUnit.SECONDS); } + // ------------------------------------------------------------------------ // slot managements // ------------------------------------------------------------------------ @@ -89,32 +105,38 @@ public SlotManager(ResourceManagerGateway resourceManagerGateway) { * RPC's main thread to avoid race condition). * * @param request The detailed request of the slot + * @return SlotRequestRegistered The confirmation message to be send to the caller */ - public void requestSlot(final SlotRequest request) { + public SlotRequestRegistered requestSlot(final SlotRequest request) { + final AllocationID allocationId = request.getAllocationId(); if (isRequestDuplicated(request)) { - LOG.warn("Duplicated slot request, AllocationID:{}", request.getAllocationId()); - return; + LOG.warn("Duplicated slot request, AllocationID:{}", allocationId); + return null; } // try to fulfil the request with current free slots - ResourceSlot slot = chooseSlotToUse(request, freeSlots); + final ResourceSlot slot = chooseSlotToUse(request, freeSlots); if (slot != null) { LOG.info("Assigning SlotID({}) to AllocationID({}), JobID:{}", slot.getSlotId(), - request.getAllocationId(), request.getJobId()); + allocationId, request.getJobId()); // record this allocation in bookkeeping - allocationMap.addAllocation(slot.getSlotId(), request.getAllocationId()); + allocationMap.addAllocation(slot.getSlotId(), allocationId); // remove selected slot from free pool freeSlots.remove(slot.getSlotId()); - // TODO: send slot request to TaskManager + final Future slotRequestReplyFuture = + slot.getTaskExecutorGateway().requestSlot(allocationId, leaderID, timeout); + // TODO handle timeouts and response } else { LOG.info("Cannot fulfil slot request, try to allocate a new container for it, " + - "AllocationID:{}, JobID:{}", request.getAllocationId(), request.getJobId()); + "AllocationID:{}, JobID:{}", allocationId, request.getJobId()); allocateContainer(request.getResourceProfile()); - pendingSlotRequests.put(request.getAllocationId(), request); + pendingSlotRequests.put(allocationId, request); } + + return new SlotRequestRegistered(allocationId); } /** @@ -126,6 +148,15 @@ public void updateSlotStatus(final SlotReport slotReport) { } } + /** + * Registers a TaskExecutor + * @param resourceID TaskExecutor's ResourceID + * @param gateway TaskExcutor's gateway + */ + public void registerTaskExecutor(ResourceID resourceID, TaskExecutorGateway gateway) { + this.taskManagerGateways.put(resourceID, gateway); + } + /** * The slot request to TaskManager may be either failed by rpc communication (timeout, network error, etc.) * or really rejected by TaskManager. We shall retry this request by: @@ -196,10 +227,11 @@ public void handleSlotRequestFailedAtTaskManager(final SlotRequest originalReque */ public void notifyTaskManagerFailure(final ResourceID resourceId) { LOG.info("Resource:{} been notified failure", resourceId); + taskManagerGateways.remove(resourceId); final Map slotIdsToRemove = registeredSlots.remove(resourceId); if (slotIdsToRemove != null) { for (SlotID slotId : slotIdsToRemove.keySet()) { - LOG.info("Removing Slot:{} upon resource failure", slotId); + LOG.info("Removing Slot: {} upon resource failure", slotId); if (freeSlots.containsKey(slotId)) { freeSlots.remove(slotId); } else if (allocationMap.isAllocated(slotId)) { @@ -234,7 +266,15 @@ public void notifyTaskManagerFailure(final ResourceID resourceId) { */ void updateSlotStatus(final SlotStatus reportedStatus) { final SlotID slotId = reportedStatus.getSlotID(); - final ResourceSlot slot = new ResourceSlot(slotId, reportedStatus.getProfiler()); + + final TaskExecutorGateway taskExecutorGateway = taskManagerGateways.get(slotId.getResourceID()); + if (taskExecutorGateway == null) { + LOG.info("Received SlotStatus but ResourceID {} is unknown to the SlotManager", + slotId.getResourceID()); + return; + } + + final ResourceSlot slot = new ResourceSlot(slotId, reportedStatus.getProfiler(), taskExecutorGateway); if (registerNewSlot(slot)) { // we have a newly registered slot @@ -244,7 +284,7 @@ void updateSlotStatus(final SlotStatus reportedStatus) { // slot in use, record this in bookkeeping allocationMap.addAllocation(slotId, reportedStatus.getAllocationID()); } else { - handleFreeSlot(new ResourceSlot(slotId, reportedStatus.getProfiler())); + handleFreeSlot(slot); } } else { // slot exists, update current information @@ -287,7 +327,7 @@ void updateSlotStatus(final SlotStatus reportedStatus) { allocationMap.removeAllocation(slotId); // we have a free slot! - handleFreeSlot(new ResourceSlot(slotId, reportedStatus.getProfiler())); + handleFreeSlot(slot); } } } @@ -304,13 +344,16 @@ private void handleFreeSlot(final ResourceSlot freeSlot) { SlotRequest chosenRequest = chooseRequestToFulfill(freeSlot, pendingSlotRequests); if (chosenRequest != null) { - pendingSlotRequests.remove(chosenRequest.getAllocationId()); + final AllocationID allocationId = chosenRequest.getAllocationId(); + pendingSlotRequests.remove(allocationId); LOG.info("Assigning SlotID({}) to AllocationID({}), JobID:{}", freeSlot.getSlotId(), - chosenRequest.getAllocationId(), chosenRequest.getJobId()); - allocationMap.addAllocation(freeSlot.getSlotId(), chosenRequest.getAllocationId()); + allocationId, chosenRequest.getJobId()); + allocationMap.addAllocation(freeSlot.getSlotId(), allocationId); - // TODO: send slot request to TaskManager + final Future slotRequestReplyFuture = + freeSlot.getTaskExecutorGateway().requestSlot(allocationId, leaderID, timeout); + // TODO handle timeouts and response } else { freeSlots.put(freeSlot.getSlotId(), freeSlot); } @@ -383,7 +426,6 @@ protected abstract SlotRequest chooseRequestToFulfill(final ResourceSlot offered */ protected abstract void allocateContainer(final ResourceProfile resourceProfile); - // ------------------------------------------------------------------------ // Helper classes // ------------------------------------------------------------------------ @@ -475,6 +517,20 @@ public int size() { } } + // ------------------------------------------------------------------------ + // High availability + // ------------------------------------------------------------------------ + + @Override + public void notifyLeaderAddress(String leaderAddress, UUID leaderSessionID) { + this.leaderID = leaderSessionID; + } + + @Override + public void handleError(Exception exception) { + LOG.error("Slot Manager received an error from the leader service", exception); + } + // ------------------------------------------------------------------------ // Testing utilities // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotStatus.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotStatus.java index 744b674a3af7b..0f57bb15cdd8c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotStatus.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotStatus.java @@ -50,7 +50,10 @@ public SlotStatus(SlotID slotID, ResourceProfile profiler) { this(slotID, profiler, null, null); } - public SlotStatus(SlotID slotID, ResourceProfile profiler, AllocationID allocationID, JobID jobID) { + public SlotStatus( + SlotID slotID, ResourceProfile profiler, + JobID jobID, + AllocationID allocationID) { this.slotID = checkNotNull(slotID, "slotID cannot be null"); this.profiler = checkNotNull(profiler, "profile cannot be null"); this.allocationID = allocationID; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java index 6c99706c6f677..72574367ad78c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java @@ -18,7 +18,12 @@ package org.apache.flink.runtime.taskexecutor; +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.resourcemanager.SlotRequestReply; import org.apache.flink.runtime.rpc.RpcGateway; +import org.apache.flink.runtime.rpc.RpcTimeout; +import scala.concurrent.Future; +import scala.concurrent.duration.FiniteDuration; import java.util.UUID; @@ -32,4 +37,16 @@ public interface TaskExecutorGateway extends RpcGateway { // ------------------------------------------------------------------------ void notifyOfNewResourceManagerLeader(String address, UUID resourceManagerLeaderId); + + /** + * Send by the ResourceManager to the TaskExecutor + * @param allocationID id for the request + * @param resourceManagerLeaderID current leader id of the ResourceManager + * @return SlotRequestReply Answer to the request + */ + + Future requestSlot( + AllocationID allocationID, + UUID resourceManagerLeaderID, + @RpcTimeout FiniteDuration timeout); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java index 5799e62686101..8183c0a857562 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java @@ -20,6 +20,7 @@ import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; +import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; import org.apache.flink.runtime.rpc.MainThreadExecutor; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcGateway; @@ -53,7 +54,8 @@ public void testGrantAndRevokeLeadership() throws Exception { TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices(); highAvailabilityServices.setResourceManagerLeaderElectionService(leaderElectionService); - final ResourceManager resourceManager = new ResourceManager(rpcService, highAvailabilityServices); + SlotManager slotManager = mock(SlotManager.class); + final ResourceManager resourceManager = new ResourceManager(rpcService, highAvailabilityServices, slotManager); resourceManager.start(); // before grant leadership, resourceManager's leaderId is null Assert.assertNull(resourceManager.getLeaderSessionID()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/SlotManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerTest.java similarity index 86% rename from flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/SlotManagerTest.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerTest.java index 52d9d06400488..9ee96903ad028 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/SlotManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.resourcemanager; +package org.apache.flink.runtime.resourcemanager.slotmanager; import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.clusterframework.types.AllocationID; @@ -24,9 +24,12 @@ import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.clusterframework.types.ResourceSlot; import org.apache.flink.runtime.clusterframework.types.SlotID; +import org.apache.flink.runtime.resourcemanager.SlotRequest; import org.apache.flink.runtime.taskexecutor.SlotStatus; -import org.junit.Before; +import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; +import org.junit.BeforeClass; import org.junit.Test; +import org.mockito.Mockito; import java.util.LinkedList; import java.util.List; @@ -49,11 +52,11 @@ public class SlotManagerTest { private static final ResourceProfile DEFAULT_TESTING_BIG_PROFILE = new ResourceProfile(DEFAULT_TESTING_CPU_CORES * 2, DEFAULT_TESTING_MEMORY * 2); - private ResourceManagerGateway resourceManagerGateway; + private static TaskExecutorGateway taskExecutorGateway; - @Before - public void setUp() { - resourceManagerGateway = mock(ResourceManagerGateway.class); + @BeforeClass + public static void setUp() { + taskExecutorGateway = Mockito.mock(TaskExecutorGateway.class); } /** @@ -61,7 +64,7 @@ public void setUp() { */ @Test public void testRequestSlotWithoutFreeSlot() { - TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); + TestingSlotManager slotManager = new TestingSlotManager(); slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE)); assertEquals(0, slotManager.getAllocatedSlotCount()); @@ -76,7 +79,7 @@ public void testRequestSlotWithoutFreeSlot() { */ @Test public void testRequestSlotWithFreeSlot() { - TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); + TestingSlotManager slotManager = new TestingSlotManager(); directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 1); assertEquals(1, slotManager.getFreeSlotCount()); @@ -93,7 +96,7 @@ public void testRequestSlotWithFreeSlot() { */ @Test public void testRequestSlotWithoutSuitableSlot() { - TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); + TestingSlotManager slotManager = new TestingSlotManager(); directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 2); assertEquals(2, slotManager.getFreeSlotCount()); @@ -111,7 +114,7 @@ public void testRequestSlotWithoutSuitableSlot() { */ @Test public void testDuplicatedSlotRequest() { - TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); + TestingSlotManager slotManager = new TestingSlotManager(); directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 1); SlotRequest request1 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE); @@ -134,7 +137,7 @@ public void testDuplicatedSlotRequest() { */ @Test public void testRequestMultipleSlots() { - TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); + TestingSlotManager slotManager = new TestingSlotManager(); directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 5); // request 3 normal slots @@ -163,11 +166,12 @@ public void testRequestMultipleSlots() { */ @Test public void testNewlyAppearedFreeSlotFulfillPendingRequest() { - TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); + TestingSlotManager slotManager = new TestingSlotManager(); slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE)); assertEquals(1, slotManager.getPendingRequestCount()); SlotID slotId = SlotID.generate(); + slotManager.registerTaskExecutor(slotId.getResourceID(), taskExecutorGateway); SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE); slotManager.updateSlotStatus(slotStatus); @@ -182,8 +186,10 @@ public void testNewlyAppearedFreeSlotFulfillPendingRequest() { */ @Test public void testNewlyAppearedFreeSlot() { - TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); + TestingSlotManager slotManager = new TestingSlotManager(); + SlotID slotId = SlotID.generate(); + slotManager.registerTaskExecutor(slotId.getResourceID(), taskExecutorGateway); SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE); slotManager.updateSlotStatus(slotStatus); @@ -196,11 +202,12 @@ public void testNewlyAppearedFreeSlot() { */ @Test public void testNewlyAppearedFreeSlotNotMatchPendingRequests() { - TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); + TestingSlotManager slotManager = new TestingSlotManager(); slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_BIG_PROFILE)); assertEquals(1, slotManager.getPendingRequestCount()); SlotID slotId = SlotID.generate(); + slotManager.registerTaskExecutor(slotId.getResourceID(), taskExecutorGateway); SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE); slotManager.updateSlotStatus(slotStatus); @@ -215,10 +222,11 @@ public void testNewlyAppearedFreeSlotNotMatchPendingRequests() { */ @Test public void testNewlyAppearedInUseSlot() { - TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); + TestingSlotManager slotManager = new TestingSlotManager(); SlotID slotId = SlotID.generate(); - SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE, new AllocationID(), new JobID()); + slotManager.registerTaskExecutor(slotId.getResourceID(), taskExecutorGateway); + SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE, new JobID(), new AllocationID()); slotManager.updateSlotStatus(slotStatus); assertEquals(1, slotManager.getAllocatedSlotCount()); @@ -231,12 +239,13 @@ public void testNewlyAppearedInUseSlot() { */ @Test public void testExistingInUseSlotUpdateStatus() { - TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); + TestingSlotManager slotManager = new TestingSlotManager(); SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE); slotManager.requestSlot(request); // make this slot in use SlotID slotId = SlotID.generate(); + slotManager.registerTaskExecutor(slotId.getResourceID(), taskExecutorGateway); SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE); slotManager.updateSlotStatus(slotStatus); @@ -246,7 +255,7 @@ public void testExistingInUseSlotUpdateStatus() { // slot status is confirmed SlotStatus slotStatus2 = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE, - request.getAllocationId(), request.getJobId()); + request.getJobId(), request.getAllocationId()); slotManager.updateSlotStatus(slotStatus2); assertEquals(1, slotManager.getAllocatedSlotCount()); @@ -259,12 +268,13 @@ public void testExistingInUseSlotUpdateStatus() { */ @Test public void testExistingInUseSlotAdjustedToEmpty() { - TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); + TestingSlotManager slotManager = new TestingSlotManager(); SlotRequest request1 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE); slotManager.requestSlot(request1); // make this slot in use SlotID slotId = SlotID.generate(); + slotManager.registerTaskExecutor(slotId.getResourceID(), taskExecutorGateway); SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE); slotManager.updateSlotStatus(slotStatus); @@ -295,12 +305,13 @@ public void testExistingInUseSlotAdjustedToEmpty() { */ @Test public void testExistingInUseSlotWithDifferentAllocationInfo() { - TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); + TestingSlotManager slotManager = new TestingSlotManager(); SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE); slotManager.requestSlot(request); // make this slot in use SlotID slotId = SlotID.generate(); + slotManager.registerTaskExecutor(slotId.getResourceID(), taskExecutorGateway); SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE); slotManager.updateSlotStatus(slotStatus); @@ -310,7 +321,7 @@ public void testExistingInUseSlotWithDifferentAllocationInfo() { assertTrue(slotManager.isAllocated(slotId)); assertTrue(slotManager.isAllocated(request.getAllocationId())); - SlotStatus slotStatus2 = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE, new AllocationID(), new JobID()); + SlotStatus slotStatus2 = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE, new JobID(), new AllocationID()); // update slot status with different allocation info slotManager.updateSlotStatus(slotStatus2); @@ -328,8 +339,8 @@ public void testExistingInUseSlotWithDifferentAllocationInfo() { */ @Test public void testExistingEmptySlotUpdateStatus() { - TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); - ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE); + TestingSlotManager slotManager = new TestingSlotManager(); + ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE, taskExecutorGateway); slotManager.addFreeSlot(slot); SlotStatus slotStatus = new SlotStatus(slot.getSlotId(), DEFAULT_TESTING_PROFILE); @@ -345,12 +356,15 @@ public void testExistingEmptySlotUpdateStatus() { */ @Test public void testExistingEmptySlotAdjustedToInUse() { - TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); - ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE); + TestingSlotManager slotManager = new TestingSlotManager(); + final SlotID slotID = SlotID.generate(); + slotManager.registerTaskExecutor(slotID.getResourceID(), taskExecutorGateway); + + ResourceSlot slot = new ResourceSlot(slotID, DEFAULT_TESTING_PROFILE, taskExecutorGateway); slotManager.addFreeSlot(slot); SlotStatus slotStatus = new SlotStatus(slot.getSlotId(), DEFAULT_TESTING_PROFILE, - new AllocationID(), new JobID()); + new JobID(), new AllocationID()); slotManager.updateSlotStatus(slotStatus); assertEquals(1, slotManager.getAllocatedSlotCount()); @@ -364,8 +378,8 @@ public void testExistingEmptySlotAdjustedToInUse() { */ @Test public void testSlotAllocationFailedAtTaskManager() { - TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); - ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE); + TestingSlotManager slotManager = new TestingSlotManager(); + ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE, taskExecutorGateway); slotManager.addFreeSlot(slot); SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE); @@ -389,8 +403,11 @@ public void testSlotAllocationFailedAtTaskManager() { */ @Test public void testSlotAllocationFailedAtTaskManagerOccupiedByOther() { - TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); - ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE); + TestingSlotManager slotManager = new TestingSlotManager(); + final SlotID slotID = SlotID.generate(); + slotManager.registerTaskExecutor(slotID.getResourceID(), taskExecutorGateway); + + ResourceSlot slot = new ResourceSlot(slotID, DEFAULT_TESTING_PROFILE, taskExecutorGateway); slotManager.addFreeSlot(slot); SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE); @@ -422,15 +439,15 @@ public void testSlotAllocationFailedAtTaskManagerOccupiedByOther() { @Test public void testNotifyTaskManagerFailure() { - TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); + TestingSlotManager slotManager = new TestingSlotManager(); ResourceID resource1 = ResourceID.generate(); ResourceID resource2 = ResourceID.generate(); - ResourceSlot slot11 = new ResourceSlot(new SlotID(resource1, 1), DEFAULT_TESTING_PROFILE); - ResourceSlot slot12 = new ResourceSlot(new SlotID(resource1, 2), DEFAULT_TESTING_PROFILE); - ResourceSlot slot21 = new ResourceSlot(new SlotID(resource2, 1), DEFAULT_TESTING_PROFILE); - ResourceSlot slot22 = new ResourceSlot(new SlotID(resource2, 2), DEFAULT_TESTING_PROFILE); + ResourceSlot slot11 = new ResourceSlot(new SlotID(resource1, 1), DEFAULT_TESTING_PROFILE, taskExecutorGateway); + ResourceSlot slot12 = new ResourceSlot(new SlotID(resource1, 2), DEFAULT_TESTING_PROFILE, taskExecutorGateway); + ResourceSlot slot21 = new ResourceSlot(new SlotID(resource2, 1), DEFAULT_TESTING_PROFILE, taskExecutorGateway); + ResourceSlot slot22 = new ResourceSlot(new SlotID(resource2, 2), DEFAULT_TESTING_PROFILE, taskExecutorGateway); slotManager.addFreeSlot(slot11); slotManager.addFreeSlot(slot21); @@ -473,7 +490,7 @@ private void directlyProvideFreeSlots( final int freeSlotNum) { for (int i = 0; i < freeSlotNum; ++i) { - slotManager.addFreeSlot(new ResourceSlot(SlotID.generate(), new ResourceProfile(resourceProfile))); + slotManager.addFreeSlot(new ResourceSlot(SlotID.generate(), new ResourceProfile(resourceProfile), taskExecutorGateway)); } } @@ -485,8 +502,7 @@ private static class TestingSlotManager extends SlotManager { private final List allocatedContainers; - TestingSlotManager(ResourceManagerGateway resourceManagerGateway) { - super(resourceManagerGateway); + TestingSlotManager() { this.allocatedContainers = new LinkedList<>(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java new file mode 100644 index 0000000000000..85d28805d4d64 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java @@ -0,0 +1,225 @@ +/* + * 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.resourcemanager.slotmanager; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.clusterframework.types.SlotID; +import org.apache.flink.runtime.highavailability.NonHaServices; +import org.apache.flink.runtime.jobmaster.JobMasterGateway; +import org.apache.flink.runtime.resourcemanager.JobMasterRegistration; +import org.apache.flink.runtime.resourcemanager.RegistrationResponse; +import org.apache.flink.runtime.resourcemanager.ResourceManager; +import org.apache.flink.runtime.resourcemanager.SlotRequest; +import org.apache.flink.runtime.resourcemanager.SlotRequestReply; +import org.apache.flink.runtime.rpc.TestingSerialRpcService; +import org.apache.flink.runtime.taskexecutor.SlotReport; +import org.apache.flink.runtime.taskexecutor.SlotStatus; +import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; +import org.apache.flink.util.TestLogger; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.mockito.Mockito; +import scala.concurrent.Await; +import scala.concurrent.Future; +import scala.concurrent.duration.Duration; +import scala.concurrent.duration.FiniteDuration; + +import java.util.Collections; +import java.util.UUID; +import java.util.concurrent.TimeUnit; + +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.timeout; +import static org.mockito.Mockito.verify; + +public class SlotProtocolTest extends TestLogger { + + private static TestingSerialRpcService testRpcService; + + @BeforeClass + public static void beforeClass() { + testRpcService = new TestingSerialRpcService(); + } + + @AfterClass + public static void afterClass() { + testRpcService.stopService(); + testRpcService = null; + } + + @Before + public void beforeTest(){ + testRpcService.clearGateways(); + } + + /** + * Tests whether + * 1) SlotRequest is routed to the SlotManager + * 2) SlotRequest is confirmed + * 3) SlotRequest leads to a container allocation + * 4) Slot becomes available and TaskExecutor gets a SlotRequest + */ + @Test + public void testSlotsUnavailableRequest() throws Exception { + final String rmAddress = "/rm1"; + final String jmAddress = "/jm1"; + final JobID jobID = new JobID(); + + testRpcService.registerGateway(jmAddress, mock(JobMasterGateway.class)); + + + TestingSlotManager slotManager = Mockito.spy(new TestingSlotManager()); + ResourceManager resourceManager = + new ResourceManager(testRpcService, new NonHaServices(rmAddress), slotManager); + resourceManager.start(); + + Future registrationFuture = + resourceManager.registerJobMaster(new JobMasterRegistration(jmAddress, jobID)); + try { + Await.ready(registrationFuture, Duration.create(5, TimeUnit.SECONDS)); + } catch (Exception e) { + Assert.fail("JobManager registration Future didn't become ready."); + } + + final AllocationID allocationID = new AllocationID(); + final ResourceProfile resourceProfile = new ResourceProfile(1.0, 100); + + SlotRequest slotRequest = new SlotRequest(jobID, allocationID, resourceProfile); + SlotRequestReply slotRequestReply = + resourceManager.requestSlot(slotRequest); + + // 1) SlotRequest is routed to the SlotManager + verify(slotManager).requestSlot(slotRequest); + + // 2) SlotRequest is confirmed + Assert.assertEquals( + slotRequestReply.getAllocationID(), + allocationID); + + // 3) SlotRequest leads to a container allocation + verify(slotManager, timeout(5000)).allocateContainer(resourceProfile); + + Assert.assertFalse(slotManager.isAllocated(allocationID)); + + // slot becomes available + final String tmAddress = "/tm1"; + TaskExecutorGateway taskExecutorGateway = mock(TaskExecutorGateway.class); + testRpcService.registerGateway(tmAddress, taskExecutorGateway); + + final ResourceID resourceID = ResourceID.generate(); + final SlotID slotID = new SlotID(resourceID, 0); + + final SlotStatus slotStatus = + new SlotStatus(slotID, resourceProfile); + final SlotReport slotReport = + new SlotReport(Collections.singletonList(slotStatus), resourceID); + // register slot at SlotManager + slotManager.registerTaskExecutor(resourceID, taskExecutorGateway); + slotManager.updateSlotStatus(slotReport); + + // 4) Slot becomes available and TaskExecutor gets a SlotRequest + verify(taskExecutorGateway, timeout(5000)).requestSlot(eq(allocationID), any(UUID.class), any(FiniteDuration.class)); + } + + /** + * Tests whether + * 1) a SlotRequest is routed to the SlotManager + * 2) a SlotRequest is confirmed + * 3) a SlotRequest leads to an allocation of a registered slot + * 4) a SlotRequest is routed to the TaskExecutor + */ + @Test + public void testSlotAvailableRequest() throws Exception { + final String rmAddress = "/rm1"; + final String jmAddress = "/jm1"; + final String tmAddress = "/tm1"; + final JobID jobID = new JobID(); + + testRpcService.registerGateway(jmAddress, mock(JobMasterGateway.class)); + + TaskExecutorGateway taskExecutorGateway = mock(TaskExecutorGateway.class); + testRpcService.registerGateway(tmAddress, taskExecutorGateway); + + TestingSlotManager slotManager = Mockito.spy(new TestingSlotManager()); + ResourceManager resourceManager = + new ResourceManager(testRpcService, new NonHaServices(rmAddress), slotManager); + resourceManager.start(); + + Future registrationFuture = + resourceManager.registerJobMaster(new JobMasterRegistration(jmAddress, jobID)); + try { + Await.ready(registrationFuture, Duration.create(5, TimeUnit.SECONDS)); + } catch (Exception e) { + Assert.fail("JobManager registration Future didn't become ready."); + } + + final ResourceID resourceID = ResourceID.generate(); + final AllocationID allocationID = new AllocationID(); + final ResourceProfile resourceProfile = new ResourceProfile(1.0, 100); + final SlotID slotID = new SlotID(resourceID, 0); + + final SlotStatus slotStatus = + new SlotStatus(slotID, resourceProfile); + final SlotReport slotReport = + new SlotReport(Collections.singletonList(slotStatus), resourceID); + // register slot at SlotManager + slotManager.registerTaskExecutor(resourceID, taskExecutorGateway); + slotManager.updateSlotStatus(slotReport); + + SlotRequest slotRequest = new SlotRequest(jobID, allocationID, resourceProfile); + SlotRequestReply slotRequestReply = + resourceManager.requestSlot(slotRequest); + + // 1) a SlotRequest is routed to the SlotManager + verify(slotManager).requestSlot(slotRequest); + + // 2) a SlotRequest is confirmed + Assert.assertEquals( + slotRequestReply.getAllocationID(), + allocationID); + + // 3) a SlotRequest leads to an allocation of a registered slot + Assert.assertTrue(slotManager.isAllocated(slotID)); + Assert.assertTrue(slotManager.isAllocated(allocationID)); + + + // 4) a SlotRequest is routed to the TaskExecutor + verify(taskExecutorGateway, timeout(5000)).requestSlot(eq(allocationID), any(UUID.class), any(FiniteDuration.class)); + } + + + private static class TestingSlotManager extends SimpleSlotManager { + + // change visibility of function to public for testing + @Override + public void allocateContainer(ResourceProfile resourceProfile) { + super.allocateContainer(resourceProfile); + } + + + } + +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java index 7e92e8df3d6be..2212680629290 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java @@ -112,4 +112,8 @@ public Future connect(String address, Class clazz) return Futures.failed(new Exception("No gateway registered under that name")); } } -} \ No newline at end of file + + public void clearGateways() { + registeredConnections.clear(); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java index 955edcc8660f3..01776edc5a549 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java @@ -137,6 +137,10 @@ public void registerGateway(String address, RpcGateway gateway) { } } + public void clearGateways() { + registeredConnections.clear(); + } + private static final class TestingSerialInvocationHandler> implements InvocationHandler, RpcGateway, MainThreadExecutor, StartStoppable { private final T rpcEndpoint; From 078dacee29210a7de016022a61ae3d5079b57a39 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 21 Sep 2016 11:47:53 +0200 Subject: [PATCH 28/50] [hotfix] Remove unused imports from SlotRequestRegistered/Rejected and ResourceSlot --- .../flink/runtime/clusterframework/types/ResourceSlot.java | 4 ---- .../flink/runtime/resourcemanager/SlotRequestRegistered.java | 4 ++-- .../flink/runtime/resourcemanager/SlotRequestRejected.java | 4 ++-- 3 files changed, 4 insertions(+), 8 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java index 5fb8aee8c9238..4a91a791ad055 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java @@ -20,8 +20,6 @@ import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; -import java.io.Serializable; - import static org.apache.flink.util.Preconditions.checkNotNull; /** @@ -30,8 +28,6 @@ */ public class ResourceSlot implements ResourceIDRetrievable { - private static final long serialVersionUID = -5853720153136840674L; - /** The unique identification of this slot */ private final SlotID slotId; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRegistered.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRegistered.java index 6b7f6dc20fdf6..f719dcec0f6c5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRegistered.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRegistered.java @@ -20,13 +20,13 @@ import org.apache.flink.runtime.clusterframework.types.AllocationID; -import java.io.Serializable; - /** * Acknowledgment by the ResourceManager for a SlotRequest from the JobManager */ public class SlotRequestRegistered extends SlotRequestReply { + private static final long serialVersionUID = 4760320859275256855L; + public SlotRequestRegistered(AllocationID allocationID) { super(allocationID); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRejected.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRejected.java index cb3ec72b6dff2..282a7d52c3dd5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRejected.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRejected.java @@ -20,13 +20,13 @@ import org.apache.flink.runtime.clusterframework.types.AllocationID; -import java.io.Serializable; - /** * Rejection message by the ResourceManager for a SlotRequest from the JobManager */ public class SlotRequestRejected extends SlotRequestReply { + private static final long serialVersionUID = 9049346740895325144L; + public SlotRequestRejected(AllocationID allocationID) { super(allocationID); } From 7a5a2b132d25bc4c05c3f51526f8bc1530428044 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 21 Sep 2016 14:14:05 +0200 Subject: [PATCH 29/50] [hotfix] Add methods defined in the gateway to the ResourceManager and TaskExecutor --- .../runtime/resourcemanager/ResourceManager.java | 1 + .../resourcemanager/ResourceManagerGateway.java | 2 +- .../flink/runtime/taskexecutor/TaskExecutor.java | 15 +++++++++++++++ .../runtime/taskexecutor/TaskExecutorGateway.java | 6 +++--- 4 files changed, 20 insertions(+), 4 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java index 29aba1ad3de03..d9a71349383dc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.resourcemanager; +import akka.dispatch.Futures; import akka.dispatch.Mapper; import org.apache.flink.annotation.VisibleForTesting; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java index e5c8b64060994..c8e348875dbb7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java @@ -58,7 +58,7 @@ Future registerJobMaster( * @param slotRequest Slot request * @return Future slot assignment */ - Future requestSlot(SlotRequest slotRequest); + Future requestSlot(SlotRequest slotRequest); /** * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java index a455fe2dec72d..fadae5f4c04ae 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java @@ -21,6 +21,7 @@ import akka.actor.ActorSystem; import akka.util.Timeout; import com.typesafe.config.Config; +import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.io.network.ConnectionManager; import org.apache.flink.runtime.io.network.LocalConnectionManager; import org.apache.flink.runtime.io.network.TaskEventDispatcher; @@ -30,6 +31,8 @@ import org.apache.flink.runtime.query.KvStateRegistry; import org.apache.flink.runtime.query.netty.DisabledKvStateRequestStats; import org.apache.flink.runtime.query.netty.KvStateServer; +import org.apache.flink.runtime.resourcemanager.SlotRequestRegistered; +import org.apache.flink.runtime.resourcemanager.SlotRequestReply; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.util.Preconditions; import org.slf4j.Logger; @@ -200,6 +203,18 @@ public void notifyOfNewResourceManagerLeader(String newLeaderAddress, UUID newLe } } + /** + * Requests a slot from the TaskManager + * + * @param allocationID id for the request + * @param resourceManagerLeaderID current leader id of the ResourceManager + * @return answer to the slot request + */ + @RpcMethod + public SlotRequestReply requestSlot(AllocationID allocationID, UUID resourceManagerLeaderID) { + return new SlotRequestRegistered(allocationID); + } + /** * Starts and runs the TaskManager. *

    diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java index 72574367ad78c..65323a8c62f12 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java @@ -39,12 +39,12 @@ public interface TaskExecutorGateway extends RpcGateway { void notifyOfNewResourceManagerLeader(String address, UUID resourceManagerLeaderId); /** - * Send by the ResourceManager to the TaskExecutor + * Requests a slot from the TaskManager + * * @param allocationID id for the request * @param resourceManagerLeaderID current leader id of the ResourceManager - * @return SlotRequestReply Answer to the request + * @return answer to the slot request */ - Future requestSlot( AllocationID allocationID, UUID resourceManagerLeaderID, From 103e44c1ed7e8d2d8b3ea764e892ba114fcfe655 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 21 Sep 2016 17:26:21 +0200 Subject: [PATCH 30/50] [FLINK-4656] [rpc] Port the existing code to Flink's own future abstraction This closes #2530. --- .../runtime/concurrent/impl/FlinkFuture.java | 4 + .../flink/runtime/jobmaster/JobMaster.java | 2 +- .../runtime/jobmaster/JobMasterGateway.java | 2 +- .../registration/RetryingRegistration.java | 65 ++++++++-------- .../resourcemanager/ResourceManager.java | 13 ++-- .../ResourceManagerGateway.java | 9 +-- .../slotmanager/SlotManager.java | 9 +-- ...xecutor.java => MainThreadExecutable.java} | 8 +- .../apache/flink/runtime/rpc/RpcEndpoint.java | 60 ++++++--------- .../apache/flink/runtime/rpc/RpcService.java | 17 ++--- .../rpc/akka/AkkaInvocationHandler.java | 42 +++++------ .../flink/runtime/rpc/akka/AkkaRpcActor.java | 21 +++++- .../runtime/rpc/akka/AkkaRpcService.java | 28 +++---- .../runtime/taskexecutor/TaskExecutor.java | 12 +-- .../taskexecutor/TaskExecutorGateway.java | 6 +- ...skExecutorToResourceManagerConnection.java | 34 ++++----- .../RetryingRegistrationTest.java | 75 ++++++++++--------- .../registration/TestRegistrationGateway.java | 6 +- .../ResourceManagerHATest.java | 4 +- .../slotmanager/SlotProtocolTest.java | 14 ++-- .../flink/runtime/rpc/AsyncCallsTest.java | 13 ++-- .../runtime/rpc/RpcCompletenessTest.java | 9 ++- .../flink/runtime/rpc/TestingGatewayBase.java | 18 ++--- .../flink/runtime/rpc/TestingRpcService.java | 20 ++--- .../runtime/rpc/TestingSerialRpcService.java | 54 +++++++------ .../runtime/rpc/akka/AkkaRpcActorTest.java | 19 ++--- .../runtime/rpc/akka/AkkaRpcServiceTest.java | 4 +- .../rpc/akka/MainThreadValidationTest.java | 7 +- .../rpc/akka/MessageSerializationTest.java | 19 ++--- .../taskexecutor/TaskExecutorTest.java | 9 +-- 30 files changed, 295 insertions(+), 308 deletions(-) rename flink-runtime/src/main/java/org/apache/flink/runtime/rpc/{MainThreadExecutor.java => MainThreadExecutable.java} (91%) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/impl/FlinkFuture.java b/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/impl/FlinkFuture.java index 3f2c5e42ec596..e9aacb8fdd211 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/impl/FlinkFuture.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/impl/FlinkFuture.java @@ -59,6 +59,10 @@ public FlinkFuture(scala.concurrent.Future scalaFuture) { this.scalaFuture = Preconditions.checkNotNull(scalaFuture); } + public scala.concurrent.Future getScalaFuture() { + return scalaFuture; + } + //----------------------------------------------------------------------------------- // Future's methods //----------------------------------------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java index 0a6a7ef0938ec..1537396f347f4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java @@ -36,7 +36,7 @@ /** * JobMaster implementation. The job master is responsible for the execution of a single - * {@link org.apache.flink.runtime.jobgraph.JobGraph}. + * {@link JobGraph}. *

    * It offers the following methods as part of its rpc interface to interact with the JobMaster * remotely: diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java index a53e383942744..86bf17c13250d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java @@ -18,10 +18,10 @@ package org.apache.flink.runtime.jobmaster; +import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.taskmanager.TaskExecutionState; -import scala.concurrent.Future; /** * {@link JobMaster} rpc gateway interface diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java index ea49e42d87826..32dd9781c8386 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java @@ -18,19 +18,17 @@ package org.apache.flink.runtime.registration; -import akka.dispatch.OnFailure; -import akka.dispatch.OnSuccess; - import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.concurrent.AcceptFunction; +import org.apache.flink.runtime.concurrent.ApplyFunction; +import org.apache.flink.runtime.concurrent.CompletableFuture; +import org.apache.flink.runtime.concurrent.Future; +import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcService; import org.slf4j.Logger; -import scala.concurrent.Future; -import scala.concurrent.Promise; -import scala.concurrent.impl.Promise.DefaultPromise; - import java.util.UUID; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -86,7 +84,7 @@ public abstract class RetryingRegistration> completionPromise; + private final CompletableFuture> completionFuture; private final long initialRegistrationTimeout; @@ -140,7 +138,7 @@ public RetryingRegistration( this.delayOnError = delayOnError; this.delayOnRefusedRegistration = delayOnRefusedRegistration; - this.completionPromise = new DefaultPromise<>(); + this.completionFuture = new FlinkCompletableFuture<>(); } // ------------------------------------------------------------------------ @@ -148,7 +146,7 @@ public RetryingRegistration( // ------------------------------------------------------------------------ public Future> getFuture() { - return completionPromise.future(); + return completionFuture; } /** @@ -184,28 +182,30 @@ public void startRegistration() { Future resourceManagerFuture = rpcService.connect(targetAddress, targetType); // upon success, start the registration attempts - resourceManagerFuture.onSuccess(new OnSuccess() { + resourceManagerFuture.thenAcceptAsync(new AcceptFunction() { @Override - public void onSuccess(Gateway result) { + public void accept(Gateway result) { log.info("Resolved {} address, beginning registration", targetName); register(result, 1, initialRegistrationTimeout); } - }, rpcService.getExecutionContext()); - + }, rpcService.getExecutor()); + // upon failure, retry, unless this is cancelled - resourceManagerFuture.onFailure(new OnFailure() { + resourceManagerFuture.exceptionallyAsync(new ApplyFunction() { @Override - public void onFailure(Throwable failure) { + public Void apply(Throwable failure) { if (!isCanceled()) { log.warn("Could not resolve {} address {}, retrying...", targetName, targetAddress, failure); startRegistration(); } + + return null; } - }, rpcService.getExecutionContext()); + }, rpcService.getExecutor()); } catch (Throwable t) { cancel(); - completionPromise.tryFailure(t); + completionFuture.completeExceptionally(t); } } @@ -225,15 +225,14 @@ private void register(final Gateway gateway, final int attempt, final long timeo Future registrationFuture = invokeRegistration(gateway, leaderId, timeoutMillis); // if the registration was successful, let the TaskExecutor know - registrationFuture.onSuccess(new OnSuccess() { - + registrationFuture.thenAcceptAsync(new AcceptFunction() { @Override - public void onSuccess(RegistrationResponse result) throws Throwable { + public void accept(RegistrationResponse result) { if (!isCanceled()) { if (result instanceof RegistrationResponse.Success) { // registration successful! Success success = (Success) result; - completionPromise.success(new Tuple2<>(gateway, success)); + completionFuture.complete(Tuple2.of(gateway, success)); } else { // registration refused or unknown @@ -241,7 +240,7 @@ public void onSuccess(RegistrationResponse result) throws Throwable { RegistrationResponse.Decline decline = (RegistrationResponse.Decline) result; log.info("Registration at {} was declined: {}", targetName, decline.getReason()); } else { - log.error("Received unknown response to registration attempt: " + result); + log.error("Received unknown response to registration attempt: {}", result); } log.info("Pausing and re-attempting registration in {} ms", delayOnRefusedRegistration); @@ -249,12 +248,12 @@ public void onSuccess(RegistrationResponse result) throws Throwable { } } } - }, rpcService.getExecutionContext()); + }, rpcService.getExecutor()); // upon failure, retry - registrationFuture.onFailure(new OnFailure() { + registrationFuture.exceptionallyAsync(new ApplyFunction() { @Override - public void onFailure(Throwable failure) { + public Void apply(Throwable failure) { if (!isCanceled()) { if (failure instanceof TimeoutException) { // we simply have not received a response in time. maybe the timeout was @@ -262,26 +261,28 @@ public void onFailure(Throwable failure) { // currently down. if (log.isDebugEnabled()) { log.debug("Registration at {} ({}) attempt {} timed out after {} ms", - targetName, targetAddress, attempt, timeoutMillis); + targetName, targetAddress, attempt, timeoutMillis); } - + long newTimeoutMillis = Math.min(2 * timeoutMillis, maxRegistrationTimeout); register(gateway, attempt + 1, newTimeoutMillis); } else { // a serious failure occurred. we still should not give up, but keep trying - log.error("Registration at " + targetName + " failed due to an error", failure); + log.error("Registration at {} failed due to an error", targetName, failure); log.info("Pausing and re-attempting registration in {} ms", delayOnError); - + registerLater(gateway, 1, initialRegistrationTimeout, delayOnError); } } + + return null; } - }, rpcService.getExecutionContext()); + }, rpcService.getExecutor()); } catch (Throwable t) { cancel(); - completionPromise.tryFailure(t); + completionFuture.completeExceptionally(t); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java index d9a71349383dc..537071085aedb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java @@ -18,12 +18,11 @@ package org.apache.flink.runtime.resourcemanager; -import akka.dispatch.Futures; -import akka.dispatch.Mapper; - import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.concurrent.ApplyFunction; +import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.leaderelection.LeaderContender; @@ -38,7 +37,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import scala.concurrent.Future; import java.util.HashMap; import java.util.Map; @@ -126,10 +124,9 @@ public Future registerJobMaster(JobMasterRegistration jobM getRpcService().connect(jobMasterRegistration.getAddress(), JobMasterGateway.class); final JobID jobID = jobMasterRegistration.getJobID(); - return jobMasterFuture.map(new Mapper() { + return jobMasterFuture.thenApplyAsync(new ApplyFunction() { @Override - public RegistrationResponse apply(final JobMasterGateway jobMasterGateway) { - + public RegistrationResponse apply(JobMasterGateway jobMasterGateway) { final JobMasterGateway existingGateway = jobMasterGateways.put(jobID, jobMasterGateway); if (existingGateway != null) { LOG.info("Replacing existing gateway {} for JobID {} with {}.", @@ -137,7 +134,7 @@ public RegistrationResponse apply(final JobMasterGateway jobMasterGateway) { } return new RegistrationResponse(true); } - }, getMainThreadExecutionContext()); + }, getMainThreadExecutor()); } /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java index c8e348875dbb7..5c8786cf0e995 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java @@ -18,14 +18,13 @@ package org.apache.flink.runtime.resourcemanager; +import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcTimeout; import org.apache.flink.runtime.jobmaster.JobMaster; -import scala.concurrent.Future; -import scala.concurrent.duration.FiniteDuration; - import java.util.UUID; /** @@ -42,7 +41,7 @@ public interface ResourceManagerGateway extends RpcGateway { */ Future registerJobMaster( JobMasterRegistration jobMasterRegistration, - @RpcTimeout FiniteDuration timeout); + @RpcTimeout Time timeout); /** * Register a {@link JobMaster} at the resource manager. @@ -73,5 +72,5 @@ Future registerTaskE UUID resourceManagerLeaderId, String taskExecutorAddress, ResourceID resourceID, - @RpcTimeout FiniteDuration timeout); + @RpcTimeout Time timeout); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java index 96fde7d1fe7c2..97176b25b32ed 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java @@ -19,11 +19,13 @@ package org.apache.flink.runtime.resourcemanager.slotmanager; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.clusterframework.types.ResourceSlot; import org.apache.flink.runtime.clusterframework.types.SlotID; +import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener; import org.apache.flink.runtime.resourcemanager.SlotRequest; import org.apache.flink.runtime.resourcemanager.SlotRequestRegistered; @@ -33,14 +35,11 @@ import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import scala.concurrent.Future; -import scala.concurrent.duration.FiniteDuration; import java.util.HashMap; import java.util.LinkedHashMap; import java.util.Map; import java.util.UUID; -import java.util.concurrent.TimeUnit; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -79,7 +78,7 @@ public abstract class SlotManager implements LeaderRetrievalListener { /** All allocations, we can lookup allocations either by SlotID or AllocationID */ private final AllocationMap allocationMap; - private final FiniteDuration timeout; + private final Time timeout; /** The current leader id set by the ResourceManager */ private UUID leaderID; @@ -90,7 +89,7 @@ public SlotManager() { this.freeSlots = new HashMap<>(16); this.allocationMap = new AllocationMap(); this.taskManagerGateways = new HashMap<>(); - this.timeout = new FiniteDuration(10, TimeUnit.SECONDS); + this.timeout = Time.seconds(10); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutable.java similarity index 91% rename from flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutable.java index 5e4fead8c5531..ec1c9842f5fc2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutable.java @@ -18,8 +18,8 @@ package org.apache.flink.runtime.rpc; -import akka.util.Timeout; -import scala.concurrent.Future; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.concurrent.Future; import java.util.concurrent.Callable; import java.util.concurrent.TimeoutException; @@ -32,7 +32,7 @@ * implementation which allows to dispatch local procedures to the main thread of the underlying * RPC endpoint. */ -public interface MainThreadExecutor { +public interface MainThreadExecutable { /** * Execute the runnable in the main thread of the underlying RPC endpoint. @@ -51,7 +51,7 @@ public interface MainThreadExecutor { * @param Return value of the callable * @return Future of the callable result */ - Future callAsync(Callable callable, Timeout callTimeout); + Future callAsync(Callable callable, Time callTimeout); /** * Execute the runnable in the main thread of the underlying RPC endpoint, with diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java index e9e2b2c40d3de..4e5e49a527708 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java @@ -18,16 +18,15 @@ package org.apache.flink.runtime.rpc; -import akka.util.Timeout; - +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.concurrent.Future; +import org.apache.flink.util.Preconditions; import org.apache.flink.util.ReflectionUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import scala.concurrent.ExecutionContext; -import scala.concurrent.Future; - import java.util.concurrent.Callable; +import java.util.concurrent.Executor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; @@ -49,8 +48,8 @@ * thread, we don't have to reason about concurrent accesses, in the same way in the Actor Model * of Erlang or Akka. * - *

    The RPC endpoint provides provides {@link #runAsync(Runnable)}, {@link #callAsync(Callable, Timeout)} - * and the {@link #getMainThreadExecutionContext()} to execute code in the RPC endoint's main thread. + *

    The RPC endpoint provides provides {@link #runAsync(Runnable)}, {@link #callAsync(Callable, Time)} + * and the {@link #getMainThreadExecutor()} to execute code in the RPC endoint's main thread. * * @param The RPC gateway counterpart for the implementing RPC endpoint */ @@ -69,9 +68,9 @@ public abstract class RpcEndpoint { /** Self gateway which can be used to schedule asynchronous calls on yourself */ private final C self; - /** The main thread execution context to be used to execute future callbacks in the main thread + /** The main thread executor to be used to execute future callbacks in the main thread * of the executing rpc server. */ - private final ExecutionContext mainThreadExecutionContext; + private final Executor mainThreadExecutor; /** A reference to the endpoint's main thread, if the current method is called by the main thread */ final AtomicReference currentMainThread = new AtomicReference<>(null); @@ -89,7 +88,7 @@ protected RpcEndpoint(final RpcService rpcService) { this.selfGatewayType = ReflectionUtil.getTemplateType1(getClass()); this.self = rpcService.startServer(this); - this.mainThreadExecutionContext = new MainThreadExecutionContext((MainThreadExecutor) self); + this.mainThreadExecutor = new MainThreadExecutor((MainThreadExecutable) self); } /** @@ -120,7 +119,7 @@ public void start() { * Shuts down the underlying RPC endpoint via the RPC service. * After this method was called, the RPC endpoint will no longer be reachable, neither remotely, * not via its {@link #getSelf() self gateway}. It will also not accepts executions in main thread - * any more (via {@link #callAsync(Callable, Timeout)} and {@link #runAsync(Runnable)}). + * any more (via {@link #callAsync(Callable, Time)} and {@link #runAsync(Runnable)}). * *

    This method can be overridden to add RPC endpoint specific shut down code. * The overridden method should always call the parent shut down method. @@ -161,8 +160,8 @@ public String getAddress() { * * @return Main thread execution context */ - protected ExecutionContext getMainThreadExecutionContext() { - return mainThreadExecutionContext; + protected Executor getMainThreadExecutor() { + return mainThreadExecutor; } /** @@ -185,7 +184,7 @@ public RpcService getRpcService() { * @param runnable Runnable to be executed in the main thread of the underlying RPC endpoint */ protected void runAsync(Runnable runnable) { - ((MainThreadExecutor) self).runAsync(runnable); + ((MainThreadExecutable) self).runAsync(runnable); } /** @@ -196,7 +195,7 @@ protected void runAsync(Runnable runnable) { * @param delay The delay after which the runnable will be executed */ protected void scheduleRunAsync(Runnable runnable, long delay, TimeUnit unit) { - ((MainThreadExecutor) self).scheduleRunAsync(runnable, unit.toMillis(delay)); + ((MainThreadExecutable) self).scheduleRunAsync(runnable, unit.toMillis(delay)); } /** @@ -209,8 +208,8 @@ protected void scheduleRunAsync(Runnable runnable, long delay, TimeUnit unit) { * @param Return type of the callable * @return Future for the result of the callable. */ - protected Future callAsync(Callable callable, Timeout timeout) { - return ((MainThreadExecutor) self).callAsync(callable, timeout); + protected Future callAsync(Callable callable, Time timeout) { + return ((MainThreadExecutable) self).callAsync(callable, timeout); } // ------------------------------------------------------------------------ @@ -241,36 +240,19 @@ public void validateRunsInMainThread() { // ------------------------------------------------------------------------ /** - * Execution context which executes runnables in the main thread context. A reported failure - * will cause the underlying rpc server to shut down. + * Executor which executes runnables in the main thread context. */ - private class MainThreadExecutionContext implements ExecutionContext { + private class MainThreadExecutor implements Executor { - private final MainThreadExecutor gateway; + private final MainThreadExecutable gateway; - MainThreadExecutionContext(MainThreadExecutor gateway) { - this.gateway = gateway; + MainThreadExecutor(MainThreadExecutable gateway) { + this.gateway = Preconditions.checkNotNull(gateway); } @Override public void execute(Runnable runnable) { gateway.runAsync(runnable); } - - @Override - public void reportFailure(final Throwable t) { - gateway.runAsync(new Runnable() { - @Override - public void run() { - log.error("Encountered failure in the main thread execution context.", t); - shutDown(); - } - }); - } - - @Override - public ExecutionContext prepare() { - return this; - } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java index 78c1cec33bc9a..a367ff2fdad9a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java @@ -18,10 +18,10 @@ package org.apache.flink.runtime.rpc; +import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException; -import scala.concurrent.ExecutionContext; -import scala.concurrent.Future; +import java.util.concurrent.Executor; import java.util.concurrent.TimeUnit; /** @@ -68,23 +68,22 @@ public interface RpcService { void stopService(); /** - * Gets the execution context, provided by this RPC service. This execution - * context can be used for example for the {@code onComplete(...)} or {@code onSuccess(...)} - * methods of Futures. + * Gets the executor, provided by this RPC service. This executor can be used for example for + * the {@code handleAsync(...)} or {@code thenAcceptAsync(...)} methods of futures. * - *

    IMPORTANT: This execution context does not isolate the method invocations against + *

    IMPORTANT: This executor does not isolate the method invocations against * any concurrent invocations and is therefore not suitable to run completion methods of futures * that modify state of an {@link RpcEndpoint}. For such operations, one needs to use the - * {@link RpcEndpoint#getMainThreadExecutionContext() MainThreadExecutionContext} of that + * {@link RpcEndpoint#getMainThreadExecutor() MainThreadExecutionContext} of that * {@code RpcEndpoint}. * * @return The execution context provided by the RPC service */ - ExecutionContext getExecutionContext(); + Executor getExecutor(); /** * Execute the runnable in the execution context of this RPC Service, as returned by - * {@link #getExecutionContext()}, after a scheduled delay. + * {@link #getExecutor()}, after a scheduled delay. * * @param runnable Runnable to be executed * @param delay The delay after which the runnable will be executed diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java index bfa04f6bedeb4..8f4deff8f144f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java @@ -20,9 +20,11 @@ import akka.actor.ActorRef; import akka.pattern.Patterns; -import akka.util.Timeout; +import org.apache.flink.api.common.time.Time; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.runtime.rpc.MainThreadExecutor; +import org.apache.flink.runtime.concurrent.Future; +import org.apache.flink.runtime.concurrent.impl.FlinkFuture; +import org.apache.flink.runtime.rpc.MainThreadExecutable; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcTimeout; import org.apache.flink.runtime.rpc.StartStoppable; @@ -34,9 +36,6 @@ import org.apache.flink.runtime.rpc.akka.messages.RunAsync; import org.apache.flink.util.Preconditions; import org.apache.log4j.Logger; -import scala.concurrent.Await; -import scala.concurrent.Future; -import scala.concurrent.duration.FiniteDuration; import java.io.IOException; import java.lang.annotation.Annotation; @@ -53,7 +52,7 @@ * rpc in a {@link LocalRpcInvocation} message and then sends it to the {@link AkkaRpcActor} where it is * executed. */ -class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThreadExecutor, StartStoppable { +class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThreadExecutable, StartStoppable { private static final Logger LOG = Logger.getLogger(AkkaInvocationHandler.class); private final String address; @@ -64,11 +63,11 @@ class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThrea private final boolean isLocal; // default timeout for asks - private final Timeout timeout; + private final Time timeout; private final long maximumFramesize; - AkkaInvocationHandler(String address, ActorRef rpcEndpoint, Timeout timeout, long maximumFramesize) { + AkkaInvocationHandler(String address, ActorRef rpcEndpoint, Time timeout, long maximumFramesize) { this.address = Preconditions.checkNotNull(address); this.rpcEndpoint = Preconditions.checkNotNull(rpcEndpoint); this.isLocal = this.rpcEndpoint.path().address().hasLocalScope(); @@ -82,7 +81,7 @@ public Object invoke(Object proxy, Method method, Object[] args) throws Throwabl Object result; - if (declaringClass.equals(AkkaGateway.class) || declaringClass.equals(MainThreadExecutor.class) || + if (declaringClass.equals(AkkaGateway.class) || declaringClass.equals(MainThreadExecutable.class) || declaringClass.equals(Object.class) || declaringClass.equals(StartStoppable.class) || declaringClass.equals(RpcGateway.class)) { result = method.invoke(this, args); @@ -90,7 +89,7 @@ public Object invoke(Object proxy, Method method, Object[] args) throws Throwabl String methodName = method.getName(); Class[] parameterTypes = method.getParameterTypes(); Annotation[][] parameterAnnotations = method.getParameterAnnotations(); - Timeout futureTimeout = extractRpcTimeout(parameterAnnotations, args, timeout); + Time futureTimeout = extractRpcTimeout(parameterAnnotations, args, timeout); Tuple2[], Object[]> filteredArguments = filterArguments( parameterTypes, @@ -130,13 +129,14 @@ public Object invoke(Object proxy, Method method, Object[] args) throws Throwabl result = null; } else if (returnType.equals(Future.class)) { // execute an asynchronous call - result = Patterns.ask(rpcEndpoint, rpcInvocation, futureTimeout); + result = new FlinkFuture<>(Patterns.ask(rpcEndpoint, rpcInvocation, futureTimeout.toMilliseconds())); } else { // execute a synchronous call - Future futureResult = Patterns.ask(rpcEndpoint, rpcInvocation, futureTimeout); - FiniteDuration duration = timeout.duration(); + scala.concurrent.Future scalaFuture = Patterns.ask(rpcEndpoint, rpcInvocation, futureTimeout.toMilliseconds()); - result = Await.result(futureResult, duration); + Future futureResult = new FlinkFuture<>(scalaFuture); + + return futureResult.get(futureTimeout.getSize(), futureTimeout.getUnit()); } } @@ -167,12 +167,12 @@ public void scheduleRunAsync(Runnable runnable, long delay) { } @Override - public Future callAsync(Callable callable, Timeout callTimeout) { + public Future callAsync(Callable callable, Time callTimeout) { if(isLocal) { @SuppressWarnings("unchecked") - Future result = (Future) Patterns.ask(rpcEndpoint, new CallAsync(callable), callTimeout); + scala.concurrent.Future result = (scala.concurrent.Future) Patterns.ask(rpcEndpoint, new CallAsync(callable), callTimeout.toMilliseconds()); - return result; + return new FlinkFuture<>(result); } else { throw new RuntimeException("Trying to send a Callable to a remote actor at " + rpcEndpoint.path() + ". This is not supported."); @@ -204,17 +204,17 @@ public void stop() { * has been found * @return Timeout extracted from the array of arguments or the default timeout */ - private static Timeout extractRpcTimeout(Annotation[][] parameterAnnotations, Object[] args, Timeout defaultTimeout) { + private static Time extractRpcTimeout(Annotation[][] parameterAnnotations, Object[] args, Time defaultTimeout) { if (args != null) { Preconditions.checkArgument(parameterAnnotations.length == args.length); for (int i = 0; i < parameterAnnotations.length; i++) { if (isRpcTimeout(parameterAnnotations[i])) { - if (args[i] instanceof FiniteDuration) { - return new Timeout((FiniteDuration) args[i]); + if (args[i] instanceof Time) { + return (Time) args[i]; } else { throw new RuntimeException("The rpc timeout parameter must be of type " + - FiniteDuration.class.getName() + ". The type " + args[i].getClass().getName() + + Time.class.getName() + ". The type " + args[i].getClass().getName() + " is not supported."); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java index 2373be9414ed2..59daa46e9f6f6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java @@ -21,8 +21,11 @@ import akka.actor.ActorRef; import akka.actor.Status; import akka.actor.UntypedActorWithStash; +import akka.dispatch.Futures; import akka.japi.Procedure; import akka.pattern.Patterns; +import org.apache.flink.runtime.concurrent.Future; +import org.apache.flink.runtime.concurrent.impl.FlinkFuture; import org.apache.flink.runtime.rpc.MainThreadValidatorUtil; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcGateway; @@ -35,7 +38,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import scala.concurrent.Future; import scala.concurrent.duration.FiniteDuration; import java.io.IOException; @@ -146,8 +148,23 @@ private void handleRpcInvocation(RpcInvocation rpcInvocation) { Object result = rpcMethod.invoke(rpcEndpoint, rpcInvocation.getArgs()); if (result instanceof Future) { + final Future future = (Future) result; + // pipe result to sender - Patterns.pipe((Future) result, getContext().dispatcher()).to(getSender()); + if (future instanceof FlinkFuture) { + // FlinkFutures are currently backed by Scala's futures + FlinkFuture flinkFuture = (FlinkFuture) future; + + Patterns.pipe(flinkFuture.getScalaFuture(), getContext().dispatcher()).to(getSender()); + } else { + // We have to unpack the Flink future and pack it into a Scala future + Patterns.pipe(Futures.future(new Callable() { + @Override + public Object call() throws Exception { + return future.get(); + } + }, getContext().dispatcher()), getContext().dispatcher()); + } } else { // tell the sender the result of the computation getSender().tell(new Status.Success(result), getSelf()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java index 060a1ef499cb0..36f11152f26fd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java @@ -26,11 +26,13 @@ import akka.actor.PoisonPill; import akka.actor.Props; import akka.dispatch.Mapper; -import akka.pattern.AskableActorSelection; -import akka.util.Timeout; +import akka.pattern.Patterns; +import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.akka.AkkaUtils; -import org.apache.flink.runtime.rpc.MainThreadExecutor; +import org.apache.flink.runtime.concurrent.Future; +import org.apache.flink.runtime.concurrent.impl.FlinkFuture; +import org.apache.flink.runtime.rpc.MainThreadExecutable; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcService; @@ -39,8 +41,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import scala.concurrent.ExecutionContext; -import scala.concurrent.Future; import scala.concurrent.duration.FiniteDuration; import javax.annotation.concurrent.ThreadSafe; @@ -48,6 +48,7 @@ import java.lang.reflect.Proxy; import java.util.HashSet; import java.util.Set; +import java.util.concurrent.Executor; import java.util.concurrent.TimeUnit; import static org.apache.flink.util.Preconditions.checkArgument; @@ -68,13 +69,13 @@ public class AkkaRpcService implements RpcService { private final Object lock = new Object(); private final ActorSystem actorSystem; - private final Timeout timeout; + private final Time timeout; private final Set actors = new HashSet<>(4); private final long maximumFramesize; private volatile boolean stopped; - public AkkaRpcService(final ActorSystem actorSystem, final Timeout timeout) { + public AkkaRpcService(final ActorSystem actorSystem, final Time timeout) { this.actorSystem = checkNotNull(actorSystem, "actor system"); this.timeout = checkNotNull(timeout, "timeout"); @@ -95,10 +96,9 @@ public Future connect(final String address, final Clas address, clazz.getName()); final ActorSelection actorSel = actorSystem.actorSelection(address); - final AskableActorSelection asker = new AskableActorSelection(actorSel); - final Future identify = asker.ask(new Identify(42), timeout); - return identify.map(new Mapper(){ + final scala.concurrent.Future identify = Patterns.ask(actorSel, new Identify(42), timeout.toMilliseconds()); + final scala.concurrent.Future resultFuture = identify.map(new Mapper(){ @Override public C checkedApply(Object obj) throws Exception { @@ -128,6 +128,8 @@ public C checkedApply(Object obj) throws Exception { } } }, actorSystem.dispatcher()); + + return new FlinkFuture<>(resultFuture); } @Override @@ -159,7 +161,7 @@ public > C startServer(S rpcEndpo classLoader, new Class[]{ rpcEndpoint.getSelfGatewayType(), - MainThreadExecutor.class, + MainThreadExecutable.class, StartStoppable.class, AkkaGateway.class}, akkaInvocationHandler); @@ -209,7 +211,7 @@ public void stopService() { } @Override - public ExecutionContext getExecutionContext() { + public Executor getExecutor() { return actorSystem.dispatcher(); } @@ -219,6 +221,6 @@ public void scheduleRunnable(Runnable runnable, long delay, TimeUnit unit) { checkNotNull(unit, "unit"); checkArgument(delay >= 0, "delay must be zero or larger"); - actorSystem.scheduler().scheduleOnce(new FiniteDuration(delay, unit), runnable, getExecutionContext()); + actorSystem.scheduler().scheduleOnce(new FiniteDuration(delay, unit), runnable, actorSystem.dispatcher()); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java index fadae5f4c04ae..d84a6a91c8d23 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java @@ -19,8 +19,8 @@ package org.apache.flink.runtime.taskexecutor; import akka.actor.ActorSystem; -import akka.util.Timeout; import com.typesafe.config.Config; +import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.io.network.ConnectionManager; import org.apache.flink.runtime.io.network.LocalConnectionManager; @@ -35,6 +35,7 @@ import org.apache.flink.runtime.resourcemanager.SlotRequestReply; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.util.Preconditions; +import org.jboss.netty.channel.ChannelException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -78,6 +79,7 @@ import java.io.File; import java.io.IOException; +import java.net.BindException; import java.net.InetAddress; import java.net.InetSocketAddress; import java.util.UUID; @@ -198,7 +200,7 @@ public void notifyOfNewResourceManagerLeader(String newLeaderAddress, UUID newLe this, newLeaderAddress, newLeaderId, - getMainThreadExecutionContext()); + getMainThreadExecutor()); resourceManagerConnection.start(); } } @@ -302,9 +304,9 @@ private static void runTaskManager( LOG.debug("Using akka configuration\n " + akkaConfig); taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig); } catch (Throwable t) { - if (t instanceof org.jboss.netty.channel.ChannelException) { + if (t instanceof ChannelException) { Throwable cause = t.getCause(); - if (cause != null && t.getCause() instanceof java.net.BindException) { + if (cause != null && t.getCause() instanceof BindException) { String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort); throw new IOException("Unable to bind TaskManager actor system to address " + address + " - " + cause.getMessage(), t); @@ -314,7 +316,7 @@ private static void runTaskManager( } // start akka rpc service based on actor system - final Timeout timeout = new Timeout(AkkaUtils.getTimeout(configuration).toMillis(), TimeUnit.MILLISECONDS); + final Time timeout = Time.milliseconds(AkkaUtils.getTimeout(configuration).toMillis()); final AkkaRpcService akkaRpcService = new AkkaRpcService(taskManagerSystem, timeout); // start high availability service to implement getResourceManagerLeaderRetriever method only diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java index 65323a8c62f12..09628021175e8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java @@ -18,12 +18,12 @@ package org.apache.flink.runtime.taskexecutor; +import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.resourcemanager.SlotRequestReply; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcTimeout; -import scala.concurrent.Future; -import scala.concurrent.duration.FiniteDuration; import java.util.UUID; @@ -48,5 +48,5 @@ public interface TaskExecutorGateway extends RpcGateway { Future requestSlot( AllocationID allocationID, UUID resourceManagerLeaderID, - @RpcTimeout FiniteDuration timeout); + @RpcTimeout Time timeout); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java index 28062b6e2496a..647359d54cfba 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java @@ -18,11 +18,12 @@ package org.apache.flink.runtime.taskexecutor; -import akka.dispatch.OnFailure; -import akka.dispatch.OnSuccess; - +import org.apache.flink.api.common.time.Time; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.concurrent.AcceptFunction; +import org.apache.flink.runtime.concurrent.ApplyFunction; +import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.registration.RegistrationResponse; @@ -31,12 +32,8 @@ import org.slf4j.Logger; -import scala.concurrent.ExecutionContext; -import scala.concurrent.Future; -import scala.concurrent.duration.FiniteDuration; - import java.util.UUID; -import java.util.concurrent.TimeUnit; +import java.util.concurrent.Executor; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.apache.flink.util.Preconditions.checkState; @@ -57,7 +54,7 @@ public class TaskExecutorToResourceManagerConnection { private final String resourceManagerAddress; /** Execution context to be used to execute the on complete action of the ResourceManagerRegistration */ - private final ExecutionContext executionContext; + private final Executor executor; private TaskExecutorToResourceManagerConnection.ResourceManagerRegistration pendingRegistration; @@ -74,13 +71,13 @@ public TaskExecutorToResourceManagerConnection( TaskExecutor taskExecutor, String resourceManagerAddress, UUID resourceManagerLeaderId, - ExecutionContext executionContext) { + Executor executor) { this.log = checkNotNull(log); this.taskExecutor = checkNotNull(taskExecutor); this.resourceManagerAddress = checkNotNull(resourceManagerAddress); this.resourceManagerLeaderId = checkNotNull(resourceManagerLeaderId); - this.executionContext = checkNotNull(executionContext); + this.executor = checkNotNull(executor); } // ------------------------------------------------------------------------ @@ -100,21 +97,22 @@ public void start() { Future> future = pendingRegistration.getFuture(); - future.onSuccess(new OnSuccess>() { + future.thenAcceptAsync(new AcceptFunction>() { @Override - public void onSuccess(Tuple2 result) { + public void accept(Tuple2 result) { registrationId = result.f1.getRegistrationId(); registeredResourceManager = result.f0; } - }, executionContext); + }, executor); // this future should only ever fail if there is a bug, not if the registration is declined - future.onFailure(new OnFailure() { + future.exceptionallyAsync(new ApplyFunction() { @Override - public void onFailure(Throwable failure) { + public Void apply(Throwable failure) { taskExecutor.onFatalErrorAsync(failure); + return null; } - }, executionContext); + }, executor); } public void close() { @@ -197,7 +195,7 @@ private static class ResourceManagerRegistration protected Future invokeRegistration( ResourceManagerGateway resourceManager, UUID leaderId, long timeoutMillis) throws Exception { - FiniteDuration timeout = new FiniteDuration(timeoutMillis, TimeUnit.MILLISECONDS); + Time timeout = Time.milliseconds(timeoutMillis); return resourceManager.registerTaskExecutor(leaderId, taskExecutorAddress, resourceID, timeout); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java index 80fa19cc67bc7..e56a9ec5432e2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java @@ -18,9 +18,9 @@ package org.apache.flink.runtime.registration; -import akka.dispatch.Futures; - import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.concurrent.Future; +import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.util.TestLogger; @@ -29,18 +29,13 @@ import org.slf4j.LoggerFactory; -import scala.concurrent.Await; -import scala.concurrent.ExecutionContext$; -import scala.concurrent.Future; -import scala.concurrent.Promise; -import scala.concurrent.duration.FiniteDuration; - import java.util.UUID; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import static java.util.concurrent.TimeUnit.SECONDS; import static org.junit.Assert.*; import static org.mockito.Mockito.*; @@ -71,8 +66,8 @@ public void testSimpleSuccessfulRegistration() throws Exception { // multiple accesses return the same future assertEquals(future, registration.getFuture()); - Tuple2 success = - Await.result(future, new FiniteDuration(10, SECONDS)); + Tuple2 success = + future.get(10L, TimeUnit.SECONDS); // validate correct invocation and result assertEquals(testId, success.f1.getCorrelationId()); @@ -83,7 +78,7 @@ public void testSimpleSuccessfulRegistration() throws Exception { rpc.stopService(); } } - + @Test public void testPropagateFailures() throws Exception { final String testExceptionMessage = "testExceptionMessage"; @@ -96,9 +91,15 @@ public void testPropagateFailures() throws Exception { registration.startRegistration(); Future future = registration.getFuture(); - assertTrue(future.failed().isCompleted()); + assertTrue(future.isDone()); - assertEquals(testExceptionMessage, future.failed().value().get().get().getMessage()); + try { + future.get(); + + fail("We expected an ExecutionException."); + } catch (ExecutionException e) { + assertEquals(testExceptionMessage, e.getCause().getMessage()); + } } @Test @@ -113,16 +114,16 @@ public void testRetryConnectOnFailure() throws Exception { // RPC service that fails upon the first connection, but succeeds on the second RpcService rpc = mock(RpcService.class); when(rpc.connect(anyString(), any(Class.class))).thenReturn( - Futures.failed(new Exception("test connect failure")), // first connection attempt fails - Futures.successful(testGateway) // second connection attempt succeeds + FlinkCompletableFuture.completedExceptionally(new Exception("test connect failure")), // first connection attempt fails + FlinkCompletableFuture.completed(testGateway) // second connection attempt succeeds ); - when(rpc.getExecutionContext()).thenReturn(ExecutionContext$.MODULE$.fromExecutor(executor)); + when(rpc.getExecutor()).thenReturn(executor); TestRetryingRegistration registration = new TestRetryingRegistration(rpc, "foobar address", leaderId); registration.startRegistration(); Tuple2 success = - Await.result(registration.getFuture(), new FiniteDuration(10, SECONDS)); + registration.getFuture().get(10L, TimeUnit.SECONDS); // validate correct invocation and result assertEquals(testId, success.f1.getCorrelationId()); @@ -151,23 +152,23 @@ public void testRetriesOnTimeouts() throws Exception { try { rpc.registerGateway(testEndpointAddress, testGateway); - + TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId); - + long started = System.nanoTime(); registration.startRegistration(); - + Future> future = registration.getFuture(); Tuple2 success = - Await.result(future, new FiniteDuration(10, SECONDS)); - + future.get(10L, TimeUnit.SECONDS); + long finished = System.nanoTime(); long elapsedMillis = (finished - started) / 1000000; - + // validate correct invocation and result assertEquals(testId, success.f1.getCorrelationId()); assertEquals(leaderId, testGateway.getInvocations().take().leaderId()); - + // validate that some retry-delay / back-off behavior happened assertTrue("retries did not properly back off", elapsedMillis >= 3 * TestRetryingRegistration.INITIAL_TIMEOUT); } @@ -199,10 +200,10 @@ public void testDecline() throws Exception { long started = System.nanoTime(); registration.startRegistration(); - + Future> future = registration.getFuture(); Tuple2 success = - Await.result(future, new FiniteDuration(10, SECONDS)); + future.get(10L, TimeUnit.SECONDS); long finished = System.nanoTime(); long elapsedMillis = (finished - started) / 1000000; @@ -212,7 +213,7 @@ public void testDecline() throws Exception { assertEquals(leaderId, testGateway.getInvocations().take().leaderId()); // validate that some retry-delay / back-off behavior happened - assertTrue("retries did not properly back off", elapsedMillis >= + assertTrue("retries did not properly back off", elapsedMillis >= 2 * TestRetryingRegistration.INITIAL_TIMEOUT + TestRetryingRegistration.DELAY_ON_DECLINE); } finally { @@ -220,7 +221,7 @@ public void testDecline() throws Exception { rpc.stopService(); } } - + @Test @SuppressWarnings("unchecked") public void testRetryOnError() throws Exception { @@ -235,9 +236,9 @@ public void testRetryOnError() throws Exception { TestRegistrationGateway testGateway = mock(TestRegistrationGateway.class); when(testGateway.registrationCall(any(UUID.class), anyLong())).thenReturn( - Futures.failed(new Exception("test exception")), - Futures.successful(new TestRegistrationSuccess(testId))); - + FlinkCompletableFuture.completedExceptionally(new Exception("test exception")), + FlinkCompletableFuture.completed(new TestRegistrationSuccess(testId))); + rpc.registerGateway(testEndpointAddress, testGateway); TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId); @@ -247,11 +248,11 @@ public void testRetryOnError() throws Exception { Future> future = registration.getFuture(); Tuple2 success = - Await.result(future, new FiniteDuration(10, SECONDS)); + future.get(10, TimeUnit.SECONDS); long finished = System.nanoTime(); long elapsedMillis = (finished - started) / 1000000; - + assertEquals(testId, success.f1.getCorrelationId()); // validate that some retry-delay / back-off behavior happened @@ -271,10 +272,10 @@ public void testCancellation() throws Exception { TestingRpcService rpc = new TestingRpcService(); try { - Promise result = Futures.promise(); + FlinkCompletableFuture result = new FlinkCompletableFuture<>(); TestRegistrationGateway testGateway = mock(TestRegistrationGateway.class); - when(testGateway.registrationCall(any(UUID.class), anyLong())).thenReturn(result.future()); + when(testGateway.registrationCall(any(UUID.class), anyLong())).thenReturn(result); rpc.registerGateway(testEndpointAddress, testGateway); @@ -283,7 +284,7 @@ public void testCancellation() throws Exception { // cancel and fail the current registration attempt registration.cancel(); - result.failure(new TimeoutException()); + result.completeExceptionally(new TimeoutException()); // there should not be a second registration attempt verify(testGateway, atMost(1)).registrationCall(any(UUID.class), anyLong()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/registration/TestRegistrationGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/TestRegistrationGateway.java index 431fbe8284cb0..2843aebf26e9f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/registration/TestRegistrationGateway.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/TestRegistrationGateway.java @@ -20,11 +20,11 @@ import akka.dispatch.Futures; +import org.apache.flink.runtime.concurrent.Future; +import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture; import org.apache.flink.runtime.rpc.TestingGatewayBase; import org.apache.flink.util.Preconditions; -import scala.concurrent.Future; - import java.util.UUID; import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; @@ -56,7 +56,7 @@ public Future registrationCall(UUID leaderId, long timeout } // return a completed future (for a proper value), or one that never completes and will time out (for null) - return response != null ? Futures.successful(response) : this.futureWithTimeout(timeout); + return response != null ? FlinkCompletableFuture.completed(response) : this.futureWithTimeout(timeout); } public BlockingQueue getInvocations() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java index 8183c0a857562..64a1191211dea 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java @@ -21,7 +21,7 @@ import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; -import org.apache.flink.runtime.rpc.MainThreadExecutor; +import org.apache.flink.runtime.rpc.MainThreadExecutable; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcService; @@ -68,7 +68,7 @@ public void testGrantAndRevokeLeadership() throws Exception { Assert.assertNull(resourceManager.getLeaderSessionID()); } - private static abstract class TestingResourceManagerGatewayProxy implements MainThreadExecutor, StartStoppable, RpcGateway { + private static abstract class TestingResourceManagerGatewayProxy implements MainThreadExecutable, StartStoppable, RpcGateway { @Override public void runAsync(Runnable runnable) { runnable.run(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java index 85d28805d4d64..1f9e7e833286b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java @@ -18,10 +18,12 @@ package org.apache.flink.runtime.resourcemanager.slotmanager; import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.clusterframework.types.SlotID; +import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.highavailability.NonHaServices; import org.apache.flink.runtime.jobmaster.JobMasterGateway; import org.apache.flink.runtime.resourcemanager.JobMasterRegistration; @@ -40,10 +42,6 @@ import org.junit.BeforeClass; import org.junit.Test; import org.mockito.Mockito; -import scala.concurrent.Await; -import scala.concurrent.Future; -import scala.concurrent.duration.Duration; -import scala.concurrent.duration.FiniteDuration; import java.util.Collections; import java.util.UUID; @@ -99,7 +97,7 @@ public void testSlotsUnavailableRequest() throws Exception { Future registrationFuture = resourceManager.registerJobMaster(new JobMasterRegistration(jmAddress, jobID)); try { - Await.ready(registrationFuture, Duration.create(5, TimeUnit.SECONDS)); + registrationFuture.get(5, TimeUnit.SECONDS); } catch (Exception e) { Assert.fail("JobManager registration Future didn't become ready."); } @@ -141,7 +139,7 @@ public void testSlotsUnavailableRequest() throws Exception { slotManager.updateSlotStatus(slotReport); // 4) Slot becomes available and TaskExecutor gets a SlotRequest - verify(taskExecutorGateway, timeout(5000)).requestSlot(eq(allocationID), any(UUID.class), any(FiniteDuration.class)); + verify(taskExecutorGateway, timeout(5000)).requestSlot(eq(allocationID), any(UUID.class), any(Time.class)); } /** @@ -171,7 +169,7 @@ public void testSlotAvailableRequest() throws Exception { Future registrationFuture = resourceManager.registerJobMaster(new JobMasterRegistration(jmAddress, jobID)); try { - Await.ready(registrationFuture, Duration.create(5, TimeUnit.SECONDS)); + registrationFuture.get(5, TimeUnit.SECONDS); } catch (Exception e) { Assert.fail("JobManager registration Future didn't become ready."); } @@ -207,7 +205,7 @@ public void testSlotAvailableRequest() throws Exception { // 4) a SlotRequest is routed to the TaskExecutor - verify(taskExecutorGateway, timeout(5000)).requestSlot(eq(allocationID), any(UUID.class), any(FiniteDuration.class)); + verify(taskExecutorGateway, timeout(5000)).requestSlot(eq(allocationID), any(UUID.class), any(Time.class)); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java index 1791056424e55..7c6b0ee34f746 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java @@ -21,18 +21,16 @@ import akka.actor.ActorSystem; import akka.util.Timeout; +import org.apache.flink.api.common.time.Time; import org.apache.flink.core.testutils.OneShotLatch; import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.rpc.akka.AkkaRpcService; import org.apache.flink.util.TestLogger; import org.junit.AfterClass; import org.junit.Test; -import scala.concurrent.Await; -import scala.concurrent.Future; -import scala.concurrent.duration.FiniteDuration; - import java.util.concurrent.Callable; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -49,7 +47,7 @@ public class AsyncCallsTest extends TestLogger { private static ActorSystem actorSystem = AkkaUtils.createDefaultActorSystem(); private static AkkaRpcService akkaRpcService = - new AkkaRpcService(actorSystem, new Timeout(10000, TimeUnit.MILLISECONDS)); + new AkkaRpcService(actorSystem, Time.milliseconds(10000L)); @AfterClass public static void shutdown() { @@ -104,8 +102,9 @@ public String call() throws Exception { } return "test"; } - }, new Timeout(30, TimeUnit.SECONDS)); - String str = Await.result(result, new FiniteDuration(30, TimeUnit.SECONDS)); + }, Time.seconds(30L)); + + String str = result.get(30, TimeUnit.SECONDS); assertEquals("test", str); // validate that no concurrent access happened diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java index b431eb92be2cd..ee3f784198e76 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java @@ -18,14 +18,14 @@ package org.apache.flink.runtime.rpc; +import org.apache.flink.api.common.time.Time; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.util.ReflectionUtil; import org.apache.flink.util.TestLogger; import org.junit.Test; import org.reflections.Reflections; -import scala.concurrent.Future; -import scala.concurrent.duration.FiniteDuration; import java.lang.annotation.Annotation; import java.lang.reflect.Method; @@ -43,6 +43,7 @@ public class RpcCompletenessTest extends TestLogger { private static final Class futureClass = Future.class; + private static final Class timeoutClass = Time.class; @Test @SuppressWarnings({"rawtypes", "unchecked"}) @@ -147,8 +148,8 @@ private void checkGatewayMethod(Method gatewayMethod) { for (int i = 0; i < parameterAnnotations.length; i++) { if (RpcCompletenessTest.isRpcTimeout(parameterAnnotations[i])) { assertTrue( - "The rpc timeout has to be of type " + FiniteDuration.class.getName() + ".", - parameterTypes[i].equals(FiniteDuration.class)); + "The rpc timeout has to be of type " + timeoutClass.getName() + ".", + parameterTypes[i].equals(timeoutClass)); rpcTimeoutParameters++; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingGatewayBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingGatewayBase.java index 8133a87e78b01..caf5e81f4099f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingGatewayBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingGatewayBase.java @@ -18,9 +18,9 @@ package org.apache.flink.runtime.rpc; -import akka.dispatch.Futures; -import scala.concurrent.Future; -import scala.concurrent.Promise; +import org.apache.flink.runtime.concurrent.CompletableFuture; +import org.apache.flink.runtime.concurrent.Future; +import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -73,25 +73,25 @@ public String getAddress() { // ------------------------------------------------------------------------ public Future futureWithTimeout(long timeoutMillis) { - Promise promise = Futures.promise(); - executor.schedule(new FutureTimeout(promise), timeoutMillis, TimeUnit.MILLISECONDS); - return promise.future(); + FlinkCompletableFuture future = new FlinkCompletableFuture<>(); + executor.schedule(new FutureTimeout(future), timeoutMillis, TimeUnit.MILLISECONDS); + return future; } // ------------------------------------------------------------------------ private static final class FutureTimeout implements Runnable { - private final Promise promise; + private final CompletableFuture promise; - private FutureTimeout(Promise promise) { + private FutureTimeout(CompletableFuture promise) { this.promise = promise; } @Override public void run() { try { - promise.failure(new TimeoutException()); + promise.completeExceptionally(new TimeoutException()); } catch (Throwable t) { System.err.println("CAUGHT AN ERROR IN THE TEST: " + t.getMessage()); t.printStackTrace(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java index 2212680629290..f1640565519c2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java @@ -18,18 +18,14 @@ package org.apache.flink.runtime.rpc; -import akka.dispatch.Futures; -import akka.util.Timeout; - +import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.concurrent.Future; +import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture; import org.apache.flink.runtime.rpc.akka.AkkaRpcService; -import scala.concurrent.Future; -import scala.concurrent.duration.FiniteDuration; - import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.TimeUnit; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -69,7 +65,7 @@ public TestingRpcService() { * Creates a new {@code TestingRpcService}, using the given configuration. */ public TestingRpcService(Configuration configuration) { - super(AkkaUtils.createLocalActorSystem(configuration), new Timeout(new FiniteDuration(10, TimeUnit.SECONDS))); + super(AkkaUtils.createLocalActorSystem(configuration), Time.seconds(10)); this.registeredConnections = new ConcurrentHashMap<>(); } @@ -103,13 +99,13 @@ public Future connect(String address, Class clazz) if (clazz.isAssignableFrom(gateway.getClass())) { @SuppressWarnings("unchecked") C typedGateway = (C) gateway; - return Futures.successful(typedGateway); + return FlinkCompletableFuture.completed(typedGateway); } else { - return Futures.failed( - new Exception("Gateway registered under " + address + " is not of type " + clazz)); + return FlinkCompletableFuture.completedExceptionally( + new Exception("Gateway registered under " + address + " is not of type " + clazz)); } } else { - return Futures.failed(new Exception("No gateway registered under that name")); + return FlinkCompletableFuture.completedExceptionally(new Exception("No gateway registered under that name")); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java index 01776edc5a549..957453a9d3660 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java @@ -18,16 +18,13 @@ package org.apache.flink.runtime.rpc; -import akka.dispatch.ExecutionContexts; import akka.dispatch.Futures; -import akka.util.Timeout; +import org.apache.flink.api.common.time.Time; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.concurrent.Future; +import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture; import org.apache.flink.runtime.util.DirectExecutorService; import org.apache.flink.util.Preconditions; -import scala.concurrent.Await; -import scala.concurrent.ExecutionContext; -import scala.concurrent.Future; -import scala.concurrent.duration.FiniteDuration; import java.lang.annotation.Annotation; import java.lang.reflect.InvocationHandler; @@ -37,6 +34,7 @@ import java.util.UUID; import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executor; import java.util.concurrent.TimeUnit; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -67,8 +65,8 @@ public void scheduleRunnable(final Runnable runnable, final long delay, final Ti } @Override - public ExecutionContext getExecutionContext() { - return ExecutionContexts.fromExecutorService(executorService); + public Executor getExecutor() { + return executorService; } @Override @@ -94,7 +92,7 @@ public > C startServer(S rpcEndpo classLoader, new Class[]{ rpcEndpoint.getSelfGatewayType(), - MainThreadExecutor.class, + MainThreadExecutable.class, StartStoppable.class, RpcGateway.class }, @@ -114,13 +112,13 @@ public Future connect(String address, Class clazz) if (clazz.isAssignableFrom(gateway.getClass())) { @SuppressWarnings("unchecked") C typedGateway = (C) gateway; - return Futures.successful(typedGateway); + return FlinkCompletableFuture.completed(typedGateway); } else { - return Futures.failed( + return FlinkCompletableFuture.completedExceptionally( new Exception("Gateway registered under " + address + " is not of type " + clazz)); } } else { - return Futures.failed(new Exception("No gateway registered under that name")); + return FlinkCompletableFuture.completedExceptionally(new Exception("No gateway registered under that name")); } } @@ -141,20 +139,20 @@ public void clearGateways() { registeredConnections.clear(); } - private static final class TestingSerialInvocationHandler> implements InvocationHandler, RpcGateway, MainThreadExecutor, StartStoppable { + private static final class TestingSerialInvocationHandler> implements InvocationHandler, RpcGateway, MainThreadExecutable, StartStoppable { private final T rpcEndpoint; /** default timeout for asks */ - private final Timeout timeout; + private final Time timeout; private final String address; private TestingSerialInvocationHandler(String address, T rpcEndpoint) { - this(address, rpcEndpoint, new Timeout(new FiniteDuration(10, TimeUnit.SECONDS))); + this(address, rpcEndpoint, Time.seconds(10)); } - private TestingSerialInvocationHandler(String address, T rpcEndpoint, Timeout timeout) { + private TestingSerialInvocationHandler(String address, T rpcEndpoint, Time timeout) { this.rpcEndpoint = rpcEndpoint; this.timeout = timeout; this.address = address; @@ -163,7 +161,7 @@ private TestingSerialInvocationHandler(String address, T rpcEndpoint, Timeout ti @Override public Object invoke(Object proxy, Method method, Object[] args) throws Throwable { Class declaringClass = method.getDeclaringClass(); - if (declaringClass.equals(MainThreadExecutor.class) || + if (declaringClass.equals(MainThreadExecutable.class) || declaringClass.equals(Object.class) || declaringClass.equals(StartStoppable.class) || declaringClass.equals(RpcGateway.class)) { return method.invoke(this, args); @@ -171,7 +169,7 @@ public Object invoke(Object proxy, Method method, Object[] args) throws Throwabl final String methodName = method.getName(); Class[] parameterTypes = method.getParameterTypes(); Annotation[][] parameterAnnotations = method.getParameterAnnotations(); - Timeout futureTimeout = extractRpcTimeout(parameterAnnotations, args, timeout); + Time futureTimeout = extractRpcTimeout(parameterAnnotations, args, timeout); final Tuple2[], Object[]> filteredArguments = filterArguments( parameterTypes, @@ -201,13 +199,13 @@ public Object invoke(Object proxy, Method method, Object[] args) throws Throwabl private Object handleRpcInvocationSync(final String methodName, final Class[] parameterTypes, final Object[] args, - final Timeout futureTimeout) throws Exception { + final Time futureTimeout) throws Exception { final Method rpcMethod = lookupRpcMethod(methodName, parameterTypes); Object result = rpcMethod.invoke(rpcEndpoint, args); if (result instanceof Future) { Future future = (Future) result; - return Await.result(future, futureTimeout.duration()); + return future.get(futureTimeout.getSize(), futureTimeout.getUnit()); } else { return result; } @@ -219,11 +217,11 @@ public void runAsync(Runnable runnable) { } @Override - public Future callAsync(Callable callable, Timeout callTimeout) { + public Future callAsync(Callable callable, Time callTimeout) { try { - return Futures.successful(callable.call()); + return FlinkCompletableFuture.completed(callable.call()); } catch (Throwable e) { - return Futures.failed(e); + return FlinkCompletableFuture.completedExceptionally(e); } } @@ -281,18 +279,18 @@ private Method lookupRpcMethod(final String methodName, * has been found * @return Timeout extracted from the array of arguments or the default timeout */ - private static Timeout extractRpcTimeout(Annotation[][] parameterAnnotations, Object[] args, - Timeout defaultTimeout) { + private static Time extractRpcTimeout(Annotation[][] parameterAnnotations, Object[] args, + Time defaultTimeout) { if (args != null) { Preconditions.checkArgument(parameterAnnotations.length == args.length); for (int i = 0; i < parameterAnnotations.length; i++) { if (isRpcTimeout(parameterAnnotations[i])) { - if (args[i] instanceof FiniteDuration) { - return new Timeout((FiniteDuration) args[i]); + if (args[i] instanceof Time) { + return (Time) args[i]; } else { throw new RuntimeException("The rpc timeout parameter must be of type " + - FiniteDuration.class.getName() + ". The type " + args[i].getClass().getName() + + Time.class.getName() + ". The type " + args[i].getClass().getName() + " is not supported."); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java index a6ceb9104a1ef..5624d12fc5255 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java @@ -19,8 +19,9 @@ package org.apache.flink.runtime.rpc.akka; import akka.actor.ActorSystem; -import akka.util.Timeout; +import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcMethod; @@ -30,13 +31,12 @@ import org.hamcrest.core.Is; import org.junit.AfterClass; import org.junit.Test; -import scala.concurrent.Await; -import scala.concurrent.Future; -import java.util.concurrent.TimeUnit; +import java.util.concurrent.ExecutionException; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; public class AkkaRpcActorTest extends TestLogger { @@ -47,7 +47,7 @@ public class AkkaRpcActorTest extends TestLogger { private static ActorSystem actorSystem = AkkaUtils.createDefaultActorSystem(); - private static Timeout timeout = new Timeout(10000, TimeUnit.MILLISECONDS); + private static Time timeout = Time.milliseconds(10000L); private static AkkaRpcService akkaRpcService = new AkkaRpcService(actorSystem, timeout); @@ -69,7 +69,7 @@ public void testAddressResolution() throws Exception { Future futureRpcGateway = akkaRpcService.connect(rpcEndpoint.getAddress(), DummyRpcGateway.class); - DummyRpcGateway rpcGateway = Await.result(futureRpcGateway, timeout.duration()); + DummyRpcGateway rpcGateway = futureRpcGateway.get(timeout.getSize(), timeout.getUnit()); assertEquals(rpcEndpoint.getAddress(), rpcGateway.getAddress()); } @@ -82,11 +82,12 @@ public void testFailingAddressResolution() throws Exception { Future futureRpcGateway = akkaRpcService.connect("foobar", DummyRpcGateway.class); try { - DummyRpcGateway gateway = Await.result(futureRpcGateway, timeout.duration()); + DummyRpcGateway gateway = futureRpcGateway.get(timeout.getSize(), timeout.getUnit()); fail("The rpc connection resolution should have failed."); - } catch (RpcConnectionException exception) { + } catch (ExecutionException exception) { // we're expecting a RpcConnectionException + assertTrue(exception.getCause() instanceof RpcConnectionException); } } @@ -111,7 +112,7 @@ public void testMessageStashing() throws Exception { // now process the rpc rpcEndpoint.start(); - Integer actualValue = Await.result(result, timeout.duration()); + Integer actualValue = result.get(timeout.getSize(), timeout.getUnit()); assertThat("The new foobar value should have been returned.", actualValue, Is.is(expectedValue)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java index f55069e44e5b1..4e9e51805b637 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.rpc.akka; import akka.actor.ActorSystem; -import akka.util.Timeout; +import org.apache.flink.api.common.time.Time; import org.apache.flink.core.testutils.OneShotLatch; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.util.TestLogger; @@ -40,7 +40,7 @@ public class AkkaRpcServiceTest extends TestLogger { private static ActorSystem actorSystem = AkkaUtils.createDefaultActorSystem(); private static AkkaRpcService akkaRpcService = - new AkkaRpcService(actorSystem, new Timeout(10000, TimeUnit.MILLISECONDS)); + new AkkaRpcService(actorSystem, Time.milliseconds(10000)); @AfterClass public static void shutdown() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java index 9ffafdaec46b6..9ec1f7edab8c7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java @@ -18,8 +18,7 @@ package org.apache.flink.runtime.rpc.akka; -import akka.util.Timeout; - +import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.rpc.RpcEndpoint; @@ -30,8 +29,6 @@ import org.apache.flink.util.TestLogger; import org.junit.Test; -import java.util.concurrent.TimeUnit; - import static org.junit.Assert.assertTrue; public class MainThreadValidationTest extends TestLogger { @@ -48,7 +45,7 @@ public void failIfNotInMainThread() { // actual test AkkaRpcService akkaRpcService = new AkkaRpcService( AkkaUtils.createDefaultActorSystem(), - new Timeout(10000, TimeUnit.MILLISECONDS)); + Time.milliseconds(10000)); try { TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java index 9d2ed99c080cd..0d5dc282d06e6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java @@ -19,10 +19,11 @@ package org.apache.flink.runtime.rpc.akka; import akka.actor.ActorSystem; -import akka.util.Timeout; import com.typesafe.config.Config; import com.typesafe.config.ConfigValueFactory; +import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcMethod; @@ -32,13 +33,9 @@ import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; -import scala.concurrent.Await; -import scala.concurrent.Future; -import scala.concurrent.duration.FiniteDuration; import java.io.IOException; import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.TimeUnit; import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; @@ -52,7 +49,7 @@ public class MessageSerializationTest extends TestLogger { private static AkkaRpcService akkaRpcService1; private static AkkaRpcService akkaRpcService2; - private static final FiniteDuration timeout = new FiniteDuration(10L, TimeUnit.SECONDS); + private static final Time timeout = Time.seconds(10L); private static final int maxFrameSize = 32000; @BeforeClass @@ -63,8 +60,8 @@ public static void setup() { actorSystem1 = AkkaUtils.createActorSystem(modifiedAkkaConfig); actorSystem2 = AkkaUtils.createActorSystem(modifiedAkkaConfig); - akkaRpcService1 = new AkkaRpcService(actorSystem1, new Timeout(timeout)); - akkaRpcService2 = new AkkaRpcService(actorSystem2, new Timeout(timeout)); + akkaRpcService1 = new AkkaRpcService(actorSystem1, timeout); + akkaRpcService2 = new AkkaRpcService(actorSystem2, timeout); } @AfterClass @@ -113,7 +110,7 @@ public void testNonSerializableRemoteMessageTransfer() throws Exception { Future remoteGatewayFuture = akkaRpcService2.connect(address, TestGateway.class); - TestGateway remoteGateway = Await.result(remoteGatewayFuture, timeout); + TestGateway remoteGateway = remoteGatewayFuture.get(timeout.getSize(), timeout.getUnit()); remoteGateway.foobar(new Object()); @@ -134,7 +131,7 @@ public void testSerializableRemoteMessageTransfer() throws Exception { Future remoteGatewayFuture = akkaRpcService2.connect(address, TestGateway.class); - TestGateway remoteGateway = Await.result(remoteGatewayFuture, timeout); + TestGateway remoteGateway = remoteGatewayFuture.get(timeout.getSize(), timeout.getUnit()); int expected = 42; @@ -158,7 +155,7 @@ public void testMaximumFramesizeRemoteMessageTransfer() throws Exception { Future remoteGatewayFuture = akkaRpcService2.connect(address, TestGateway.class); - TestGateway remoteGateway = Await.result(remoteGatewayFuture, timeout); + TestGateway remoteGateway = remoteGatewayFuture.get(timeout.getSize(), timeout.getUnit()); int bufferSize = maxFrameSize + 1; byte[] buffer = new byte[bufferSize]; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java index a8d5bd76cf829..09aab183782e2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.taskexecutor; +import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.highavailability.NonHaServices; @@ -29,8 +30,6 @@ import org.junit.Test; -import scala.concurrent.duration.FiniteDuration; - import java.util.UUID; import static org.junit.Assert.*; @@ -56,7 +55,7 @@ public void testImmediatelyRegistersIfLeaderIsKnown() throws Exception { taskManager.start(); verify(rmGateway, timeout(5000)).registerTaskExecutor( - any(UUID.class), eq(taskManagerAddress), eq(resourceID), any(FiniteDuration.class)); + any(UUID.class), eq(taskManagerAddress), eq(resourceID), any(Time.class)); } finally { rpc.stopService(); @@ -97,7 +96,7 @@ public void testTriggerRegistrationOnLeaderChange() throws Exception { testLeaderService.notifyListener(address1, leaderId1); verify(rmGateway1, timeout(5000)).registerTaskExecutor( - eq(leaderId1), eq(taskManagerAddress), eq(resourceID), any(FiniteDuration.class)); + eq(leaderId1), eq(taskManagerAddress), eq(resourceID), any(Time.class)); assertNotNull(taskManager.getResourceManagerConnection()); // cancel the leader @@ -107,7 +106,7 @@ public void testTriggerRegistrationOnLeaderChange() throws Exception { testLeaderService.notifyListener(address2, leaderId2); verify(rmGateway2, timeout(5000)).registerTaskExecutor( - eq(leaderId2), eq(taskManagerAddress), eq(resourceID), any(FiniteDuration.class)); + eq(leaderId2), eq(taskManagerAddress), eq(resourceID), any(Time.class)); assertNotNull(taskManager.getResourceManagerConnection()); } finally { From 6159f56a16d407f4a3bf74cdbc578a705e3e063b Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 21 Sep 2016 18:16:27 +0200 Subject: [PATCH 31/50] [FLINK-4658] [rpc] Allow RpcService to execute Runnables and Callables in its executor This closes #2531. --- .../apache/flink/runtime/rpc/RpcService.java | 31 +++++++++++++ .../runtime/rpc/akka/AkkaRpcService.java | 14 ++++++ .../flink/runtime/rpc/AsyncCallsTest.java | 1 - .../runtime/rpc/TestingSerialRpcService.java | 16 +++++++ .../runtime/rpc/akka/AkkaRpcServiceTest.java | 45 +++++++++++++++++++ 5 files changed, 106 insertions(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java index a367ff2fdad9a..437e08b0a3e59 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java @@ -21,6 +21,7 @@ import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException; +import java.util.concurrent.Callable; import java.util.concurrent.Executor; import java.util.concurrent.TimeUnit; @@ -89,4 +90,34 @@ public interface RpcService { * @param delay The delay after which the runnable will be executed */ void scheduleRunnable(Runnable runnable, long delay, TimeUnit unit); + + /** + * Execute the given runnable in the executor of the RPC service. This method can be used to run + * code outside of the main thread of a {@link RpcEndpoint}. + * + *

    IMPORTANT: This executor does not isolate the method invocations against + * any concurrent invocations and is therefore not suitable to run completion methods of futures + * that modify state of an {@link RpcEndpoint}. For such operations, one needs to use the + * {@link RpcEndpoint#getMainThreadExecutor() MainThreadExecutionContext} of that + * {@code RpcEndpoint}. + * + * @param runnable to execute + */ + void execute(Runnable runnable); + + /** + * Execute the given callable and return its result as a {@link Future}. This method can be used + * to run code outside of the main thread of a {@link RpcEndpoint}. + * + *

    IMPORTANT: This executor does not isolate the method invocations against + * any concurrent invocations and is therefore not suitable to run completion methods of futures + * that modify state of an {@link RpcEndpoint}. For such operations, one needs to use the + * {@link RpcEndpoint#getMainThreadExecutor() MainThreadExecutionContext} of that + * {@code RpcEndpoint}. + * + * @param callable to execute + * @param is the return value type + * @return Future containing the callable's future result + */ + Future execute(Callable callable); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java index 36f11152f26fd..cee19c4d00b97 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java @@ -25,6 +25,7 @@ import akka.actor.Identify; import akka.actor.PoisonPill; import akka.actor.Props; +import akka.dispatch.Futures; import akka.dispatch.Mapper; import akka.pattern.Patterns; @@ -48,6 +49,7 @@ import java.lang.reflect.Proxy; import java.util.HashSet; import java.util.Set; +import java.util.concurrent.Callable; import java.util.concurrent.Executor; import java.util.concurrent.TimeUnit; @@ -223,4 +225,16 @@ public void scheduleRunnable(Runnable runnable, long delay, TimeUnit unit) { actorSystem.scheduler().scheduleOnce(new FiniteDuration(delay, unit), runnable, actorSystem.dispatcher()); } + + @Override + public void execute(Runnable runnable) { + actorSystem.dispatcher().execute(runnable); + } + + @Override + public Future execute(Callable callable) { + scala.concurrent.Future scalaFuture = Futures.future(callable, actorSystem.dispatcher()); + + return new FlinkFuture<>(scalaFuture); + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java index 7c6b0ee34f746..e8255d40dacee 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java @@ -19,7 +19,6 @@ package org.apache.flink.runtime.rpc; import akka.actor.ActorSystem; -import akka.util.Timeout; import org.apache.flink.api.common.time.Time; import org.apache.flink.core.testutils.OneShotLatch; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java index 957453a9d3660..c58ea205c953a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java @@ -64,6 +64,22 @@ public void scheduleRunnable(final Runnable runnable, final long delay, final Ti } } + @Override + public void execute(Runnable runnable) { + runnable.run(); + } + + @Override + public Future execute(Callable callable) { + try { + T result = callable.call(); + + return FlinkCompletableFuture.completed(result); + } catch (Exception e) { + return FlinkCompletableFuture.completedExceptionally(e); + } + } + @Override public Executor getExecutor() { return executorService; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java index 4e9e51805b637..5550cb5701aa2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java @@ -22,13 +22,18 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.core.testutils.OneShotLatch; import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.util.TestLogger; import org.junit.AfterClass; import org.junit.Test; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; public class AkkaRpcServiceTest extends TestLogger { @@ -70,4 +75,44 @@ public void run() { assertTrue("call was not properly delayed", ((stop - start) / 1000000) >= delay); } + + /** + * Tests that the {@link AkkaRpcService} can execute runnables + */ + @Test + public void testExecuteRunnable() throws Exception { + final OneShotLatch latch = new OneShotLatch(); + + akkaRpcService.execute(new Runnable() { + @Override + public void run() { + latch.trigger(); + } + }); + + latch.await(30L, TimeUnit.SECONDS); + } + + /** + * Tests that the {@link AkkaRpcService} can execute callables and returns their result as + * a {@link Future}. + */ + @Test + public void testExecuteCallable() throws InterruptedException, ExecutionException, TimeoutException { + final OneShotLatch latch = new OneShotLatch(); + final int expected = 42; + + Future result = akkaRpcService.execute(new Callable() { + @Override + public Integer call() throws Exception { + latch.trigger(); + return expected; + } + }); + + int actual = result.get(30L, TimeUnit.SECONDS); + + assertEquals(expected, actual); + assertTrue(latch.isTriggered()); + } } From deb8022624498bc8843db18570c531ab90145031 Mon Sep 17 00:00:00 2001 From: beyond1920 Date: Thu, 1 Sep 2016 15:27:20 +0800 Subject: [PATCH 32/50] [FLINK-4537] [cluster management] ResourceManager registration with JobManager --- .../HighAvailabilityServices.java | 9 + .../highavailability/NonHaServices.java | 19 +++ .../JobMasterRegistrationSuccess.java | 49 ++++++ .../JobMasterRegistration.java | 39 ++++- .../resourcemanager/ResourceManager.java | 125 ++++++++++++-- .../ResourceManagerGateway.java | 34 ++-- .../exceptions/LeaderSessionIDException.java | 60 +++++++ .../runtime/taskexecutor/TaskExecutor.java | 5 + .../TestingHighAvailabilityServices.java | 17 ++ .../resourcemanager/ResourceManagerTest.java | 160 ++++++++++++++++++ 10 files changed, 483 insertions(+), 34 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterRegistrationSuccess.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/LeaderSessionIDException.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java index 298147cf1dce1..7634176b4dca6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java @@ -39,6 +39,15 @@ public interface HighAvailabilityServices { */ LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception; + /** + * Gets the leader retriever for the job JobMaster which is responsible for the given job + * + * @param jobID The identifier of the job. + * @return + * @throws Exception + */ + LeaderRetrievalService getJobMasterLeaderRetriever(JobID jobID) throws Exception; + /** * Gets the leader election service for the cluster's resource manager. * @return diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java index 292a404118665..33dc2d7c40168 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.highavailability; import org.apache.flink.api.common.JobID; +import org.apache.flink.hadoop.shaded.org.jboss.netty.util.internal.ConcurrentHashMap; import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.leaderelection.StandaloneLeaderElectionService; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; @@ -42,6 +43,8 @@ public class NonHaServices implements HighAvailabilityServices { /** The fix address of the ResourceManager */ private final String resourceManagerAddress; + private final ConcurrentHashMap jobMastersAddress; + /** * Creates a new services class for the fix pre-defined leaders. * @@ -49,6 +52,17 @@ public class NonHaServices implements HighAvailabilityServices { */ public NonHaServices(String resourceManagerAddress) { this.resourceManagerAddress = checkNotNull(resourceManagerAddress); + this.jobMastersAddress = new ConcurrentHashMap<>(16); + } + + /** + * Binds address of a specified job master + * + * @param jobID JobID for the specified job master + * @param jobMasterAddress address for the specified job master + */ + public void bindJobMasterLeaderAddress(JobID jobID, String jobMasterAddress) { + jobMastersAddress.put(jobID, jobMasterAddress); } // ------------------------------------------------------------------------ @@ -60,6 +74,11 @@ public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Excepti return new StandaloneLeaderRetrievalService(resourceManagerAddress, new UUID(0, 0)); } + @Override + public LeaderRetrievalService getJobMasterLeaderRetriever(JobID jobID) throws Exception { + return new StandaloneLeaderRetrievalService(jobMastersAddress.get(jobID), new UUID(0, 0)); + } + @Override public LeaderElectionService getResourceManagerLeaderElectionService() throws Exception { return new StandaloneLeaderElectionService(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterRegistrationSuccess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterRegistrationSuccess.java new file mode 100644 index 0000000000000..031c38e47326f --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterRegistrationSuccess.java @@ -0,0 +1,49 @@ +/* + * 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.jobmaster; + +import org.apache.flink.runtime.registration.RegistrationResponse; + +/** + * Base class for responses from the ResourceManager to a registration attempt by a JobMaster. + */ +public class JobMasterRegistrationSuccess extends RegistrationResponse.Success { + + private static final long serialVersionUID = 5577641250204140415L; + + private final long heartbeatInterval; + + public JobMasterRegistrationSuccess(long heartbeatInterval) { + this.heartbeatInterval = heartbeatInterval; + } + + /** + * Gets the interval in which the ResourceManager will heartbeat the JobMaster. + * + * @return the interval in which the ResourceManager will heartbeat the JobMaster + */ + public long getHeartbeatInterval() { + return heartbeatInterval; + } + + @Override + public String toString() { + return "JobMasterRegistrationSuccess(" + heartbeatInterval + ')'; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java index 439e56b0402d8..7b8ec707eb943 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java @@ -18,23 +18,56 @@ package org.apache.flink.runtime.resourcemanager; +<<<<<<< HEAD import org.apache.flink.api.common.JobID; +======= +import org.apache.flink.runtime.jobmaster.JobMasterGateway; +>>>>>>> db98efb... rsourceManager registration with JobManager import java.io.Serializable; +import java.util.UUID; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * This class is responsible for group the JobMasterGateway and the LeaderSessionID of a registered job master + */ public class JobMasterRegistration implements Serializable { - private static final long serialVersionUID = 8411214999193765202L; +<<<<<<< HEAD private final String address; private final JobID jobID; public JobMasterRegistration(String address, JobID jobID) { this.address = address; this.jobID = jobID; +======= + private static final long serialVersionUID = -2316627821716999527L; + + private final JobMasterGateway jobMasterGateway; + + private UUID jobMasterLeaderSessionID; + + public JobMasterRegistration(JobMasterGateway jobMasterGateway) { + this.jobMasterGateway = checkNotNull(jobMasterGateway); + } + + public JobMasterRegistration(JobMasterGateway jobMasterGateway, UUID jobMasterLeaderSessionID) { + this.jobMasterGateway = checkNotNull(jobMasterGateway); + this.jobMasterLeaderSessionID = jobMasterLeaderSessionID; + } + + public JobMasterGateway getJobMasterGateway() { + return jobMasterGateway; + } + + public void setJobMasterLeaderSessionID(UUID leaderSessionID) { + this.jobMasterLeaderSessionID = jobMasterLeaderSessionID; +>>>>>>> db98efb... rsourceManager registration with JobManager } - public String getAddress() { - return address; + public UUID getJobMasterLeaderSessionID() { + return jobMasterLeaderSessionID; } public JobID getJobID() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java index 537071085aedb..8be145555cbc1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.resourcemanager; +import akka.dispatch.Futures; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.clusterframework.types.ResourceID; @@ -25,15 +26,22 @@ import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.instance.InstanceID; +import org.apache.flink.runtime.jobmaster.JobMasterRegistrationSuccess; import org.apache.flink.runtime.leaderelection.LeaderContender; import org.apache.flink.runtime.leaderelection.LeaderElectionService; +<<<<<<< HEAD import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; +======= +import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener; +>>>>>>> db98efb... rsourceManager registration with JobManager import org.apache.flink.runtime.rpc.RpcMethod; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.jobmaster.JobMaster; import org.apache.flink.runtime.jobmaster.JobMasterGateway; +import org.apache.flink.runtime.rpc.exceptions.LeaderSessionIDException; import org.apache.flink.runtime.taskexecutor.TaskExecutorRegistrationSuccess; +import org.apache.flink.runtime.registration.RegistrationResponse; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -50,15 +58,21 @@ * * It offers the following methods as part of its rpc interface to interact with the him remotely: *

      - *
    • {@link #registerJobMaster(JobMasterRegistration)} registers a {@link JobMaster} at the resource manager
    • + *
    • {@link #registerJobMaster(UUID, String, JobID)} registers a {@link JobMaster} at the resource manager
    • *
    • {@link #requestSlot(SlotRequest)} requests a slot from the resource manager
    • *
    */ +<<<<<<< HEAD public class ResourceManager extends RpcEndpoint implements LeaderContender { private final Logger LOG = LoggerFactory.getLogger(getClass()); private final Map jobMasterGateways; +======= +public class ResourceManager extends RpcEndpoint { + /** the mapping relationship of JobID and JobMasterGateway */ + private final Map jobMasters; +>>>>>>> db98efb... rsourceManager registration with JobManager private final HighAvailabilityServices highAvailabilityServices; @@ -74,8 +88,12 @@ public ResourceManager( SlotManager slotManager) { super(rpcService); this.highAvailabilityServices = checkNotNull(highAvailabilityServices); +<<<<<<< HEAD this.jobMasterGateways = new HashMap<>(); this.slotManager = slotManager; +======= + this.jobMasters = new HashMap<>(16); +>>>>>>> db98efb... rsourceManager registration with JobManager } @Override @@ -95,8 +113,11 @@ public void start() { public void shutDown() { try { leaderElectionService.stop(); + for(JobID jobID : jobMasters.keySet()) { + highAvailabilityServices.getJobMasterLeaderRetriever(jobID).stop(); + } super.shutDown(); - } catch(Throwable e) { + } catch (Throwable e) { log.error("A fatal error happened when shutdown the ResourceManager", e); throw new RuntimeException("A fatal error happened when shutdown the ResourceManager", e); } @@ -115,24 +136,58 @@ UUID getLeaderSessionID() { /** * Register a {@link JobMaster} at the resource manager. * - * @param jobMasterRegistration Job master registration information + * @param resourceManagerLeaderId The fencing token for the ResourceManager leader + * @param jobMasterAddress The address of the JobMaster that registers + * @param jobID The Job ID of the JobMaster that registers * @return Future registration response */ @RpcMethod +<<<<<<< HEAD public Future registerJobMaster(JobMasterRegistration jobMasterRegistration) { final Future jobMasterFuture = getRpcService().connect(jobMasterRegistration.getAddress(), JobMasterGateway.class); final JobID jobID = jobMasterRegistration.getJobID(); +======= + public Future registerJobMaster(UUID resourceManagerLeaderId, final String jobMasterAddress, final JobID jobID) { + + if(!leaderSessionID.equals(resourceManagerLeaderId)) { + log.warn("Discard registration from JobMaster {} at ({}) because the expected leader session ID {} did not equal the received leader session ID {}", + jobID, jobMasterAddress, leaderSessionID, resourceManagerLeaderId); + return Futures.failed(new LeaderSessionIDException(leaderSessionID, resourceManagerLeaderId)); + } + + Future jobMasterFuture = getRpcService().connect(jobMasterAddress, JobMasterGateway.class); +>>>>>>> db98efb... rsourceManager registration with JobManager return jobMasterFuture.thenApplyAsync(new ApplyFunction() { @Override public RegistrationResponse apply(JobMasterGateway jobMasterGateway) { +<<<<<<< HEAD final JobMasterGateway existingGateway = jobMasterGateways.put(jobID, jobMasterGateway); if (existingGateway != null) { LOG.info("Replacing existing gateway {} for JobID {} with {}.", existingGateway, jobID, jobMasterGateway); } return new RegistrationResponse(true); +======= + if (jobMasters.containsKey(jobID)) { + JobMasterRegistration jobMasterRegistration = new JobMasterRegistration(jobMasterGateway, jobMasters.get(jobID).getJobMasterLeaderSessionID()); + jobMasters.put(jobID, jobMasterRegistration); + log.info("Replacing gateway for registered JobID {}.", jobID); + } else { + JobMasterRegistration jobMasterRegistration = new JobMasterRegistration(jobMasterGateway); + jobMasters.put(jobID, jobMasterRegistration); + try { + highAvailabilityServices.getJobMasterLeaderRetriever(jobID).start(new JobMasterLeaderListener(jobID)); + } catch(Throwable e) { + log.warn("Decline registration from JobMaster {} at ({}) because fail to get the leader retriever for the given job JobMaster", + jobID, jobMasterAddress); + return new RegistrationResponse.Decline("Fail to get the leader retriever for the given JobMaster"); + } + } + + return new JobMasterRegistrationSuccess(5000); +>>>>>>> db98efb... rsourceManager registration with JobManager } }, getMainThreadExecutor()); } @@ -158,26 +213,41 @@ public SlotRequestReply requestSlot(SlotRequest slotRequest) { /** - * - * @param resourceManagerLeaderId The fencing token for the ResourceManager leader - * @param taskExecutorAddress The address of the TaskExecutor that registers - * @param resourceID The resource ID of the TaskExecutor that registers - * + * @param resourceManagerLeaderId The fencing token for the ResourceManager leader + * @param taskExecutorAddress The address of the TaskExecutor that registers + * @param resourceID The resource ID of the TaskExecutor that registers * @return The response by the ResourceManager. */ @RpcMethod - public org.apache.flink.runtime.registration.RegistrationResponse registerTaskExecutor( - UUID resourceManagerLeaderId, - String taskExecutorAddress, - ResourceID resourceID) { + public RegistrationResponse registerTaskExecutor( + UUID resourceManagerLeaderId, + String taskExecutorAddress, + ResourceID resourceID) { return new TaskExecutorRegistrationSuccess(new InstanceID(), 5000); } +<<<<<<< HEAD // ------------------------------------------------------------------------ // Leader Contender // ------------------------------------------------------------------------ +======= + /** + * Callback method when current resourceManager lose leadership. + */ + @Override + public void revokeLeadership() { + runAsync(new Runnable() { + @Override + public void run() { + log.info("ResourceManager {} was revoked leadership.", getAddress()); + jobMasters.clear(); + leaderSessionID = null; + } + }); + } +>>>>>>> db98efb... rsourceManager registration with JobManager /** * Callback method when current resourceManager is granted leadership @@ -232,4 +302,35 @@ public void run() { } }); } + + private class JobMasterLeaderListener implements LeaderRetrievalListener { + private final JobID jobID; + + private JobMasterLeaderListener(JobID jobID) { + this.jobID = jobID; + } + + @Override + public void notifyLeaderAddress(final String leaderAddress, final UUID leaderSessionID) { + runAsync(new Runnable() { + @Override + public void run() { + log.info("A new leader for JobMaster {} is elected, address is {}, leaderSessionID is {}", jobID, leaderAddress, leaderSessionID); + // update job master leader session id + JobMasterRegistration jobMasterRegistration = jobMasters.get(jobID); + jobMasterRegistration.setJobMasterLeaderSessionID(leaderSessionID); + } + }); + } + + @Override + public void handleError(final Exception exception) { + runAsync(new Runnable() { + @Override + public void run() { + log.error("JobMasterLeaderListener received an error from the LeaderRetrievalService", exception); + } + }); + } + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java index 5c8786cf0e995..1ee11a13d5da7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java @@ -18,12 +18,13 @@ package org.apache.flink.runtime.resourcemanager; -import org.apache.flink.api.common.time.Time; +import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcTimeout; import org.apache.flink.runtime.jobmaster.JobMaster; +import org.apache.flink.runtime.registration.RegistrationResponse; import java.util.UUID; @@ -35,21 +36,18 @@ public interface ResourceManagerGateway extends RpcGateway { /** * Register a {@link JobMaster} at the resource manager. * - * @param jobMasterRegistration Job master registration information - * @param timeout Timeout for the future to complete + * @param resourceManagerLeaderId The fencing token for the ResourceManager leader + * @param jobMasterAddress The address of the JobMaster that registers + * @param jobID The Job ID of the JobMaster that registers + * @param timeout Timeout for the future to complete * @return Future registration response */ Future registerJobMaster( - JobMasterRegistration jobMasterRegistration, - @RpcTimeout Time timeout); + UUID resourceManagerLeaderId, + String jobMasterAddress, + JobID jobID, + @RpcTimeout Time timeout); - /** - * Register a {@link JobMaster} at the resource manager. - * - * @param jobMasterRegistration Job master registration information - * @return Future registration response - */ - Future registerJobMaster(JobMasterRegistration jobMasterRegistration); /** * Requests a slot from the resource manager. @@ -60,15 +58,13 @@ Future registerJobMaster( Future requestSlot(SlotRequest slotRequest); /** - * - * @param resourceManagerLeaderId The fencing token for the ResourceManager leader - * @param taskExecutorAddress The address of the TaskExecutor that registers - * @param resourceID The resource ID of the TaskExecutor that registers - * @param timeout The timeout for the response. - * + * @param resourceManagerLeaderId The fencing token for the ResourceManager leader + * @param taskExecutorAddress The address of the TaskExecutor that registers + * @param resourceID The resource ID of the TaskExecutor that registers + * @param timeout The timeout for the response. * @return The future to the response by the ResourceManager. */ - Future registerTaskExecutor( + Future registerTaskExecutor( UUID resourceManagerLeaderId, String taskExecutorAddress, ResourceID resourceID, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/LeaderSessionIDException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/LeaderSessionIDException.java new file mode 100644 index 0000000000000..cd14a0dc3c329 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/LeaderSessionIDException.java @@ -0,0 +1,60 @@ +/* + * 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.rpc.exceptions; + +import java.util.UUID; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * An exception specifying that the received leader session ID is not the same as expected. + */ +public class LeaderSessionIDException extends Exception { + + private static final long serialVersionUID = -3276145308053264636L; + + /** expected leader session id */ + private final UUID expectedLeaderSessionID; + + /** actual leader session id */ + private final UUID actualLeaderSessionID; + + public LeaderSessionIDException(UUID expectedLeaderSessionID, UUID actualLeaderSessionID) { + super("Unmatched leader session ID : expected " + expectedLeaderSessionID + ", actual " + actualLeaderSessionID); + this.expectedLeaderSessionID = checkNotNull(expectedLeaderSessionID); + this.actualLeaderSessionID = checkNotNull(actualLeaderSessionID); + } + + /** + * Get expected leader session id + * + * @return expect leader session id + */ + public UUID getExpectedLeaderSessionID() { + return expectedLeaderSessionID; + } + + /** + * Get actual leader session id + * + * @return actual leader session id + */ + public UUID getActualLeaderSessionID() { + return actualLeaderSessionID; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java index d84a6a91c8d23..cf709c861df77 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java @@ -326,6 +326,11 @@ public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Excepti return LeaderRetrievalUtils.createLeaderRetrievalService(configuration); } + @Override + public LeaderRetrievalService getJobMasterLeaderRetriever(JobID jobID) throws Exception { + return null; + } + @Override public LeaderElectionService getResourceManagerLeaderElectionService() throws Exception { return null; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java index 3162f40391ea5..2ac43be56b9be 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.highavailability; import org.apache.flink.api.common.JobID; +import org.apache.flink.hadoop.shaded.org.jboss.netty.util.internal.ConcurrentHashMap; import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; @@ -30,6 +31,8 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices private volatile LeaderRetrievalService resourceManagerLeaderRetriever; + private ConcurrentHashMap jobMasterLeaderRetrievers = new ConcurrentHashMap<>(); + private volatile LeaderElectionService jobMasterLeaderElectionService; private volatile LeaderElectionService resourceManagerLeaderElectionService; @@ -43,6 +46,10 @@ public void setResourceManagerLeaderRetriever(LeaderRetrievalService resourceMan this.resourceManagerLeaderRetriever = resourceManagerLeaderRetriever; } + public void setJobMasterLeaderRetriever(JobID jobID, LeaderRetrievalService jobMasterLeaderRetriever) { + this.jobMasterLeaderRetrievers.put(jobID, jobMasterLeaderRetriever); + } + public void setJobMasterLeaderElectionService(LeaderElectionService leaderElectionService) { this.jobMasterLeaderElectionService = leaderElectionService; } @@ -65,6 +72,16 @@ public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Excepti } } + @Override + public LeaderRetrievalService getJobMasterLeaderRetriever(JobID jobID) throws Exception { + LeaderRetrievalService service = this.jobMasterLeaderRetrievers.get(jobID); + if (service != null) { + return service; + } else { + throw new IllegalStateException("JobMasterLeaderRetriever has not been set"); + } + } + @Override public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception { LeaderElectionService service = jobMasterLeaderElectionService; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java new file mode 100644 index 0000000000000..4d04001a4f27b --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java @@ -0,0 +1,160 @@ +/* + * 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.resourcemanager; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; +import org.apache.flink.runtime.jobmaster.JobMasterGateway; +import org.apache.flink.runtime.jobmaster.JobMasterRegistrationSuccess; +import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; +import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService; +import org.apache.flink.runtime.rpc.TestingSerialRpcService; +import org.apache.flink.runtime.registration.RegistrationResponse; +import org.apache.flink.runtime.rpc.exceptions.LeaderSessionIDException; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import scala.concurrent.Await; +import scala.concurrent.Future; +import scala.concurrent.duration.FiniteDuration; + +import java.util.UUID; +import java.util.concurrent.TimeUnit; + +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.*; + +public class ResourceManagerTest { + + private TestingSerialRpcService rpcService; + + @Before + public void setup() throws Exception { + rpcService = new TestingSerialRpcService(); + } + + @After + public void teardown() throws Exception { + rpcService.stopService(); + } + + /** + * Test receive normal registration from job master and receive duplicate registration from job master + * + * @throws Exception + */ + @Test + public void testRegisterJobMaster() throws Exception { + String jobMasterAddress = "/jobMasterAddress1"; + JobID jobID = mockJobMaster(jobMasterAddress); + TestingLeaderElectionService resourceManagerLeaderElectionService = new TestingLeaderElectionService(); + TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService(); + final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService); + final UUID leaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); + + // test response successful + Future successfulFuture = resourceManager.registerJobMaster(leaderSessionId, jobMasterAddress, jobID); + RegistrationResponse response = Await.result(successfulFuture, new FiniteDuration(0, TimeUnit.SECONDS)); + assertTrue(response instanceof JobMasterRegistrationSuccess); + } + + /** + * Test receive registration with unmatched leadershipId from job master + * + * @throws Exception + */ + @Test(expected = LeaderSessionIDException.class) + public void testRegisterJobMasterWithUnmatchedLeaderSessionId() throws Exception { + String jobMasterAddress = "/jobMasterAddress1"; + JobID jobID = mockJobMaster(jobMasterAddress); + TestingLeaderElectionService resourceManagerLeaderElectionService = new TestingLeaderElectionService(); + TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService(); + final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService); + final UUID leaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); + + // test throw exception when receive a registration from job master which takes unmatched leaderSessionId + UUID differentLeaderSessionID = UUID.randomUUID(); + Future unMatchedLeaderFuture = resourceManager.registerJobMaster(differentLeaderSessionID, jobMasterAddress, jobID); + Await.result(unMatchedLeaderFuture, new FiniteDuration(200, TimeUnit.MILLISECONDS)); + } + + /** + * Test receive registration with invalid address from job master + * + * @throws Exception + */ + @Test(expected = Exception.class) + public void testRegisterJobMasterFromInvalidAddress() throws Exception { + String jobMasterAddress = "/jobMasterAddress1"; + JobID jobID = mockJobMaster(jobMasterAddress); + TestingLeaderElectionService resourceManagerLeaderElectionService = new TestingLeaderElectionService(); + TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService(); + final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService); + final UUID leaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); + + // test throw exception when receive a registration from job master which takes invalid address + String invalidAddress = "/jobMasterAddress2"; + Future invalidAddressFuture = resourceManager.registerJobMaster(leaderSessionId, invalidAddress, jobID); + Await.result(invalidAddressFuture, new FiniteDuration(200, TimeUnit.MILLISECONDS)); + } + + /** + * Check and verify return RegistrationResponse.Decline when failed to start a job master Leader retrieval listener + * + * @throws Exception + */ + @Test + public void testRegisterJobMasterWithFailureLeaderListener() throws Exception { + String jobMasterAddress = "/jobMasterAddress1"; + JobID jobID = mockJobMaster(jobMasterAddress); + TestingLeaderElectionService resourceManagerLeaderElectionService = new TestingLeaderElectionService(); + TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService(); + final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService); + final UUID leaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); + + JobID unknownJobIDToHAServices = new JobID(); + // verify return RegistrationResponse.Decline when failed to start a job master Leader retrieval listener + Future declineFuture = resourceManager.registerJobMaster(leaderSessionId, jobMasterAddress, unknownJobIDToHAServices); + RegistrationResponse response = Await.result(declineFuture, new FiniteDuration(0, TimeUnit.SECONDS)); + assertTrue(response instanceof RegistrationResponse.Decline); + } + + private JobID mockJobMaster(String jobMasterAddress) { + JobID jobID = new JobID(); + JobMasterGateway jobMasterGateway = mock(JobMasterGateway.class); + rpcService.registerGateway(jobMasterAddress, jobMasterGateway); + return jobID; + } + + private ResourceManager createAndStartResourceManager(TestingLeaderElectionService resourceManagerLeaderElectionService, JobID jobID, TestingLeaderRetrievalService jobMasterLeaderRetrievalService) { + TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices(); + highAvailabilityServices.setResourceManagerLeaderElectionService(resourceManagerLeaderElectionService); + highAvailabilityServices.setJobMasterLeaderRetriever(jobID, jobMasterLeaderRetrievalService); + ResourceManager resourceManager = new ResourceManager(rpcService, highAvailabilityServices); + resourceManager.start(); + return resourceManager; + } + + private UUID grantResourceManagerLeadership(TestingLeaderElectionService resourceManagerLeaderElectionService) { + UUID leaderSessionId = UUID.randomUUID(); + resourceManagerLeaderElectionService.isLeader(leaderSessionId); + return leaderSessionId; + } + +} From 2fbfe071415fe08c378b6dbea64693fc67194ec4 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Wed, 21 Sep 2016 14:13:12 +0200 Subject: [PATCH 33/50] [FLINK-4537] rebase and refine --- .../JobMasterRegistration.java | 52 +++--- .../resourcemanager/ResourceManager.java | 165 ++++++++---------- .../slotmanager/SlotManager.java | 29 ++- 3 files changed, 110 insertions(+), 136 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java index 7b8ec707eb943..981441fd8ea94 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java @@ -18,59 +18,47 @@ package org.apache.flink.runtime.resourcemanager; -<<<<<<< HEAD import org.apache.flink.api.common.JobID; -======= import org.apache.flink.runtime.jobmaster.JobMasterGateway; ->>>>>>> db98efb... rsourceManager registration with JobManager +import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener; -import java.io.Serializable; import java.util.UUID; -import static org.apache.flink.util.Preconditions.checkNotNull; - /** * This class is responsible for group the JobMasterGateway and the LeaderSessionID of a registered job master */ -public class JobMasterRegistration implements Serializable { +public class JobMasterRegistration implements LeaderRetrievalListener { -<<<<<<< HEAD - private final String address; + private final JobMasterGateway gateway; private final JobID jobID; + private final UUID leaderSessionID; + private LeaderRetrievalListener retriever; - public JobMasterRegistration(String address, JobID jobID) { - this.address = address; + public JobMasterRegistration(JobMasterGateway gateway, JobID jobID, UUID leaderSessionID) { + this.gateway = gateway; this.jobID = jobID; -======= - private static final long serialVersionUID = -2316627821716999527L; - - private final JobMasterGateway jobMasterGateway; - - private UUID jobMasterLeaderSessionID; - - public JobMasterRegistration(JobMasterGateway jobMasterGateway) { - this.jobMasterGateway = checkNotNull(jobMasterGateway); + this.leaderSessionID = leaderSessionID; } - public JobMasterRegistration(JobMasterGateway jobMasterGateway, UUID jobMasterLeaderSessionID) { - this.jobMasterGateway = checkNotNull(jobMasterGateway); - this.jobMasterLeaderSessionID = jobMasterLeaderSessionID; + public JobMasterGateway getGateway() { + return gateway; } - public JobMasterGateway getJobMasterGateway() { - return jobMasterGateway; + public UUID getLeaderSessionID() { + return leaderSessionID; } - public void setJobMasterLeaderSessionID(UUID leaderSessionID) { - this.jobMasterLeaderSessionID = jobMasterLeaderSessionID; ->>>>>>> db98efb... rsourceManager registration with JobManager + public JobID getJobID() { + return jobID; } - public UUID getJobMasterLeaderSessionID() { - return jobMasterLeaderSessionID; + @Override + public void notifyLeaderAddress(String leaderAddress, UUID leaderSessionID) { + } - public JobID getJobID() { - return jobID; + @Override + public void handleError(Exception exception) { + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java index 8be145555cbc1..aae487464855e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.resourcemanager; -import akka.dispatch.Futures; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.clusterframework.types.ResourceID; @@ -29,26 +28,31 @@ import org.apache.flink.runtime.jobmaster.JobMasterRegistrationSuccess; import org.apache.flink.runtime.leaderelection.LeaderContender; import org.apache.flink.runtime.leaderelection.LeaderElectionService; -<<<<<<< HEAD -import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; -======= import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener; ->>>>>>> db98efb... rsourceManager registration with JobManager +import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; +import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; import org.apache.flink.runtime.rpc.RpcMethod; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.jobmaster.JobMaster; import org.apache.flink.runtime.jobmaster.JobMasterGateway; -import org.apache.flink.runtime.rpc.exceptions.LeaderSessionIDException; import org.apache.flink.runtime.taskexecutor.TaskExecutorRegistrationSuccess; import org.apache.flink.runtime.registration.RegistrationResponse; +import org.apache.flink.runtime.concurrent.Future; + +import org.apache.flink.runtime.util.LeaderConnectionInfo; +import org.apache.flink.runtime.util.LeaderRetrievalUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import scala.concurrent.duration.FiniteDuration; import java.util.HashMap; +import java.util.HashSet; import java.util.Map; +import java.util.Set; import java.util.UUID; +import java.util.concurrent.TimeUnit; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -62,17 +66,13 @@ *
  • {@link #requestSlot(SlotRequest)} requests a slot from the resource manager
  • * */ -<<<<<<< HEAD public class ResourceManager extends RpcEndpoint implements LeaderContender { private final Logger LOG = LoggerFactory.getLogger(getClass()); private final Map jobMasterGateways; -======= -public class ResourceManager extends RpcEndpoint { - /** the mapping relationship of JobID and JobMasterGateway */ - private final Map jobMasters; ->>>>>>> db98efb... rsourceManager registration with JobManager + + private final Set jobMasterLeaderRetrievalListeners; private final HighAvailabilityServices highAvailabilityServices; @@ -88,12 +88,9 @@ public ResourceManager( SlotManager slotManager) { super(rpcService); this.highAvailabilityServices = checkNotNull(highAvailabilityServices); -<<<<<<< HEAD this.jobMasterGateways = new HashMap<>(); this.slotManager = slotManager; -======= - this.jobMasters = new HashMap<>(16); ->>>>>>> db98efb... rsourceManager registration with JobManager + this.jobMasterLeaderRetrievalListeners = new HashSet<>(); } @Override @@ -113,7 +110,7 @@ public void start() { public void shutDown() { try { leaderElectionService.stop(); - for(JobID jobID : jobMasters.keySet()) { + for(JobID jobID : jobMasterGateways.keySet()) { highAvailabilityServices.getJobMasterLeaderRetriever(jobID).stop(); } super.shutDown(); @@ -142,52 +139,64 @@ UUID getLeaderSessionID() { * @return Future registration response */ @RpcMethod -<<<<<<< HEAD - public Future registerJobMaster(JobMasterRegistration jobMasterRegistration) { - final Future jobMasterFuture = - getRpcService().connect(jobMasterRegistration.getAddress(), JobMasterGateway.class); - final JobID jobID = jobMasterRegistration.getJobID(); -======= - public Future registerJobMaster(UUID resourceManagerLeaderId, final String jobMasterAddress, final JobID jobID) { + public Future registerJobMaster( + final UUID resourceManagerLeaderId, final UUID jobMasterLeaderId, + final String jobMasterAddress, final JobID jobID) { + + checkNotNull(resourceManagerLeaderId); + checkNotNull(jobMasterAddress); + checkNotNull(jobID); + + // TODO mxm The leader retrieval needs to be split up in an async part which runs outside the main execution thread + // The state updates should be performed inside the main thread + + final FlinkCompletableFuture future = new FlinkCompletableFuture<>(); if(!leaderSessionID.equals(resourceManagerLeaderId)) { - log.warn("Discard registration from JobMaster {} at ({}) because the expected leader session ID {} did not equal the received leader session ID {}", + log.warn("Discard registration from JobMaster {} at ({}) because the expected leader session ID {}" + + " did not equal the received leader session ID {}", jobID, jobMasterAddress, leaderSessionID, resourceManagerLeaderId); - return Futures.failed(new LeaderSessionIDException(leaderSessionID, resourceManagerLeaderId)); + future.complete(new RegistrationResponse.Decline("Invalid leader session id")); + return future; } - Future jobMasterFuture = getRpcService().connect(jobMasterAddress, JobMasterGateway.class); ->>>>>>> db98efb... rsourceManager registration with JobManager + final LeaderConnectionInfo jobMasterLeaderInfo; + try { + jobMasterLeaderInfo = LeaderRetrievalUtils.retrieveLeaderConnectionInfo( + highAvailabilityServices.getJobMasterLeaderRetriever(jobID), new FiniteDuration(5, TimeUnit.SECONDS)); + } catch (Exception e) { + LOG.warn("Failed to start JobMasterLeaderRetriever for JobID {}", jobID); + future.complete(new RegistrationResponse.Decline("Failed to retrieve JobMasterLeaderRetriever")); + return future; + } + + if (!jobMasterLeaderId.equals(jobMasterLeaderInfo.getLeaderSessionID())) { + LOG.info("Declining registration request from non-leading JobManager {}", jobMasterAddress); + future.complete(new RegistrationResponse.Decline("JobManager is not leading")); + return future; + } - return jobMasterFuture.thenApplyAsync(new ApplyFunction() { + Future jobMasterGatewayFuture = + getRpcService().connect(jobMasterAddress, JobMasterGateway.class); + + return jobMasterGatewayFuture.thenApplyAsync(new ApplyFunction() { @Override public RegistrationResponse apply(JobMasterGateway jobMasterGateway) { -<<<<<<< HEAD + + final JobMasterLeaderListener jobMasterLeaderListener = new JobMasterLeaderListener(jobID); + try { + LeaderRetrievalService jobMasterLeaderRetriever = highAvailabilityServices.getJobMasterLeaderRetriever(jobID); + jobMasterLeaderRetriever.start(jobMasterLeaderListener); + } catch (Exception e) { + LOG.warn("Failed to start JobMasterLeaderRetriever for JobID {}", jobID); + return new RegistrationResponse.Decline("Failed to retrieve JobMasterLeaderRetriever"); + } + jobMasterLeaderRetrievalListeners.add(jobMasterLeaderListener); final JobMasterGateway existingGateway = jobMasterGateways.put(jobID, jobMasterGateway); if (existingGateway != null) { - LOG.info("Replacing existing gateway {} for JobID {} with {}.", - existingGateway, jobID, jobMasterGateway); - } - return new RegistrationResponse(true); -======= - if (jobMasters.containsKey(jobID)) { - JobMasterRegistration jobMasterRegistration = new JobMasterRegistration(jobMasterGateway, jobMasters.get(jobID).getJobMasterLeaderSessionID()); - jobMasters.put(jobID, jobMasterRegistration); log.info("Replacing gateway for registered JobID {}.", jobID); - } else { - JobMasterRegistration jobMasterRegistration = new JobMasterRegistration(jobMasterGateway); - jobMasters.put(jobID, jobMasterRegistration); - try { - highAvailabilityServices.getJobMasterLeaderRetriever(jobID).start(new JobMasterLeaderListener(jobID)); - } catch(Throwable e) { - log.warn("Decline registration from JobMaster {} at ({}) because fail to get the leader retriever for the given job JobMaster", - jobID, jobMasterAddress); - return new RegistrationResponse.Decline("Fail to get the leader retriever for the given JobMaster"); - } } - return new JobMasterRegistrationSuccess(5000); ->>>>>>> db98efb... rsourceManager registration with JobManager } }, getMainThreadExecutor()); } @@ -228,26 +237,9 @@ public RegistrationResponse registerTaskExecutor( } -<<<<<<< HEAD // ------------------------------------------------------------------------ // Leader Contender // ------------------------------------------------------------------------ -======= - /** - * Callback method when current resourceManager lose leadership. - */ - @Override - public void revokeLeadership() { - runAsync(new Runnable() { - @Override - public void run() { - log.info("ResourceManager {} was revoked leadership.", getAddress()); - jobMasters.clear(); - leaderSessionID = null; - } - }); - } ->>>>>>> db98efb... rsourceManager registration with JobManager /** * Callback method when current resourceManager is granted leadership @@ -263,7 +255,7 @@ public void run() { // confirming the leader session ID might be blocking, leaderElectionService.confirmLeaderSessionID(leaderSessionID); // notify SlotManager - slotManager.notifyLeaderAddress(getAddress(), leaderSessionID); + slotManager.setLeaderUUID(leaderSessionID); ResourceManager.this.leaderSessionID = leaderSessionID; } }); @@ -279,7 +271,8 @@ public void revokeLeadership() { public void run() { log.info("ResourceManager {} was revoked leadership.", getAddress()); jobMasterGateways.clear(); - ResourceManager.this.leaderSessionID = null; + slotManager.clearState(); + leaderSessionID = null; } }); } @@ -291,20 +284,15 @@ public void run() { */ @Override public void handleError(final Exception exception) { - runAsync(new Runnable() { - @Override - public void run() { - log.error("ResourceManager received an error from the LeaderElectionService.", exception); - // notify SlotManager - slotManager.handleError(exception); - // terminate ResourceManager in case of an error - shutDown(); - } - }); + log.error("ResourceManager received an error from the LeaderElectionService.", exception); + // terminate ResourceManager in case of an error + shutDown(); } - private class JobMasterLeaderListener implements LeaderRetrievalListener { + private static class JobMasterLeaderListener implements LeaderRetrievalListener { + private final JobID jobID; + private UUID leaderID; private JobMasterLeaderListener(JobID jobID) { this.jobID = jobID; @@ -312,25 +300,12 @@ private JobMasterLeaderListener(JobID jobID) { @Override public void notifyLeaderAddress(final String leaderAddress, final UUID leaderSessionID) { - runAsync(new Runnable() { - @Override - public void run() { - log.info("A new leader for JobMaster {} is elected, address is {}, leaderSessionID is {}", jobID, leaderAddress, leaderSessionID); - // update job master leader session id - JobMasterRegistration jobMasterRegistration = jobMasters.get(jobID); - jobMasterRegistration.setJobMasterLeaderSessionID(leaderSessionID); - } - }); + this.leaderID = leaderSessionID; } @Override public void handleError(final Exception exception) { - runAsync(new Runnable() { - @Override - public void run() { - log.error("JobMasterLeaderListener received an error from the LeaderRetrievalService", exception); - } - }); + // TODO } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java index 97176b25b32ed..5d0013ce5a2b6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java @@ -59,7 +59,7 @@ * * IMPORTANT: This class is Not Thread-safe. */ -public abstract class SlotManager implements LeaderRetrievalListener { +public abstract class SlotManager { protected final Logger LOG = LoggerFactory.getLogger(getClass()); @@ -514,22 +514,33 @@ SlotID getSlotID(final AllocationID allocationId) { public int size() { return allocatedSlots.size(); } + + public void clear() { + allocatedSlots.clear(); + allocatedSlotsByAllocationId.clear(); + } + } + + /** + * Clears the state of the SlotManager after leadership revokal + */ + public void clearState() { + taskManagerGateways.clear(); + registeredSlots.clear(); + pendingSlotRequests.clear(); + freeSlots.clear(); + allocationMap.clear(); + leaderID = null; } // ------------------------------------------------------------------------ - // High availability + // High availability (called by the ResourceManager) // ------------------------------------------------------------------------ - @Override - public void notifyLeaderAddress(String leaderAddress, UUID leaderSessionID) { + public void setLeaderUUID(UUID leaderSessionID) { this.leaderID = leaderSessionID; } - @Override - public void handleError(Exception exception) { - LOG.error("Slot Manager received an error from the leader service", exception); - } - // ------------------------------------------------------------------------ // Testing utilities // ------------------------------------------------------------------------ From 0e4957f8eeb14d0cee1c08442bd03b1c8a995e82 Mon Sep 17 00:00:00 2001 From: beyond1920 Date: Thu, 1 Sep 2016 11:14:00 +0800 Subject: [PATCH 34/50] [FLINK-4535] [cluster management] resourceManager process the registration from TaskExecutor --- .../resourcemanager/ResourceManager.java | 288 +++++++----------- .../ResourceManagerGateway.java | 45 +-- ...nse.java => TaskExecutorRegistration.java} | 29 +- .../exceptions/LeaderSessionIDException.java | 1 + .../resourcemanager/ResourceManagerTest.java | 119 +++----- 5 files changed, 212 insertions(+), 270 deletions(-) rename flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/{RegistrationResponse.java => TaskExecutorRegistration.java} (53%) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java index aae487464855e..15692b6b46571 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java @@ -18,41 +18,29 @@ package org.apache.flink.runtime.resourcemanager; +import akka.dispatch.Futures; +import akka.dispatch.Mapper; + import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.clusterframework.types.ResourceID; -import org.apache.flink.runtime.concurrent.ApplyFunction; -import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.instance.InstanceID; -import org.apache.flink.runtime.jobmaster.JobMasterRegistrationSuccess; import org.apache.flink.runtime.leaderelection.LeaderContender; import org.apache.flink.runtime.leaderelection.LeaderElectionService; -import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener; -import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; -import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; import org.apache.flink.runtime.rpc.RpcMethod; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.jobmaster.JobMaster; import org.apache.flink.runtime.jobmaster.JobMasterGateway; +import org.apache.flink.runtime.rpc.exceptions.LeaderSessionIDException; +import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; import org.apache.flink.runtime.taskexecutor.TaskExecutorRegistrationSuccess; import org.apache.flink.runtime.registration.RegistrationResponse; - -import org.apache.flink.runtime.concurrent.Future; - -import org.apache.flink.runtime.util.LeaderConnectionInfo; -import org.apache.flink.runtime.util.LeaderRetrievalUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import scala.concurrent.duration.FiniteDuration; +import scala.concurrent.Future; import java.util.HashMap; -import java.util.HashSet; import java.util.Map; -import java.util.Set; import java.util.UUID; -import java.util.concurrent.TimeUnit; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -62,35 +50,25 @@ * * It offers the following methods as part of its rpc interface to interact with the him remotely: *
      - *
    • {@link #registerJobMaster(UUID, String, JobID)} registers a {@link JobMaster} at the resource manager
    • + *
    • {@link #registerJobMaster(JobMasterRegistration)} registers a {@link JobMaster} at the resource manager
    • *
    • {@link #requestSlot(SlotRequest)} requests a slot from the resource manager
    • *
    */ -public class ResourceManager extends RpcEndpoint implements LeaderContender { - - private final Logger LOG = LoggerFactory.getLogger(getClass()); +public class ResourceManager extends RpcEndpoint { + private final Map jobMasterGateways; - private final Map jobMasterGateways; - - private final Set jobMasterLeaderRetrievalListeners; + /** ResourceID and TaskExecutorRegistration mapping relationship of registered taskExecutors */ + private final Map startedTaskExecutorGateways; private final HighAvailabilityServices highAvailabilityServices; + private LeaderElectionService leaderElectionService = null; + private UUID leaderSessionID = null; - private LeaderElectionService leaderElectionService; - - private final SlotManager slotManager; - - private UUID leaderSessionID; - - public ResourceManager( - RpcService rpcService, - HighAvailabilityServices highAvailabilityServices, - SlotManager slotManager) { + public ResourceManager(RpcService rpcService, HighAvailabilityServices highAvailabilityServices) { super(rpcService); this.highAvailabilityServices = checkNotNull(highAvailabilityServices); - this.jobMasterGateways = new HashMap<>(); - this.slotManager = slotManager; - this.jobMasterLeaderRetrievalListeners = new HashSet<>(); + this.jobMasterGateways = new HashMap<>(16); + this.startedTaskExecutorGateways = new HashMap<>(16); } @Override @@ -99,7 +77,7 @@ public void start() { try { super.start(); leaderElectionService = highAvailabilityServices.getResourceManagerLeaderElectionService(); - leaderElectionService.start(this); + leaderElectionService.start(new ResourceManagerLeaderContender()); } catch (Throwable e) { log.error("A fatal error happened when starting the ResourceManager", e); throw new RuntimeException("A fatal error happened when starting the ResourceManager", e); @@ -110,11 +88,8 @@ public void start() { public void shutDown() { try { leaderElectionService.stop(); - for(JobID jobID : jobMasterGateways.keySet()) { - highAvailabilityServices.getJobMasterLeaderRetriever(jobID).stop(); - } super.shutDown(); - } catch (Throwable e) { + } catch(Throwable e) { log.error("A fatal error happened when shutdown the ResourceManager", e); throw new RuntimeException("A fatal error happened when shutdown the ResourceManager", e); } @@ -127,78 +102,34 @@ public void shutDown() { */ @VisibleForTesting UUID getLeaderSessionID() { - return this.leaderSessionID; + return leaderSessionID; } /** * Register a {@link JobMaster} at the resource manager. * - * @param resourceManagerLeaderId The fencing token for the ResourceManager leader - * @param jobMasterAddress The address of the JobMaster that registers - * @param jobID The Job ID of the JobMaster that registers + * @param jobMasterRegistration Job master registration information * @return Future registration response */ @RpcMethod - public Future registerJobMaster( - final UUID resourceManagerLeaderId, final UUID jobMasterLeaderId, - final String jobMasterAddress, final JobID jobID) { - - checkNotNull(resourceManagerLeaderId); - checkNotNull(jobMasterAddress); - checkNotNull(jobID); + public Future registerJobMaster(JobMasterRegistration jobMasterRegistration) { + Future jobMasterFuture = getRpcService().connect(jobMasterRegistration.getAddress(), JobMasterGateway.class); - // TODO mxm The leader retrieval needs to be split up in an async part which runs outside the main execution thread - // The state updates should be performed inside the main thread - - final FlinkCompletableFuture future = new FlinkCompletableFuture<>(); - - if(!leaderSessionID.equals(resourceManagerLeaderId)) { - log.warn("Discard registration from JobMaster {} at ({}) because the expected leader session ID {}" + - " did not equal the received leader session ID {}", - jobID, jobMasterAddress, leaderSessionID, resourceManagerLeaderId); - future.complete(new RegistrationResponse.Decline("Invalid leader session id")); - return future; - } - - final LeaderConnectionInfo jobMasterLeaderInfo; - try { - jobMasterLeaderInfo = LeaderRetrievalUtils.retrieveLeaderConnectionInfo( - highAvailabilityServices.getJobMasterLeaderRetriever(jobID), new FiniteDuration(5, TimeUnit.SECONDS)); - } catch (Exception e) { - LOG.warn("Failed to start JobMasterLeaderRetriever for JobID {}", jobID); - future.complete(new RegistrationResponse.Decline("Failed to retrieve JobMasterLeaderRetriever")); - return future; - } - - if (!jobMasterLeaderId.equals(jobMasterLeaderInfo.getLeaderSessionID())) { - LOG.info("Declining registration request from non-leading JobManager {}", jobMasterAddress); - future.complete(new RegistrationResponse.Decline("JobManager is not leading")); - return future; - } - - Future jobMasterGatewayFuture = - getRpcService().connect(jobMasterAddress, JobMasterGateway.class); - - return jobMasterGatewayFuture.thenApplyAsync(new ApplyFunction() { + return jobMasterFuture.map(new Mapper() { @Override - public RegistrationResponse apply(JobMasterGateway jobMasterGateway) { - - final JobMasterLeaderListener jobMasterLeaderListener = new JobMasterLeaderListener(jobID); - try { - LeaderRetrievalService jobMasterLeaderRetriever = highAvailabilityServices.getJobMasterLeaderRetriever(jobID); - jobMasterLeaderRetriever.start(jobMasterLeaderListener); - } catch (Exception e) { - LOG.warn("Failed to start JobMasterLeaderRetriever for JobID {}", jobID); - return new RegistrationResponse.Decline("Failed to retrieve JobMasterLeaderRetriever"); - } - jobMasterLeaderRetrievalListeners.add(jobMasterLeaderListener); - final JobMasterGateway existingGateway = jobMasterGateways.put(jobID, jobMasterGateway); - if (existingGateway != null) { - log.info("Replacing gateway for registered JobID {}.", jobID); + public RegistrationResponse apply(final JobMasterGateway jobMasterGateway) { + InstanceID instanceID; + + if (jobMasterGateways.containsKey(jobMasterGateway)) { + instanceID = jobMasterGateways.get(jobMasterGateway); + } else { + instanceID = new InstanceID(); + jobMasterGateways.put(jobMasterGateway, instanceID); } - return new JobMasterRegistrationSuccess(5000); + + return new TaskExecutorRegistrationSuccess(instanceID, 5000); } - }, getMainThreadExecutor()); + }, getMainThreadExecutionContext()); } /** @@ -208,104 +139,111 @@ public RegistrationResponse apply(JobMasterGateway jobMasterGateway) { * @return Slot assignment */ @RpcMethod - public SlotRequestReply requestSlot(SlotRequest slotRequest) { - final JobID jobId = slotRequest.getJobId(); - final JobMasterGateway jobMasterGateway = jobMasterGateways.get(jobId); - - if (jobMasterGateway != null) { - return slotManager.requestSlot(slotRequest); - } else { - LOG.info("Ignoring slot request for unknown JobMaster with JobID {}", jobId); - return new SlotRequestRejected(slotRequest.getAllocationId()); - } + public SlotAssignment requestSlot(SlotRequest slotRequest) { + System.out.println("SlotRequest: " + slotRequest); + return new SlotAssignment(); } /** - * @param resourceManagerLeaderId The fencing token for the ResourceManager leader - * @param taskExecutorAddress The address of the TaskExecutor that registers - * @param resourceID The resource ID of the TaskExecutor that registers + * Register a {@link org.apache.flink.runtime.taskexecutor.TaskExecutor} at the resource manager + * + * @param resourceManagerLeaderId The fencing token for the ResourceManager leader + * @param taskExecutorAddress The address of the TaskExecutor that registers + * @param resourceID The resource ID of the TaskExecutor that registers + * * @return The response by the ResourceManager. */ @RpcMethod - public RegistrationResponse registerTaskExecutor( - UUID resourceManagerLeaderId, - String taskExecutorAddress, - ResourceID resourceID) { + public Future registerTaskExecutor( + final UUID resourceManagerLeaderId, + final String taskExecutorAddress, + final ResourceID resourceID) { - return new TaskExecutorRegistrationSuccess(new InstanceID(), 5000); - } + if(!leaderSessionID.equals(resourceManagerLeaderId)) { + log.warn("Discard registration from TaskExecutor {} at ({}) because the expected leader session ID {} did not equal the received leader session ID {}", + resourceID, taskExecutorAddress, leaderSessionID, resourceManagerLeaderId); + return Futures.failed(new LeaderSessionIDException(leaderSessionID, resourceManagerLeaderId)); + } + Future taskExecutorGatewayFuture = getRpcService().connect(taskExecutorAddress, TaskExecutorGateway.class); - // ------------------------------------------------------------------------ - // Leader Contender - // ------------------------------------------------------------------------ + return taskExecutorGatewayFuture.map(new Mapper() { - /** - * Callback method when current resourceManager is granted leadership - * - * @param leaderSessionID unique leadershipID - */ - @Override - public void grantLeadership(final UUID leaderSessionID) { - runAsync(new Runnable() { @Override - public void run() { - log.info("ResourceManager {} was granted leadership with leader session ID {}", getAddress(), leaderSessionID); - // confirming the leader session ID might be blocking, - leaderElectionService.confirmLeaderSessionID(leaderSessionID); - // notify SlotManager - slotManager.setLeaderUUID(leaderSessionID); - ResourceManager.this.leaderSessionID = leaderSessionID; - } - }); - } + public RegistrationResponse apply(final TaskExecutorGateway taskExecutorGateway) { + InstanceID instanceID = null; + TaskExecutorRegistration taskExecutorRegistration = startedTaskExecutorGateways.get(resourceID); + if(taskExecutorRegistration != null) { + log.warn("Receive a duplicate registration from TaskExecutor {} at ({})", resourceID, taskExecutorAddress); + instanceID = taskExecutorRegistration.getInstanceID(); + } else { + instanceID = new InstanceID(); + startedTaskExecutorGateways.put(resourceID, new TaskExecutorRegistration(taskExecutorGateway, instanceID)); + } - /** - * Callback method when current resourceManager lose leadership. - */ - @Override - public void revokeLeadership() { - runAsync(new Runnable() { - @Override - public void run() { - log.info("ResourceManager {} was revoked leadership.", getAddress()); - jobMasterGateways.clear(); - slotManager.clearState(); - leaderSessionID = null; + return new TaskExecutorRegistrationSuccess(instanceID, 5000); } - }); + }, getMainThreadExecutionContext()); } - /** - * Handles error occurring in the leader election service - * - * @param exception Exception being thrown in the leader election service - */ - @Override - public void handleError(final Exception exception) { - log.error("ResourceManager received an error from the LeaderElectionService.", exception); - // terminate ResourceManager in case of an error - shutDown(); - } - private static class JobMasterLeaderListener implements LeaderRetrievalListener { + private class ResourceManagerLeaderContender implements LeaderContender { - private final JobID jobID; - private UUID leaderID; + /** + * Callback method when current resourceManager is granted leadership + * + * @param leaderSessionID unique leadershipID + */ + @Override + public void grantLeadership(final UUID leaderSessionID) { + runAsync(new Runnable() { + @Override + public void run() { + log.info("ResourceManager {} was granted leadership with leader session ID {}", getAddress(), leaderSessionID); + ResourceManager.this.leaderSessionID = leaderSessionID; + // confirming the leader session ID might be blocking, + leaderElectionService.confirmLeaderSessionID(leaderSessionID); + } + }); + } - private JobMasterLeaderListener(JobID jobID) { - this.jobID = jobID; + /** + * Callback method when current resourceManager lose leadership. + */ + @Override + public void revokeLeadership() { + runAsync(new Runnable() { + @Override + public void run() { + log.info("ResourceManager {} was revoked leadership.", getAddress()); + jobMasterGateways.clear(); + startedTaskExecutorGateways.clear(); + leaderSessionID = null; + } + }); } @Override - public void notifyLeaderAddress(final String leaderAddress, final UUID leaderSessionID) { - this.leaderID = leaderSessionID; + public String getAddress() { + return ResourceManager.this.getAddress(); } + /** + * Handles error occurring in the leader election service + * + * @param exception Exception being thrown in the leader election service + */ @Override public void handleError(final Exception exception) { - // TODO + runAsync(new Runnable() { + @Override + public void run() { + log.error("ResourceManager received an error from the LeaderElectionService.", exception); + // terminate ResourceManager in case of an error + shutDown(); + } + }); } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java index 1ee11a13d5da7..30a096febab62 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java @@ -18,13 +18,14 @@ package org.apache.flink.runtime.resourcemanager; -import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.clusterframework.types.ResourceID; -import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcTimeout; import org.apache.flink.runtime.jobmaster.JobMaster; + import org.apache.flink.runtime.registration.RegistrationResponse; +import scala.concurrent.Future; +import scala.concurrent.duration.FiniteDuration; import java.util.UUID; @@ -36,18 +37,21 @@ public interface ResourceManagerGateway extends RpcGateway { /** * Register a {@link JobMaster} at the resource manager. * - * @param resourceManagerLeaderId The fencing token for the ResourceManager leader - * @param jobMasterAddress The address of the JobMaster that registers - * @param jobID The Job ID of the JobMaster that registers - * @param timeout Timeout for the future to complete + * @param jobMasterRegistration Job master registration information + * @param timeout Timeout for the future to complete * @return Future registration response */ Future registerJobMaster( - UUID resourceManagerLeaderId, - String jobMasterAddress, - JobID jobID, - @RpcTimeout Time timeout); + JobMasterRegistration jobMasterRegistration, + @RpcTimeout FiniteDuration timeout); + /** + * Register a {@link JobMaster} at the resource manager. + * + * @param jobMasterRegistration Job master registration information + * @return Future registration response + */ + Future registerJobMaster(JobMasterRegistration jobMasterRegistration); /** * Requests a slot from the resource manager. @@ -55,18 +59,21 @@ Future registerJobMaster( * @param slotRequest Slot request * @return Future slot assignment */ - Future requestSlot(SlotRequest slotRequest); + Future requestSlot(SlotRequest slotRequest); /** - * @param resourceManagerLeaderId The fencing token for the ResourceManager leader - * @param taskExecutorAddress The address of the TaskExecutor that registers - * @param resourceID The resource ID of the TaskExecutor that registers - * @param timeout The timeout for the response. + * Register a {@link org.apache.flink.runtime.taskexecutor.TaskExecutor} at the resource manager. + * + * @param resourceManagerLeaderId The fencing token for the ResourceManager leader + * @param taskExecutorAddress The address of the TaskExecutor that registers + * @param resourceID The resource ID of the TaskExecutor that registers + * @param timeout The timeout for the response. + * * @return The future to the response by the ResourceManager. */ Future registerTaskExecutor( - UUID resourceManagerLeaderId, - String taskExecutorAddress, - ResourceID resourceID, - @RpcTimeout Time timeout); + UUID resourceManagerLeaderId, + String taskExecutorAddress, + ResourceID resourceID, + @RpcTimeout FiniteDuration timeout); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/RegistrationResponse.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/TaskExecutorRegistration.java similarity index 53% rename from flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/RegistrationResponse.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/TaskExecutorRegistration.java index 796e634ed37a4..bd78a475ba77c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/RegistrationResponse.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/TaskExecutorRegistration.java @@ -18,19 +18,34 @@ package org.apache.flink.runtime.resourcemanager; +import org.apache.flink.runtime.instance.InstanceID; +import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; + import java.io.Serializable; -public class RegistrationResponse implements Serializable { - private static final long serialVersionUID = -2379003255993119993L; +/** + * This class is responsible for group the TaskExecutorGateway and the InstanceID of a registered task executor. + */ +public class TaskExecutorRegistration implements Serializable { + + private static final long serialVersionUID = -2062957799469434614L; - private final boolean isSuccess; + private TaskExecutorGateway taskExecutorGateway; + + private InstanceID instanceID; + + public TaskExecutorRegistration(TaskExecutorGateway taskExecutorGateway, + InstanceID instanceID) { + this.taskExecutorGateway = taskExecutorGateway; + this.instanceID = instanceID; + } - public RegistrationResponse(boolean isSuccess) { - this.isSuccess = isSuccess; + public InstanceID getInstanceID() { + return instanceID; } - public boolean isSuccess() { - return isSuccess; + public TaskExecutorGateway getTaskExecutorGateway() { + return taskExecutorGateway; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/LeaderSessionIDException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/LeaderSessionIDException.java index cd14a0dc3c329..d3ba9a97e8576 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/LeaderSessionIDException.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/LeaderSessionIDException.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.rpc.exceptions; import java.util.UUID; + import static org.apache.flink.util.Preconditions.checkNotNull; /** diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java index 4d04001a4f27b..b75d9b8eaad1d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java @@ -18,15 +18,14 @@ package org.apache.flink.runtime.resourcemanager; -import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; -import org.apache.flink.runtime.jobmaster.JobMasterGateway; -import org.apache.flink.runtime.jobmaster.JobMasterRegistrationSuccess; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; -import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService; import org.apache.flink.runtime.rpc.TestingSerialRpcService; import org.apache.flink.runtime.registration.RegistrationResponse; import org.apache.flink.runtime.rpc.exceptions.LeaderSessionIDException; +import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; +import org.apache.flink.runtime.taskexecutor.TaskExecutorRegistrationSuccess; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -37,8 +36,9 @@ import java.util.UUID; import java.util.concurrent.TimeUnit; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import static org.mockito.Mockito.*; +import static org.mockito.Mockito.mock; public class ResourceManagerTest { @@ -55,105 +55,86 @@ public void teardown() throws Exception { } /** - * Test receive normal registration from job master and receive duplicate registration from job master + * Test receive normal registration from task executor and receive duplicate registration from task executor * * @throws Exception */ @Test - public void testRegisterJobMaster() throws Exception { - String jobMasterAddress = "/jobMasterAddress1"; - JobID jobID = mockJobMaster(jobMasterAddress); - TestingLeaderElectionService resourceManagerLeaderElectionService = new TestingLeaderElectionService(); - TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService(); - final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService); - final UUID leaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); + public void testRegisterTaskExecutor() throws Exception { + String taskExecutorAddress = "/taskExecutor1"; + ResourceID taskExecutorResourceID = mockTaskExecutor(taskExecutorAddress); + TestingLeaderElectionService leaderElectionService = new TestingLeaderElectionService(); + final ResourceManager resourceManager = createAndStartResourceManager(leaderElectionService); + final UUID leaderSessionId = grantResourceManagerLeadership(leaderElectionService); // test response successful - Future successfulFuture = resourceManager.registerJobMaster(leaderSessionId, jobMasterAddress, jobID); + Future successfulFuture = resourceManager.registerTaskExecutor(leaderSessionId, taskExecutorAddress, taskExecutorResourceID); RegistrationResponse response = Await.result(successfulFuture, new FiniteDuration(0, TimeUnit.SECONDS)); - assertTrue(response instanceof JobMasterRegistrationSuccess); + assertTrue(response instanceof TaskExecutorRegistrationSuccess); + + // test response successful with previous instanceID when receive duplicate registration from taskExecutor + Future duplicateFuture = resourceManager.registerTaskExecutor(leaderSessionId, taskExecutorAddress, taskExecutorResourceID); + RegistrationResponse duplicateResponse = Await.result(duplicateFuture, new FiniteDuration(0, TimeUnit.SECONDS)); + assertTrue(duplicateResponse instanceof TaskExecutorRegistrationSuccess); + assertEquals(((TaskExecutorRegistrationSuccess) response).getRegistrationId(), ((TaskExecutorRegistrationSuccess) duplicateResponse).getRegistrationId()); } /** - * Test receive registration with unmatched leadershipId from job master + * Test receive registration with unmatched leadershipId from task executor * * @throws Exception */ @Test(expected = LeaderSessionIDException.class) - public void testRegisterJobMasterWithUnmatchedLeaderSessionId() throws Exception { - String jobMasterAddress = "/jobMasterAddress1"; - JobID jobID = mockJobMaster(jobMasterAddress); - TestingLeaderElectionService resourceManagerLeaderElectionService = new TestingLeaderElectionService(); - TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService(); - final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService); - final UUID leaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); - - // test throw exception when receive a registration from job master which takes unmatched leaderSessionId + public void testRegisterTaskExecutorWithUnmatchedLeaderSessionId() throws Exception { + String taskExecutorAddress = "/taskExecutor1"; + ResourceID taskExecutorResourceID = mockTaskExecutor(taskExecutorAddress); + TestingLeaderElectionService leaderElectionService = new TestingLeaderElectionService(); + final ResourceManager resourceManager = createAndStartResourceManager(leaderElectionService); + final UUID leaderSessionId = grantResourceManagerLeadership(leaderElectionService); + + // test throw exception when receive a registration from taskExecutor which takes unmatched leaderSessionId UUID differentLeaderSessionID = UUID.randomUUID(); - Future unMatchedLeaderFuture = resourceManager.registerJobMaster(differentLeaderSessionID, jobMasterAddress, jobID); + Future unMatchedLeaderFuture = resourceManager.registerTaskExecutor(differentLeaderSessionID, taskExecutorAddress, taskExecutorResourceID); Await.result(unMatchedLeaderFuture, new FiniteDuration(200, TimeUnit.MILLISECONDS)); } /** - * Test receive registration with invalid address from job master + * Test receive registration with invalid address from task executor * * @throws Exception */ @Test(expected = Exception.class) - public void testRegisterJobMasterFromInvalidAddress() throws Exception { - String jobMasterAddress = "/jobMasterAddress1"; - JobID jobID = mockJobMaster(jobMasterAddress); - TestingLeaderElectionService resourceManagerLeaderElectionService = new TestingLeaderElectionService(); - TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService(); - final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService); - final UUID leaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); - - // test throw exception when receive a registration from job master which takes invalid address - String invalidAddress = "/jobMasterAddress2"; - Future invalidAddressFuture = resourceManager.registerJobMaster(leaderSessionId, invalidAddress, jobID); + public void testRegisterTaskExecutorFromInvalidAddress() throws Exception { + String taskExecutorAddress = "/taskExecutor1"; + ResourceID taskExecutorResourceID = mockTaskExecutor(taskExecutorAddress); + TestingLeaderElectionService leaderElectionService = new TestingLeaderElectionService(); + final ResourceManager resourceManager = createAndStartResourceManager(leaderElectionService); + final UUID leaderSessionId = grantResourceManagerLeadership(leaderElectionService); + + // test throw exception when receive a registration from taskExecutor which takes invalid address + String invalidAddress = "/taskExecutor2"; + Future invalidAddressFuture = resourceManager.registerTaskExecutor(leaderSessionId, invalidAddress, taskExecutorResourceID); Await.result(invalidAddressFuture, new FiniteDuration(200, TimeUnit.MILLISECONDS)); } - /** - * Check and verify return RegistrationResponse.Decline when failed to start a job master Leader retrieval listener - * - * @throws Exception - */ - @Test - public void testRegisterJobMasterWithFailureLeaderListener() throws Exception { - String jobMasterAddress = "/jobMasterAddress1"; - JobID jobID = mockJobMaster(jobMasterAddress); - TestingLeaderElectionService resourceManagerLeaderElectionService = new TestingLeaderElectionService(); - TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService(); - final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService); - final UUID leaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); - - JobID unknownJobIDToHAServices = new JobID(); - // verify return RegistrationResponse.Decline when failed to start a job master Leader retrieval listener - Future declineFuture = resourceManager.registerJobMaster(leaderSessionId, jobMasterAddress, unknownJobIDToHAServices); - RegistrationResponse response = Await.result(declineFuture, new FiniteDuration(0, TimeUnit.SECONDS)); - assertTrue(response instanceof RegistrationResponse.Decline); - } - - private JobID mockJobMaster(String jobMasterAddress) { - JobID jobID = new JobID(); - JobMasterGateway jobMasterGateway = mock(JobMasterGateway.class); - rpcService.registerGateway(jobMasterAddress, jobMasterGateway); - return jobID; + private ResourceID mockTaskExecutor(String taskExecutorAddress) { + TaskExecutorGateway taskExecutorGateway = mock(TaskExecutorGateway.class); + ResourceID taskExecutorResourceID = ResourceID.generate(); + rpcService.registerGateway(taskExecutorAddress, taskExecutorGateway); + return taskExecutorResourceID; } - private ResourceManager createAndStartResourceManager(TestingLeaderElectionService resourceManagerLeaderElectionService, JobID jobID, TestingLeaderRetrievalService jobMasterLeaderRetrievalService) { + private ResourceManager createAndStartResourceManager(TestingLeaderElectionService leaderElectionService) { TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices(); - highAvailabilityServices.setResourceManagerLeaderElectionService(resourceManagerLeaderElectionService); - highAvailabilityServices.setJobMasterLeaderRetriever(jobID, jobMasterLeaderRetrievalService); + highAvailabilityServices.setResourceManagerLeaderElectionService(leaderElectionService); ResourceManager resourceManager = new ResourceManager(rpcService, highAvailabilityServices); resourceManager.start(); return resourceManager; } - private UUID grantResourceManagerLeadership(TestingLeaderElectionService resourceManagerLeaderElectionService) { + private UUID grantResourceManagerLeadership(TestingLeaderElectionService leaderElectionService) { UUID leaderSessionId = UUID.randomUUID(); - resourceManagerLeaderElectionService.isLeader(leaderSessionId); + leaderElectionService.isLeader(leaderSessionId); return leaderSessionId; } From ceac48314ceeac0b5c0afa6585417aec0391b3ae Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Wed, 21 Sep 2016 20:20:25 +0200 Subject: [PATCH 35/50] [FLINK-4535] rebase and refine --- .../JobMasterRegistration.java | 64 ---- .../resourcemanager/ResourceManager.java | 322 ++++++++++++------ .../ResourceManagerGateway.java | 36 +- .../TaskExecutorRegistration.java | 2 +- .../slotmanager/SlotManager.java | 1 - .../ResourceManagerJobMasterTest.java | 174 ++++++++++ ...a => ResourceManagerTaskExecutorTest.java} | 52 ++- .../slotmanager/SlotProtocolTest.java | 43 ++- 8 files changed, 462 insertions(+), 232 deletions(-) delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java rename flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/{ResourceManagerTest.java => ResourceManagerTaskExecutorTest.java} (74%) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java deleted file mode 100644 index 981441fd8ea94..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java +++ /dev/null @@ -1,64 +0,0 @@ -/* - * 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.resourcemanager; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.runtime.jobmaster.JobMasterGateway; -import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener; - -import java.util.UUID; - -/** - * This class is responsible for group the JobMasterGateway and the LeaderSessionID of a registered job master - */ -public class JobMasterRegistration implements LeaderRetrievalListener { - - private final JobMasterGateway gateway; - private final JobID jobID; - private final UUID leaderSessionID; - private LeaderRetrievalListener retriever; - - public JobMasterRegistration(JobMasterGateway gateway, JobID jobID, UUID leaderSessionID) { - this.gateway = gateway; - this.jobID = jobID; - this.leaderSessionID = leaderSessionID; - } - - public JobMasterGateway getGateway() { - return gateway; - } - - public UUID getLeaderSessionID() { - return leaderSessionID; - } - - public JobID getJobID() { - return jobID; - } - - @Override - public void notifyLeaderAddress(String leaderAddress, UUID leaderSessionID) { - - } - - @Override - public void handleError(Exception exception) { - - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java index 15692b6b46571..88b8a115eb215 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java @@ -18,29 +18,41 @@ package org.apache.flink.runtime.resourcemanager; -import akka.dispatch.Futures; -import akka.dispatch.Mapper; - import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.concurrent.BiFunction; +import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.instance.InstanceID; +import org.apache.flink.runtime.jobmaster.JobMasterRegistrationSuccess; import org.apache.flink.runtime.leaderelection.LeaderContender; import org.apache.flink.runtime.leaderelection.LeaderElectionService; +import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener; +import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; +import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; import org.apache.flink.runtime.rpc.RpcMethod; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.jobmaster.JobMaster; import org.apache.flink.runtime.jobmaster.JobMasterGateway; -import org.apache.flink.runtime.rpc.exceptions.LeaderSessionIDException; +import org.apache.flink.runtime.registration.RegistrationResponse; + import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; import org.apache.flink.runtime.taskexecutor.TaskExecutorRegistrationSuccess; -import org.apache.flink.runtime.registration.RegistrationResponse; -import scala.concurrent.Future; +import org.apache.flink.runtime.util.LeaderConnectionInfo; +import org.apache.flink.runtime.util.LeaderRetrievalUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.concurrent.duration.FiniteDuration; import java.util.HashMap; +import java.util.HashSet; import java.util.Map; +import java.util.Set; import java.util.UUID; +import java.util.concurrent.Callable; +import java.util.concurrent.TimeUnit; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -50,25 +62,38 @@ * * It offers the following methods as part of its rpc interface to interact with the him remotely: *
      - *
    • {@link #registerJobMaster(JobMasterRegistration)} registers a {@link JobMaster} at the resource manager
    • + *
    • {@link #registerJobMaster(UUID, UUID, String, JobID)} registers a {@link JobMaster} at the resource manager
    • *
    • {@link #requestSlot(SlotRequest)} requests a slot from the resource manager
    • *
    */ -public class ResourceManager extends RpcEndpoint { - private final Map jobMasterGateways; +public class ResourceManager extends RpcEndpoint implements LeaderContender { + + private final Logger LOG = LoggerFactory.getLogger(getClass()); - /** ResourceID and TaskExecutorRegistration mapping relationship of registered taskExecutors */ - private final Map startedTaskExecutorGateways; + private final Map jobMasterGateways; + + private final Set jobMasterLeaderRetrievalListeners; + + private final Map taskExecutorGateways; private final HighAvailabilityServices highAvailabilityServices; - private LeaderElectionService leaderElectionService = null; - private UUID leaderSessionID = null; - public ResourceManager(RpcService rpcService, HighAvailabilityServices highAvailabilityServices) { + private LeaderElectionService leaderElectionService; + + private final SlotManager slotManager; + + private UUID leaderSessionID; + + public ResourceManager( + RpcService rpcService, + HighAvailabilityServices highAvailabilityServices, + SlotManager slotManager) { super(rpcService); this.highAvailabilityServices = checkNotNull(highAvailabilityServices); - this.jobMasterGateways = new HashMap<>(16); - this.startedTaskExecutorGateways = new HashMap<>(16); + this.jobMasterGateways = new HashMap<>(); + this.slotManager = slotManager; + this.jobMasterLeaderRetrievalListeners = new HashSet<>(); + this.taskExecutorGateways = new HashMap<>(); } @Override @@ -77,7 +102,7 @@ public void start() { try { super.start(); leaderElectionService = highAvailabilityServices.getResourceManagerLeaderElectionService(); - leaderElectionService.start(new ResourceManagerLeaderContender()); + leaderElectionService.start(this); } catch (Throwable e) { log.error("A fatal error happened when starting the ResourceManager", e); throw new RuntimeException("A fatal error happened when starting the ResourceManager", e); @@ -88,8 +113,11 @@ public void start() { public void shutDown() { try { leaderElectionService.stop(); + for(JobID jobID : jobMasterGateways.keySet()) { + highAvailabilityServices.getJobMasterLeaderRetriever(jobID).stop(); + } super.shutDown(); - } catch(Throwable e) { + } catch (Throwable e) { log.error("A fatal error happened when shutdown the ResourceManager", e); throw new RuntimeException("A fatal error happened when shutdown the ResourceManager", e); } @@ -102,48 +130,79 @@ public void shutDown() { */ @VisibleForTesting UUID getLeaderSessionID() { - return leaderSessionID; + return this.leaderSessionID; } /** * Register a {@link JobMaster} at the resource manager. * - * @param jobMasterRegistration Job master registration information + * @param resourceManagerLeaderId The fencing token for the ResourceManager leader + * @param jobMasterAddress The address of the JobMaster that registers + * @param jobID The Job ID of the JobMaster that registers * @return Future registration response */ @RpcMethod - public Future registerJobMaster(JobMasterRegistration jobMasterRegistration) { - Future jobMasterFuture = getRpcService().connect(jobMasterRegistration.getAddress(), JobMasterGateway.class); + public Future registerJobMaster( + final UUID resourceManagerLeaderId, final UUID jobMasterLeaderId, + final String jobMasterAddress, final JobID jobID) { - return jobMasterFuture.map(new Mapper() { - @Override - public RegistrationResponse apply(final JobMasterGateway jobMasterGateway) { - InstanceID instanceID; + checkNotNull(jobMasterAddress); + checkNotNull(jobID); - if (jobMasterGateways.containsKey(jobMasterGateway)) { - instanceID = jobMasterGateways.get(jobMasterGateway); - } else { - instanceID = new InstanceID(); - jobMasterGateways.put(jobMasterGateway, instanceID); - } + return getRpcService() + .execute(new Callable() { + @Override + public JobMasterGateway call() throws Exception { - return new TaskExecutorRegistrationSuccess(instanceID, 5000); - } - }, getMainThreadExecutionContext()); - } + if (!leaderSessionID.equals(resourceManagerLeaderId)) { + log.warn("Discard registration from JobMaster {} at ({}) because the expected leader session ID {}" + + " did not equal the received leader session ID {}", + jobID, jobMasterAddress, leaderSessionID, resourceManagerLeaderId); + throw new Exception("Invalid leader session id"); + } - /** - * Requests a slot from the resource manager. - * - * @param slotRequest Slot request - * @return Slot assignment - */ - @RpcMethod - public SlotAssignment requestSlot(SlotRequest slotRequest) { - System.out.println("SlotRequest: " + slotRequest); - return new SlotAssignment(); - } + final LeaderConnectionInfo jobMasterLeaderInfo; + try { + jobMasterLeaderInfo = LeaderRetrievalUtils.retrieveLeaderConnectionInfo( + highAvailabilityServices.getJobMasterLeaderRetriever(jobID), new FiniteDuration(5, TimeUnit.SECONDS)); + } catch (Exception e) { + LOG.warn("Failed to start JobMasterLeaderRetriever for JobID {}", jobID); + throw new Exception("Failed to retrieve JobMasterLeaderRetriever"); + } + + if (!jobMasterLeaderId.equals(jobMasterLeaderInfo.getLeaderSessionID())) { + LOG.info("Declining registration request from non-leading JobManager {}", jobMasterAddress); + throw new Exception("JobManager is not leading"); + } + return getRpcService().connect(jobMasterAddress, JobMasterGateway.class).get(5, TimeUnit.SECONDS); + } + }) + .handleAsync(new BiFunction() { + @Override + public RegistrationResponse apply(JobMasterGateway jobMasterGateway, Throwable throwable) { + + if (throwable != null) { + return new RegistrationResponse.Decline(throwable.getMessage()); + } else { + JobMasterLeaderListener jobMasterLeaderListener = new JobMasterLeaderListener(jobID); + try { + LeaderRetrievalService jobMasterLeaderRetriever = highAvailabilityServices.getJobMasterLeaderRetriever(jobID); + jobMasterLeaderRetriever.start(jobMasterLeaderListener); + } catch (Exception e) { + LOG.warn("Failed to start JobMasterLeaderRetriever for JobID {}", jobID); + return new RegistrationResponse.Decline("Failed to retrieve JobMasterLeaderRetriever"); + } + jobMasterLeaderRetrievalListeners.add(jobMasterLeaderListener); + final JobMasterGateway existingGateway = jobMasterGateways.put(jobID, jobMasterGateway); + if (existingGateway != null) { + log.info("Replacing gateway for registered JobID {}.", jobID); + } + return new JobMasterRegistrationSuccess(5000); + } + } + }, getMainThreadExecutor()); + } /** * Register a {@link org.apache.flink.runtime.taskexecutor.TaskExecutor} at the resource manager @@ -160,90 +219,129 @@ public Future registerTaskExecutor( final String taskExecutorAddress, final ResourceID resourceID) { - if(!leaderSessionID.equals(resourceManagerLeaderId)) { - log.warn("Discard registration from TaskExecutor {} at ({}) because the expected leader session ID {} did not equal the received leader session ID {}", - resourceID, taskExecutorAddress, leaderSessionID, resourceManagerLeaderId); - return Futures.failed(new LeaderSessionIDException(leaderSessionID, resourceManagerLeaderId)); + return getRpcService().execute(new Callable() { + @Override + public TaskExecutorGateway call() throws Exception { + if (!leaderSessionID.equals(resourceManagerLeaderId)) { + log.warn("Discard registration from TaskExecutor {} at ({}) because the expected leader session ID {} did " + + "not equal the received leader session ID {}", + resourceID, taskExecutorAddress, leaderSessionID, resourceManagerLeaderId); + throw new Exception("Invalid leader session id"); + } + + return getRpcService().connect(taskExecutorAddress, TaskExecutorGateway.class).get(5, TimeUnit.SECONDS); + } + }).handleAsync(new BiFunction() { + @Override + public RegistrationResponse apply(TaskExecutorGateway taskExecutorGateway, Throwable throwable) { + if (throwable != null) { + return new RegistrationResponse.Decline(throwable.getMessage()); + } else { + InstanceID id = new InstanceID(); + TaskExecutorRegistration oldTaskExecutor = + taskExecutorGateways.put(resourceID, new TaskExecutorRegistration(taskExecutorGateway, id)); + if (oldTaskExecutor != null) { + log.warn("Receive a duplicate registration from TaskExecutor {} at ({})", resourceID, taskExecutorAddress); + } + return new TaskExecutorRegistrationSuccess(id, 5000); + } + } + }, getMainThreadExecutor()); + } + + /** + * Requests a slot from the resource manager. + * + * @param slotRequest Slot request + * @return Slot assignment + */ + @RpcMethod + public SlotRequestReply requestSlot(SlotRequest slotRequest) { + final JobID jobId = slotRequest.getJobId(); + final JobMasterGateway jobMasterGateway = jobMasterGateways.get(jobId); + + if (jobMasterGateway != null) { + return slotManager.requestSlot(slotRequest); + } else { + LOG.info("Ignoring slot request for unknown JobMaster with JobID {}", jobId); + return new SlotRequestRejected(slotRequest.getAllocationId()); } + } - Future taskExecutorGatewayFuture = getRpcService().connect(taskExecutorAddress, TaskExecutorGateway.class); - return taskExecutorGatewayFuture.map(new Mapper() { + + // ------------------------------------------------------------------------ + // Leader Contender + // ------------------------------------------------------------------------ + + /** + * Callback method when current resourceManager is granted leadership + * + * @param leaderSessionID unique leadershipID + */ + @Override + public void grantLeadership(final UUID leaderSessionID) { + runAsync(new Runnable() { @Override - public RegistrationResponse apply(final TaskExecutorGateway taskExecutorGateway) { - InstanceID instanceID = null; - TaskExecutorRegistration taskExecutorRegistration = startedTaskExecutorGateways.get(resourceID); - if(taskExecutorRegistration != null) { - log.warn("Receive a duplicate registration from TaskExecutor {} at ({})", resourceID, taskExecutorAddress); - instanceID = taskExecutorRegistration.getInstanceID(); - } else { - instanceID = new InstanceID(); - startedTaskExecutorGateways.put(resourceID, new TaskExecutorRegistration(taskExecutorGateway, instanceID)); - } + public void run() { + log.info("ResourceManager {} was granted leadership with leader session ID {}", getAddress(), leaderSessionID); + // confirming the leader session ID might be blocking, + leaderElectionService.confirmLeaderSessionID(leaderSessionID); + // notify SlotManager + slotManager.setLeaderUUID(leaderSessionID); + ResourceManager.this.leaderSessionID = leaderSessionID; + } + }); + } - return new TaskExecutorRegistrationSuccess(instanceID, 5000); + /** + * Callback method when current resourceManager lose leadership. + */ + @Override + public void revokeLeadership() { + runAsync(new Runnable() { + @Override + public void run() { + log.info("ResourceManager {} was revoked leadership.", getAddress()); + jobMasterGateways.clear(); + taskExecutorGateways.clear(); + slotManager.clearState(); + leaderSessionID = null; } - }, getMainThreadExecutionContext()); + }); } + /** + * Handles error occurring in the leader election service + * + * @param exception Exception being thrown in the leader election service + */ + @Override + public void handleError(final Exception exception) { + log.error("ResourceManager received an error from the LeaderElectionService.", exception); + // terminate ResourceManager in case of an error + shutDown(); + } - private class ResourceManagerLeaderContender implements LeaderContender { + private static class JobMasterLeaderListener implements LeaderRetrievalListener { - /** - * Callback method when current resourceManager is granted leadership - * - * @param leaderSessionID unique leadershipID - */ - @Override - public void grantLeadership(final UUID leaderSessionID) { - runAsync(new Runnable() { - @Override - public void run() { - log.info("ResourceManager {} was granted leadership with leader session ID {}", getAddress(), leaderSessionID); - ResourceManager.this.leaderSessionID = leaderSessionID; - // confirming the leader session ID might be blocking, - leaderElectionService.confirmLeaderSessionID(leaderSessionID); - } - }); - } + private final JobID jobID; + private UUID leaderID; - /** - * Callback method when current resourceManager lose leadership. - */ - @Override - public void revokeLeadership() { - runAsync(new Runnable() { - @Override - public void run() { - log.info("ResourceManager {} was revoked leadership.", getAddress()); - jobMasterGateways.clear(); - startedTaskExecutorGateways.clear(); - leaderSessionID = null; - } - }); + private JobMasterLeaderListener(JobID jobID) { + this.jobID = jobID; } @Override - public String getAddress() { - return ResourceManager.this.getAddress(); + public void notifyLeaderAddress(final String leaderAddress, final UUID leaderSessionID) { + this.leaderID = leaderSessionID; } - /** - * Handles error occurring in the leader election service - * - * @param exception Exception being thrown in the leader election service - */ @Override public void handleError(final Exception exception) { - runAsync(new Runnable() { - @Override - public void run() { - log.error("ResourceManager received an error from the LeaderElectionService.", exception); - // terminate ResourceManager in case of an error - shutDown(); - } - }); + // TODO } } } + diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java index 30a096febab62..d8b8ebe34db71 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java @@ -18,15 +18,16 @@ package org.apache.flink.runtime.resourcemanager; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcTimeout; import org.apache.flink.runtime.jobmaster.JobMaster; - import org.apache.flink.runtime.registration.RegistrationResponse; -import scala.concurrent.Future; -import scala.concurrent.duration.FiniteDuration; +import org.apache.flink.runtime.registration.RegistrationResponse; import java.util.UUID; /** @@ -37,21 +38,18 @@ public interface ResourceManagerGateway extends RpcGateway { /** * Register a {@link JobMaster} at the resource manager. * - * @param jobMasterRegistration Job master registration information - * @param timeout Timeout for the future to complete + * @param resourceManagerLeaderId The fencing token for the ResourceManager leader + * @param jobMasterAddress The address of the JobMaster that registers + * @param jobID The Job ID of the JobMaster that registers + * @param timeout Timeout for the future to complete * @return Future registration response */ Future registerJobMaster( - JobMasterRegistration jobMasterRegistration, - @RpcTimeout FiniteDuration timeout); + UUID resourceManagerLeaderId, + String jobMasterAddress, + JobID jobID, + @RpcTimeout Time timeout); - /** - * Register a {@link JobMaster} at the resource manager. - * - * @param jobMasterRegistration Job master registration information - * @return Future registration response - */ - Future registerJobMaster(JobMasterRegistration jobMasterRegistration); /** * Requests a slot from the resource manager. @@ -59,15 +57,15 @@ Future registerJobMaster( * @param slotRequest Slot request * @return Future slot assignment */ - Future requestSlot(SlotRequest slotRequest); + Future requestSlot(SlotRequest slotRequest); /** * Register a {@link org.apache.flink.runtime.taskexecutor.TaskExecutor} at the resource manager. * * @param resourceManagerLeaderId The fencing token for the ResourceManager leader - * @param taskExecutorAddress The address of the TaskExecutor that registers - * @param resourceID The resource ID of the TaskExecutor that registers - * @param timeout The timeout for the response. + * @param taskExecutorAddress The address of the TaskExecutor that registers + * @param resourceID The resource ID of the TaskExecutor that registers + * @param timeout The timeout for the response. * * @return The future to the response by the ResourceManager. */ @@ -75,5 +73,5 @@ Future registerTaskExecutor( UUID resourceManagerLeaderId, String taskExecutorAddress, ResourceID resourceID, - @RpcTimeout FiniteDuration timeout); + @RpcTimeout Time timeout); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/TaskExecutorRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/TaskExecutorRegistration.java index bd78a475ba77c..f8dfdc7f9f809 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/TaskExecutorRegistration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/TaskExecutorRegistration.java @@ -35,7 +35,7 @@ public class TaskExecutorRegistration implements Serializable { private InstanceID instanceID; public TaskExecutorRegistration(TaskExecutorGateway taskExecutorGateway, - InstanceID instanceID) { + InstanceID instanceID) { this.taskExecutorGateway = taskExecutorGateway; this.instanceID = instanceID; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java index 5d0013ce5a2b6..a6d219695d52f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java @@ -26,7 +26,6 @@ import org.apache.flink.runtime.clusterframework.types.ResourceSlot; import org.apache.flink.runtime.clusterframework.types.SlotID; import org.apache.flink.runtime.concurrent.Future; -import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener; import org.apache.flink.runtime.resourcemanager.SlotRequest; import org.apache.flink.runtime.resourcemanager.SlotRequestRegistered; import org.apache.flink.runtime.resourcemanager.SlotRequestReply; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java new file mode 100644 index 0000000000000..332c093590f7b --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java @@ -0,0 +1,174 @@ +/* + * 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.resourcemanager; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.concurrent.Future; +import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; +import org.apache.flink.runtime.jobmaster.JobMasterGateway; +import org.apache.flink.runtime.jobmaster.JobMasterRegistrationSuccess; +import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; +import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService; +import org.apache.flink.runtime.resourcemanager.slotmanager.SimpleSlotManager; +import org.apache.flink.runtime.rpc.TestingSerialRpcService; +import org.apache.flink.runtime.registration.RegistrationResponse; +import org.apache.flink.runtime.rpc.exceptions.LeaderSessionIDException; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.util.UUID; +import java.util.concurrent.TimeUnit; + +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.*; + +public class ResourceManagerJobMasterTest { + + private TestingSerialRpcService rpcService; + + @Before + public void setup() throws Exception { + rpcService = new TestingSerialRpcService(); + } + + @After + public void teardown() throws Exception { + rpcService.stopService(); + } + + /** + * Test receive normal registration from job master and receive duplicate registration from job master + */ + @Test + public void testRegisterJobMaster() throws Exception { + String jobMasterAddress = "/jobMasterAddress1"; + JobID jobID = mockJobMaster(jobMasterAddress); + TestingLeaderElectionService resourceManagerLeaderElectionService = new TestingLeaderElectionService(); + UUID jmLeaderID = UUID.randomUUID(); + TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService(jobMasterAddress, jmLeaderID); + final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService); + final UUID rmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); + + // test response successful + Future successfulFuture = resourceManager.registerJobMaster(rmLeaderSessionId, jmLeaderID, jobMasterAddress, jobID); + RegistrationResponse response = successfulFuture.get(5, TimeUnit.SECONDS); + assertTrue(response instanceof JobMasterRegistrationSuccess); + } + + /** + * Test receive registration with unmatched leadershipId from job master + */ + @Test + public void testRegisterJobMasterWithUnmatchedLeaderSessionId1() throws Exception { + String jobMasterAddress = "/jobMasterAddress1"; + JobID jobID = mockJobMaster(jobMasterAddress); + TestingLeaderElectionService resourceManagerLeaderElectionService = new TestingLeaderElectionService(); + UUID jmLeaderID = UUID.randomUUID(); + TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService(jobMasterAddress, jmLeaderID); + final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService); + final UUID rmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); + + // test throw exception when receive a registration from job master which takes unmatched leaderSessionId + UUID differentLeaderSessionID = UUID.randomUUID(); + Future unMatchedLeaderFuture = resourceManager.registerJobMaster(differentLeaderSessionID, jmLeaderID, jobMasterAddress, jobID); + assertTrue(unMatchedLeaderFuture.get(5, TimeUnit.SECONDS) instanceof RegistrationResponse.Decline); + } + + /** + * Test receive registration with unmatched leadershipId from job master + */ + @Test + public void testRegisterJobMasterWithUnmatchedLeaderSessionId2() throws Exception { + String jobMasterAddress = "/jobMasterAddress1"; + JobID jobID = mockJobMaster(jobMasterAddress); + TestingLeaderElectionService resourceManagerLeaderElectionService = new TestingLeaderElectionService(); + TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService(); + final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService); + final UUID rmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); + final UUID jmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); + + // test throw exception when receive a registration from job master which takes unmatched leaderSessionId + UUID differentLeaderSessionID = UUID.randomUUID(); + Future unMatchedLeaderFuture = resourceManager.registerJobMaster(rmLeaderSessionId, differentLeaderSessionID, jobMasterAddress, jobID); + assertTrue(unMatchedLeaderFuture.get(5, TimeUnit.SECONDS) instanceof RegistrationResponse.Decline); + } + + /** + * Test receive registration with invalid address from job master + */ + @Test + public void testRegisterJobMasterFromInvalidAddress() throws Exception { + String jobMasterAddress = "/jobMasterAddress1"; + JobID jobID = mockJobMaster(jobMasterAddress); + TestingLeaderElectionService resourceManagerLeaderElectionService = new TestingLeaderElectionService(); + TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService(); + final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService); + final UUID rmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); + final UUID jmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); + + // test throw exception when receive a registration from job master which takes invalid address + String invalidAddress = "/jobMasterAddress2"; + Future invalidAddressFuture = resourceManager.registerJobMaster(rmLeaderSessionId, jmLeaderSessionId, invalidAddress, jobID); + assertTrue(invalidAddressFuture.get(5, TimeUnit.SECONDS) instanceof RegistrationResponse.Decline); + } + + /** + * Check and verify return RegistrationResponse.Decline when failed to start a job master Leader retrieval listener + */ + @Test + public void testRegisterJobMasterWithFailureLeaderListener() throws Exception { + String jobMasterAddress = "/jobMasterAddress1"; + JobID jobID = mockJobMaster(jobMasterAddress); + TestingLeaderElectionService resourceManagerLeaderElectionService = new TestingLeaderElectionService(); + TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService(); + final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService); + final UUID rmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); + final UUID jmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); + + JobID unknownJobIDToHAServices = new JobID(); + // verify return RegistrationResponse.Decline when failed to start a job master Leader retrieval listener + Future declineFuture = resourceManager.registerJobMaster(rmLeaderSessionId, jmLeaderSessionId, jobMasterAddress, unknownJobIDToHAServices); + RegistrationResponse response = declineFuture.get(5, TimeUnit.SECONDS); + assertTrue(response instanceof RegistrationResponse.Decline); + } + + private JobID mockJobMaster(String jobMasterAddress) { + JobID jobID = new JobID(); + JobMasterGateway jobMasterGateway = mock(JobMasterGateway.class); + rpcService.registerGateway(jobMasterAddress, jobMasterGateway); + return jobID; + } + + private ResourceManager createAndStartResourceManager(TestingLeaderElectionService resourceManagerLeaderElectionService, JobID jobID, TestingLeaderRetrievalService jobMasterLeaderRetrievalService) { + TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices(); + highAvailabilityServices.setResourceManagerLeaderElectionService(resourceManagerLeaderElectionService); + highAvailabilityServices.setJobMasterLeaderRetriever(jobID, jobMasterLeaderRetrievalService); + ResourceManager resourceManager = new ResourceManager(rpcService, highAvailabilityServices, new SimpleSlotManager()); + resourceManager.start(); + return resourceManager; + } + + private UUID grantResourceManagerLeadership(TestingLeaderElectionService resourceManagerLeaderElectionService) { + UUID leaderSessionId = UUID.randomUUID(); + resourceManagerLeaderElectionService.isLeader(leaderSessionId); + return leaderSessionId; + } + +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java similarity index 74% rename from flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java index b75d9b8eaad1d..ed7c7d7a2cbc0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java @@ -19,8 +19,10 @@ package org.apache.flink.runtime.resourcemanager; import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; +import org.apache.flink.runtime.resourcemanager.slotmanager.SimpleSlotManager; import org.apache.flink.runtime.rpc.TestingSerialRpcService; import org.apache.flink.runtime.registration.RegistrationResponse; import org.apache.flink.runtime.rpc.exceptions.LeaderSessionIDException; @@ -29,18 +31,16 @@ import org.junit.After; import org.junit.Before; import org.junit.Test; -import scala.concurrent.Await; -import scala.concurrent.Future; -import scala.concurrent.duration.FiniteDuration; import java.util.UUID; import java.util.concurrent.TimeUnit; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.mock; -public class ResourceManagerTest { +public class ResourceManagerTaskExecutorTest { private TestingSerialRpcService rpcService; @@ -56,65 +56,59 @@ public void teardown() throws Exception { /** * Test receive normal registration from task executor and receive duplicate registration from task executor - * - * @throws Exception */ @Test public void testRegisterTaskExecutor() throws Exception { String taskExecutorAddress = "/taskExecutor1"; ResourceID taskExecutorResourceID = mockTaskExecutor(taskExecutorAddress); - TestingLeaderElectionService leaderElectionService = new TestingLeaderElectionService(); - final ResourceManager resourceManager = createAndStartResourceManager(leaderElectionService); - final UUID leaderSessionId = grantResourceManagerLeadership(leaderElectionService); + TestingLeaderElectionService rmLeaderElectionService = new TestingLeaderElectionService(); + final ResourceManager resourceManager = createAndStartResourceManager(rmLeaderElectionService); + final UUID leaderSessionId = grantLeadership(rmLeaderElectionService); // test response successful Future successfulFuture = resourceManager.registerTaskExecutor(leaderSessionId, taskExecutorAddress, taskExecutorResourceID); - RegistrationResponse response = Await.result(successfulFuture, new FiniteDuration(0, TimeUnit.SECONDS)); + RegistrationResponse response = successfulFuture.get(5, TimeUnit.SECONDS); assertTrue(response instanceof TaskExecutorRegistrationSuccess); - // test response successful with previous instanceID when receive duplicate registration from taskExecutor + // test response successful with instanceID not equal to previous when receive duplicate registration from taskExecutor Future duplicateFuture = resourceManager.registerTaskExecutor(leaderSessionId, taskExecutorAddress, taskExecutorResourceID); - RegistrationResponse duplicateResponse = Await.result(duplicateFuture, new FiniteDuration(0, TimeUnit.SECONDS)); + RegistrationResponse duplicateResponse = duplicateFuture.get(); assertTrue(duplicateResponse instanceof TaskExecutorRegistrationSuccess); - assertEquals(((TaskExecutorRegistrationSuccess) response).getRegistrationId(), ((TaskExecutorRegistrationSuccess) duplicateResponse).getRegistrationId()); + assertNotEquals(((TaskExecutorRegistrationSuccess) response).getRegistrationId(), ((TaskExecutorRegistrationSuccess) duplicateResponse).getRegistrationId()); } /** * Test receive registration with unmatched leadershipId from task executor - * - * @throws Exception */ - @Test(expected = LeaderSessionIDException.class) + @Test public void testRegisterTaskExecutorWithUnmatchedLeaderSessionId() throws Exception { String taskExecutorAddress = "/taskExecutor1"; ResourceID taskExecutorResourceID = mockTaskExecutor(taskExecutorAddress); - TestingLeaderElectionService leaderElectionService = new TestingLeaderElectionService(); - final ResourceManager resourceManager = createAndStartResourceManager(leaderElectionService); - final UUID leaderSessionId = grantResourceManagerLeadership(leaderElectionService); + TestingLeaderElectionService rmLeaderElectionService = new TestingLeaderElectionService(); + final ResourceManager resourceManager = createAndStartResourceManager(rmLeaderElectionService); + final UUID leaderSessionId = grantLeadership(rmLeaderElectionService); // test throw exception when receive a registration from taskExecutor which takes unmatched leaderSessionId UUID differentLeaderSessionID = UUID.randomUUID(); Future unMatchedLeaderFuture = resourceManager.registerTaskExecutor(differentLeaderSessionID, taskExecutorAddress, taskExecutorResourceID); - Await.result(unMatchedLeaderFuture, new FiniteDuration(200, TimeUnit.MILLISECONDS)); + assertTrue(unMatchedLeaderFuture.get(5, TimeUnit.SECONDS) instanceof RegistrationResponse.Decline); } /** * Test receive registration with invalid address from task executor - * - * @throws Exception */ - @Test(expected = Exception.class) + @Test public void testRegisterTaskExecutorFromInvalidAddress() throws Exception { String taskExecutorAddress = "/taskExecutor1"; ResourceID taskExecutorResourceID = mockTaskExecutor(taskExecutorAddress); TestingLeaderElectionService leaderElectionService = new TestingLeaderElectionService(); final ResourceManager resourceManager = createAndStartResourceManager(leaderElectionService); - final UUID leaderSessionId = grantResourceManagerLeadership(leaderElectionService); + final UUID leaderSessionId = grantLeadership(leaderElectionService); // test throw exception when receive a registration from taskExecutor which takes invalid address String invalidAddress = "/taskExecutor2"; Future invalidAddressFuture = resourceManager.registerTaskExecutor(leaderSessionId, invalidAddress, taskExecutorResourceID); - Await.result(invalidAddressFuture, new FiniteDuration(200, TimeUnit.MILLISECONDS)); + assertTrue(invalidAddressFuture.get(5, TimeUnit.SECONDS) instanceof RegistrationResponse.Decline); } private ResourceID mockTaskExecutor(String taskExecutorAddress) { @@ -124,15 +118,15 @@ private ResourceID mockTaskExecutor(String taskExecutorAddress) { return taskExecutorResourceID; } - private ResourceManager createAndStartResourceManager(TestingLeaderElectionService leaderElectionService) { + private ResourceManager createAndStartResourceManager(TestingLeaderElectionService rmLeaderElectionService) { TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices(); - highAvailabilityServices.setResourceManagerLeaderElectionService(leaderElectionService); - ResourceManager resourceManager = new ResourceManager(rpcService, highAvailabilityServices); + highAvailabilityServices.setResourceManagerLeaderElectionService(rmLeaderElectionService); + ResourceManager resourceManager = new ResourceManager(rpcService, highAvailabilityServices, new SimpleSlotManager()); resourceManager.start(); return resourceManager; } - private UUID grantResourceManagerLeadership(TestingLeaderElectionService leaderElectionService) { + private UUID grantLeadership(TestingLeaderElectionService leaderElectionService) { UUID leaderSessionId = UUID.randomUUID(); leaderElectionService.isLeader(leaderSessionId); return leaderSessionId; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java index 1f9e7e833286b..0232fab062acf 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java @@ -24,10 +24,14 @@ import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.clusterframework.types.SlotID; import org.apache.flink.runtime.concurrent.Future; +import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.NonHaServices; +import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.jobmaster.JobMasterGateway; -import org.apache.flink.runtime.resourcemanager.JobMasterRegistration; -import org.apache.flink.runtime.resourcemanager.RegistrationResponse; +import org.apache.flink.runtime.leaderelection.LeaderElectionService; +import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; +import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService; +import org.apache.flink.runtime.registration.RegistrationResponse; import org.apache.flink.runtime.resourcemanager.ResourceManager; import org.apache.flink.runtime.resourcemanager.SlotRequest; import org.apache.flink.runtime.resourcemanager.SlotRequestReply; @@ -88,14 +92,20 @@ public void testSlotsUnavailableRequest() throws Exception { testRpcService.registerGateway(jmAddress, mock(JobMasterGateway.class)); + final TestingHighAvailabilityServices testingHaServices = new TestingHighAvailabilityServices(); + final UUID rmLeaderID = UUID.randomUUID(); + final UUID jmLeaderID = UUID.randomUUID(); + TestingLeaderElectionService rmLeaderElectionService = + configureHA(testingHaServices, jobID, rmAddress, rmLeaderID, jmAddress, jmLeaderID); TestingSlotManager slotManager = Mockito.spy(new TestingSlotManager()); ResourceManager resourceManager = - new ResourceManager(testRpcService, new NonHaServices(rmAddress), slotManager); + new ResourceManager(testRpcService, testingHaServices, slotManager); resourceManager.start(); + rmLeaderElectionService.isLeader(rmLeaderID); Future registrationFuture = - resourceManager.registerJobMaster(new JobMasterRegistration(jmAddress, jobID)); + resourceManager.registerJobMaster(rmLeaderID, jmLeaderID, jmAddress, jobID); try { registrationFuture.get(5, TimeUnit.SECONDS); } catch (Exception e) { @@ -158,16 +168,23 @@ public void testSlotAvailableRequest() throws Exception { testRpcService.registerGateway(jmAddress, mock(JobMasterGateway.class)); + final TestingHighAvailabilityServices testingHaServices = new TestingHighAvailabilityServices(); + final UUID rmLeaderID = UUID.randomUUID(); + final UUID jmLeaderID = UUID.randomUUID(); + TestingLeaderElectionService rmLeaderElectionService = + configureHA(testingHaServices, jobID, rmAddress, rmLeaderID, jmAddress, jmLeaderID); + TaskExecutorGateway taskExecutorGateway = mock(TaskExecutorGateway.class); testRpcService.registerGateway(tmAddress, taskExecutorGateway); TestingSlotManager slotManager = Mockito.spy(new TestingSlotManager()); ResourceManager resourceManager = - new ResourceManager(testRpcService, new NonHaServices(rmAddress), slotManager); + new ResourceManager(testRpcService, testingHaServices, slotManager); resourceManager.start(); + rmLeaderElectionService.isLeader(rmLeaderID); Future registrationFuture = - resourceManager.registerJobMaster(new JobMasterRegistration(jmAddress, jobID)); + resourceManager.registerJobMaster(rmLeaderID, jmLeaderID, jmAddress, jobID); try { registrationFuture.get(5, TimeUnit.SECONDS); } catch (Exception e) { @@ -208,6 +225,20 @@ public void testSlotAvailableRequest() throws Exception { verify(taskExecutorGateway, timeout(5000)).requestSlot(eq(allocationID), any(UUID.class), any(Time.class)); } + private static TestingLeaderElectionService configureHA( + TestingHighAvailabilityServices testingHA, JobID jobID, String rmAddress, UUID rmID, String jmAddress, UUID jmID) { + final TestingLeaderElectionService rmLeaderElectionService = new TestingLeaderElectionService(); + testingHA.setResourceManagerLeaderElectionService(rmLeaderElectionService); + final TestingLeaderRetrievalService rmLeaderRetrievalService = new TestingLeaderRetrievalService(rmAddress, rmID); + testingHA.setResourceManagerLeaderRetriever(rmLeaderRetrievalService); + + final TestingLeaderElectionService jmLeaderElectionService = new TestingLeaderElectionService(); + testingHA.setJobMasterLeaderElectionService(jmLeaderElectionService); + final TestingLeaderRetrievalService jmLeaderRetrievalService = new TestingLeaderRetrievalService(jmAddress, jmID); + testingHA.setJobMasterLeaderRetriever(jobID, jmLeaderRetrievalService); + + return rmLeaderElectionService; + } private static class TestingSlotManager extends SimpleSlotManager { From 7c59c8b3b9dfd2f7cff24c2e152d5d8c033b13ad Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Thu, 22 Sep 2016 13:56:00 +0200 Subject: [PATCH 36/50] [hotfix] fix ResourceManagerGateway --- .../flink/runtime/resourcemanager/ResourceManagerGateway.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java index d8b8ebe34db71..484cea7459f56 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java @@ -27,7 +27,6 @@ import org.apache.flink.runtime.jobmaster.JobMaster; import org.apache.flink.runtime.registration.RegistrationResponse; -import org.apache.flink.runtime.registration.RegistrationResponse; import java.util.UUID; /** @@ -39,6 +38,7 @@ public interface ResourceManagerGateway extends RpcGateway { * Register a {@link JobMaster} at the resource manager. * * @param resourceManagerLeaderId The fencing token for the ResourceManager leader + * @param jobMasterLeaderId The fencing token for the JobMaster leader * @param jobMasterAddress The address of the JobMaster that registers * @param jobID The Job ID of the JobMaster that registers * @param timeout Timeout for the future to complete @@ -46,6 +46,7 @@ public interface ResourceManagerGateway extends RpcGateway { */ Future registerJobMaster( UUID resourceManagerLeaderId, + UUID jobMasterLeaderId, String jobMasterAddress, JobID jobID, @RpcTimeout Time timeout); From 073f6089d857784bdae384640fd200383e01c85e Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Wed, 21 Sep 2016 13:03:17 +0200 Subject: [PATCH 37/50] [hotfix] [rpc] Add RpcConnectionTest to validate that connection buildup fails fast when endpoint is unreachable. --- .../flink/runtime/rpc/AsyncCallsTest.java | 4 +- .../runtime/rpc/RpcCompletenessTest.java | 14 +-- .../flink/runtime/rpc/RpcConnectionTest.java | 86 +++++++++++++++++++ 3 files changed, 96 insertions(+), 8 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcConnectionTest.java diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java index e8255d40dacee..7affdb942777b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java @@ -43,9 +43,9 @@ public class AsyncCallsTest extends TestLogger { // shared test members // ------------------------------------------------------------------------ - private static ActorSystem actorSystem = AkkaUtils.createDefaultActorSystem(); + private static final ActorSystem actorSystem = AkkaUtils.createDefaultActorSystem(); - private static AkkaRpcService akkaRpcService = + private static final AkkaRpcService akkaRpcService = new AkkaRpcService(actorSystem, Time.milliseconds(10000L)); @AfterClass diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java index ee3f784198e76..53355e805e0af 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java @@ -30,6 +30,7 @@ import java.lang.annotation.Annotation; import java.lang.reflect.Method; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -69,7 +70,8 @@ public void testRpcCompleteness() { @SuppressWarnings("rawtypes") private void checkCompleteness(Class rpcEndpoint, Class rpcGateway) { - Method[] gatewayMethods = getRpcMethodsFromGateway(rpcGateway).toArray(new Method[0]); + List rpcMethodsFromGateway = getRpcMethodsFromGateway(rpcGateway); + Method[] gatewayMethods = rpcMethodsFromGateway.toArray(new Method[rpcMethodsFromGateway.size()]); Method[] serverMethods = rpcEndpoint.getMethods(); Map> rpcMethods = new HashMap<>(); @@ -360,13 +362,13 @@ private List getRpcMethodsFromGateway(Class interf } // Get all methods declared in current interface - for(Method method : interfaceClass.getDeclaredMethods()) { - allMethods.add(method); - } + Collections.addAll(allMethods, interfaceClass.getDeclaredMethods()); // Get all method inherited from super interface - for(Class superClass : interfaceClass.getInterfaces()) { - allMethods.addAll(getRpcMethodsFromGateway(superClass)); + for (Class superClass : interfaceClass.getInterfaces()) { + @SuppressWarnings("unchecked") + Class gatewayClass = (Class) superClass; + allMethods.addAll(getRpcMethodsFromGateway(gatewayClass)); } return allMethods; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcConnectionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcConnectionTest.java new file mode 100644 index 0000000000000..6363662dbf31a --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcConnectionTest.java @@ -0,0 +1,86 @@ +/* + * 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.rpc; + +import akka.actor.ActorSystem; +import akka.util.Timeout; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.rpc.akka.AkkaRpcService; +import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException; +import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; + +import org.junit.AfterClass; +import org.junit.Test; + +import scala.Option; +import scala.Tuple2; +import scala.concurrent.Await; +import scala.concurrent.Future; +import scala.concurrent.duration.FiniteDuration; + +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import static org.junit.Assert.*; + +/** + * This test validates that the RPC service gives a good message when it cannot + * connect to an RpcEndpoint. + */ +public class RpcConnectionTest { + + @Test + public void testConnectFailure() { + ActorSystem actorSystem = null; + RpcService rpcService = null; + try { + actorSystem = AkkaUtils.createActorSystem( + new Configuration(), Option.apply(new Tuple2("localhost", 0))); + + // we start the RPC service with a very long timeout to ensure that the test + // can only pass if the connection problem is not recognized merely via a timeout + rpcService = new AkkaRpcService(actorSystem, new Timeout(10000000, TimeUnit.SECONDS)); + + Future future = rpcService.connect("foo.bar.com.test.invalid", TaskExecutorGateway.class); + + Await.result(future, new FiniteDuration(10000000, TimeUnit.SECONDS)); + fail("should never complete normally"); + } + catch (TimeoutException e) { + fail("should not fail with a generic timeout exception"); + } + catch (RpcConnectionException e) { + // that is what we want + assertTrue("wrong error message", e.getMessage().contains("foo.bar.com.test.invalid")); + } + catch (Throwable t) { + fail("wrong exception: " + t); + } + finally { + if (rpcService != null) { + rpcService.stopService(); + } + if (actorSystem != null) { + actorSystem.shutdown(); + } + } + } +} From d9c4814c96653bf885e41e1e023e2d8dd6688a76 Mon Sep 17 00:00:00 2001 From: Kurt Young Date: Thu, 8 Sep 2016 12:00:13 +0800 Subject: [PATCH 38/50] [FLINK-4408] [JobManager] Introduce JobMasterRunner and implement job submission & setting up the ExecutionGraph This closes #2480 --- .../flink/api/common/JobExecutionResult.java | 2 +- .../flink/api/common/JobSubmissionResult.java | 2 +- .../HighAvailabilityServices.java | 12 + .../highavailability/NonHaServices.java | 16 +- .../jobmanager/OnCompletionActions.java | 31 ++ .../jobmanager/scheduler/Scheduler.java | 9 + .../runtime/jobmaster/JobManagerRunner.java | 288 +++++++++++ .../runtime/jobmaster/JobManagerServices.java | 73 +++ .../flink/runtime/jobmaster/JobMaster.java | 485 +++++++++++++----- .../runtime/jobmaster/JobMasterGateway.java | 13 + .../jobmaster/MiniClusterJobDispatcher.java | 385 ++++++++++++++ .../flink/runtime/rpc/FatalErrorHandler.java | 24 + .../runtime/taskexecutor/TaskExecutor.java | 12 + .../TestingHighAvailabilityServices.java | 39 +- .../jobmaster/JobManagerRunnerMockTest.java | 254 +++++++++ .../flink/runtime/rpc/RpcConnectionTest.java | 17 +- 16 files changed, 1533 insertions(+), 129 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/OnCompletionActions.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerServices.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/MiniClusterJobDispatcher.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/FatalErrorHandler.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerMockTest.java diff --git a/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java b/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java index cb4ecc548a852..7286cc54bddff 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java @@ -32,7 +32,7 @@ @Public public class JobExecutionResult extends JobSubmissionResult { - private long netRuntime; + private final long netRuntime; private final Map accumulatorResults; diff --git a/flink-core/src/main/java/org/apache/flink/api/common/JobSubmissionResult.java b/flink-core/src/main/java/org/apache/flink/api/common/JobSubmissionResult.java index c5dc86957089c..b0e7e24777a5f 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/JobSubmissionResult.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/JobSubmissionResult.java @@ -26,7 +26,7 @@ @Public public class JobSubmissionResult { - private JobID jobID; + private final JobID jobID; public JobSubmissionResult(JobID jobID) { this.jobID = jobID; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java index 7634176b4dca6..d67e9279b9223 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java @@ -19,6 +19,8 @@ package org.apache.flink.runtime.highavailability; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; +import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore; import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; @@ -61,4 +63,14 @@ public interface HighAvailabilityServices { * @param jobID The identifier of the job running the election. */ LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception; + + /** + * Gets the checkpoint recovery factory for the job manager + */ + CheckpointRecoveryFactory getCheckpointRecoveryFactory() throws Exception; + + /** + * Gets the submitted job graph store for the job manager + */ + SubmittedJobGraphStore getSubmittedJobGraphStore() throws Exception; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java index 33dc2d7c40168..a2c9cc4ee22a5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java @@ -19,13 +19,17 @@ package org.apache.flink.runtime.highavailability; import org.apache.flink.api.common.JobID; -import org.apache.flink.hadoop.shaded.org.jboss.netty.util.internal.ConcurrentHashMap; +import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; +import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory; +import org.apache.flink.runtime.jobmanager.StandaloneSubmittedJobGraphStore; +import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore; import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.leaderelection.StandaloneLeaderElectionService; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.leaderretrieval.StandaloneLeaderRetrievalService; import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -88,4 +92,14 @@ public LeaderElectionService getResourceManagerLeaderElectionService() throws Ex public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception { return new StandaloneLeaderElectionService(); } + + @Override + public CheckpointRecoveryFactory getCheckpointRecoveryFactory() throws Exception { + return new StandaloneCheckpointRecoveryFactory(); + } + + @Override + public SubmittedJobGraphStore getSubmittedJobGraphStore() throws Exception { + return new StandaloneSubmittedJobGraphStore(); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/OnCompletionActions.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/OnCompletionActions.java new file mode 100644 index 0000000000000..6de42539d13bd --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/OnCompletionActions.java @@ -0,0 +1,31 @@ +/* + * 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.jobmanager; + +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.runtime.rpc.FatalErrorHandler; + +public interface OnCompletionActions extends FatalErrorHandler { + + void jobFinished(JobExecutionResult result); + + void jobFailed(Throwable cause); + + void jobFinishedByOther(); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java index b839e0e5f6c7f..aa0931439a31d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java @@ -31,6 +31,7 @@ import java.util.Set; import java.util.concurrent.BlockingQueue; import java.util.concurrent.Callable; +import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingQueue; import akka.dispatch.Futures; @@ -57,6 +58,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import scala.concurrent.ExecutionContext; +import scala.concurrent.ExecutionContext$; /** * The scheduler is responsible for distributing the ready-to-run tasks among instances and slots. @@ -107,6 +109,13 @@ public class Scheduler implements InstanceListener, SlotAvailabilityListener, Sl // ------------------------------------------------------------------------ + /** + * Creates a new scheduler. + */ + public Scheduler(ExecutorService executor) { + this(ExecutionContext$.MODULE$.fromExecutor(executor)); + } + /** * Creates a new scheduler. */ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java new file mode 100644 index 0000000000000..bc2bf9a8f57a5 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java @@ -0,0 +1,288 @@ +/* + * 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.jobmaster; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; +import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmanager.OnCompletionActions; +import org.apache.flink.runtime.jobmanager.scheduler.Scheduler; +import org.apache.flink.runtime.leaderelection.LeaderContender; +import org.apache.flink.runtime.leaderelection.LeaderElectionService; +import org.apache.flink.runtime.rpc.RpcService; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.UUID; +import java.util.concurrent.Executor; + +/** + * The runner for the job manager. It deals with job level leader election and make underlying job manager + * properly reacted. + */ +public class JobManagerRunner implements LeaderContender, OnCompletionActions { + + private final Logger log = LoggerFactory.getLogger(JobManagerRunner.class); + + /** Lock to ensure that this runner can deal with leader election event and job completion notifies simultaneously */ + private final Object lock = new Object(); + + /** The job graph needs to run */ + private final JobGraph jobGraph; + + private final OnCompletionActions toNotify; + + /** The execution context which is used to execute futures */ + private final Executor executionContext; + + // TODO: use this to decide whether the job is finished by other + private final CheckpointRecoveryFactory checkpointRecoveryFactory; + + /** Leader election for this job */ + private final LeaderElectionService leaderElectionService; + + private final JobMaster jobManager; + + /** Leader session id when granted leadership */ + private UUID leaderSessionID; + + /** flag marking the runner as shut down */ + private volatile boolean shutdown; + + public JobManagerRunner( + final JobGraph jobGraph, + final Configuration configuration, + final RpcService rpcService, + final HighAvailabilityServices haServices, + final OnCompletionActions toNotify) throws Exception + { + this(jobGraph, configuration, rpcService, haServices, + JobManagerServices.fromConfiguration(configuration), toNotify); + } + + public JobManagerRunner( + final JobGraph jobGraph, + final Configuration configuration, + final RpcService rpcService, + final HighAvailabilityServices haServices, + final JobManagerServices jobManagerServices, + final OnCompletionActions toNotify) throws Exception + { + this.jobGraph = jobGraph; + this.toNotify = toNotify; + this.executionContext = rpcService.getExecutor(); + this.checkpointRecoveryFactory = haServices.getCheckpointRecoveryFactory(); + this.leaderElectionService = haServices.getJobMasterLeaderElectionService(jobGraph.getJobID()); + this.leaderSessionID = null; + + this.jobManager = new JobMaster( + jobGraph, configuration, rpcService, haServices, + jobManagerServices.libraryCacheManager, + jobManagerServices.restartStrategyFactory, + jobManagerServices.savepointStore, + jobManagerServices.timeout, + new Scheduler(jobManagerServices.executorService), + jobManagerServices.jobManagerMetricGroup, + this); + } + + //---------------------------------------------------------------------------------------------- + // Lifecycle management + //---------------------------------------------------------------------------------------------- + + public void start() throws Exception { + jobManager.init(); + jobManager.start(); + + try { + leaderElectionService.start(this); + } + catch (Exception e) { + log.error("Could not start the JobManager because the leader election service did not start.", e); + throw new Exception("Could not start the leader election service.", e); + } + } + + public void shutdown() { + shutdown(new Exception("The JobManager runner is shutting down")); + } + + public void shutdown(Throwable cause) { + // TODO what is the cause used for ? + shutdownInternally(); + } + + private void shutdownInternally() { + synchronized (lock) { + shutdown = true; + + if (leaderElectionService != null) { + try { + leaderElectionService.stop(); + } catch (Exception e) { + log.error("Could not properly shutdown the leader election service."); + } + } + + jobManager.shutDown(); + } + } + + //---------------------------------------------------------------------------------------------- + // Result and error handling methods + //---------------------------------------------------------------------------------------------- + + /** + * Job completion notification triggered by JobManager + */ + @Override + public void jobFinished(JobExecutionResult result) { + try { + shutdownInternally(); + } + finally { + if (toNotify != null) { + toNotify.jobFinished(result); + } + } + } + + /** + * Job completion notification triggered by JobManager + */ + @Override + public void jobFailed(Throwable cause) { + try { + shutdownInternally(); + } + finally { + if (toNotify != null) { + toNotify.jobFailed(cause); + } + } + } + + /** + * Job completion notification triggered by self + */ + @Override + public void jobFinishedByOther() { + try { + shutdownInternally(); + } + finally { + if (toNotify != null) { + toNotify.jobFinishedByOther(); + } + } + } + + /** + * Job completion notification triggered by JobManager or self + */ + @Override + public void onFatalError(Throwable exception) { + // first and in any case, notify our handler, so it can react fast + try { + if (toNotify != null) { + toNotify.onFatalError(exception); + } + } + finally { + log.error("JobManager runner encountered a fatal error.", exception); + shutdownInternally(); + } + } + + //---------------------------------------------------------------------------------------------- + // Leadership methods + //---------------------------------------------------------------------------------------------- + + @Override + public void grantLeadership(final UUID leaderSessionID) { + synchronized (lock) { + if (shutdown) { + log.info("JobManagerRunner already shutdown."); + return; + } + + log.info("JobManager runner for job {} ({}) was granted leadership with session id {} at {}.", + jobGraph.getName(), jobGraph.getJobID(), leaderSessionID, getAddress()); + + // The operation may be blocking, but since this runner is idle before it been granted the leadership, + // it's okay that job manager wait for the operation complete + leaderElectionService.confirmLeaderSessionID(leaderSessionID); + this.leaderSessionID = leaderSessionID; + + // Double check the leadership after we confirm that, there is a small chance that multiple + // job managers schedule the same job after if they try to recover at the same time. + // This will eventually be noticed, but can not be ruled out from the beginning. + if (leaderElectionService.hasLeadership()) { + if (isJobFinishedByOthers()) { + log.info("Job {} ({}) already finished by others.", jobGraph.getName(), jobGraph.getJobID()); + jobFinishedByOther(); + } else { + jobManager.getSelf().startJob(); + } + } + } + } + + @Override + public void revokeLeadership() { + synchronized (lock) { + if (shutdown) { + log.info("JobManagerRunner already shutdown."); + return; + } + + log.info("JobManager for job {} ({}) was revoked leadership at {}.", + jobGraph.getName(), jobGraph.getJobID(), getAddress()); + + leaderSessionID = null; + jobManager.getSelf().suspendJob(new Exception("JobManager is no longer the leader.")); + } + } + + @Override + public String getAddress() { + return jobManager.getAddress(); + } + + @Override + public void handleError(Exception exception) { + log.error("Leader Election Service encountered a fatal error.", exception); + onFatalError(exception); + } + + @VisibleForTesting + boolean isJobFinishedByOthers() { + // TODO + return false; + } + + @VisibleForTesting + boolean isShutdown() { + return shutdown; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerServices.java new file mode 100644 index 0000000000000..e6beba6808188 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerServices.java @@ -0,0 +1,73 @@ +/* + * 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.jobmaster; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.checkpoint.savepoint.SavepointStore; +import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager; +import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory; +import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; + +import java.util.concurrent.ExecutorService; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Utility class to hold all auxiliary services used by the {@link JobMaster}. + */ +public class JobManagerServices { + + public final ExecutorService executorService; + + public final BlobLibraryCacheManager libraryCacheManager; + + public final RestartStrategyFactory restartStrategyFactory; + + public final SavepointStore savepointStore; + + public final Time timeout; + + public final JobManagerMetricGroup jobManagerMetricGroup; + + public JobManagerServices( + ExecutorService executorService, + BlobLibraryCacheManager libraryCacheManager, + RestartStrategyFactory restartStrategyFactory, + SavepointStore savepointStore, + Time timeout, + JobManagerMetricGroup jobManagerMetricGroup) { + + this.executorService = checkNotNull(executorService); + this.libraryCacheManager = checkNotNull(libraryCacheManager); + this.restartStrategyFactory = checkNotNull(restartStrategyFactory); + this.savepointStore = checkNotNull(savepointStore); + this.timeout = checkNotNull(timeout); + this.jobManagerMetricGroup = checkNotNull(jobManagerMetricGroup); + } + + // ------------------------------------------------------------------------ + // Creating the components from a configuration + // ------------------------------------------------------------------------ + + public static JobManagerServices fromConfiguration(Configuration config) throws Exception { + // TODO not yet implemented + return null; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java index 1537396f347f4..b52a23c925100 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java @@ -18,21 +18,50 @@ package org.apache.flink.runtime.jobmaster; -import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; +import org.apache.flink.runtime.checkpoint.CheckpointIDCounter; +import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; +import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore; +import org.apache.flink.runtime.checkpoint.savepoint.SavepointStore; +import org.apache.flink.runtime.checkpoint.stats.CheckpointStatsTracker; +import org.apache.flink.runtime.checkpoint.stats.DisabledCheckpointStatsTracker; +import org.apache.flink.runtime.checkpoint.stats.SimpleCheckpointStatsTracker; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.runtime.client.JobSubmissionException; +import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager; +import org.apache.flink.runtime.executiongraph.ExecutionGraph; +import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; +import org.apache.flink.runtime.executiongraph.restart.RestartStrategy; +import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.leaderelection.LeaderContender; -import org.apache.flink.runtime.leaderelection.LeaderElectionService; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobgraph.jsonplan.JsonPlanGenerator; +import org.apache.flink.runtime.jobgraph.tasks.JobSnapshottingSettings; +import org.apache.flink.runtime.jobmanager.OnCompletionActions; +import org.apache.flink.runtime.jobmanager.scheduler.Scheduler; import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.rpc.RpcMethod; +import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.rpc.RpcEndpoint; +import org.apache.flink.runtime.rpc.RpcMethod; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.taskmanager.TaskExecutionState; -import org.apache.flink.util.Preconditions; +import scala.concurrent.ExecutionContext$; +import scala.concurrent.duration.FiniteDuration; -import java.util.UUID; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.Executor; + +import static org.apache.flink.util.Preconditions.checkNotNull; /** * JobMaster implementation. The job master is responsible for the execution of a single @@ -41,7 +70,7 @@ * It offers the following methods as part of its rpc interface to interact with the JobMaster * remotely: *
      - *
    • {@link #updateTaskExecutionState(TaskExecutionState)} updates the task execution state for + *
    • {@link #updateTaskExecutionState(TaskExecutionState)} updates the task execution state for * given task
    • *
    */ @@ -52,7 +81,6 @@ public class JobMaster extends RpcEndpoint { /** Logical representation of the job */ private final JobGraph jobGraph; - private final JobID jobID; /** Configuration of the job */ private final Configuration configuration; @@ -60,32 +88,67 @@ public class JobMaster extends RpcEndpoint { /** Service to contend for and retrieve the leadership of JM and RM */ private final HighAvailabilityServices highAvailabilityServices; - /** Leader Management */ - private LeaderElectionService leaderElectionService = null; - private UUID leaderSessionID; + /** Blob cache manager used across jobs */ + private final BlobLibraryCacheManager libraryCacheManager; + + /** Factory to create restart strategy for this job */ + private final RestartStrategyFactory restartStrategyFactory; + + /** Store for save points */ + private final SavepointStore savepointStore; + + /** The timeout for this job */ + private final Time timeout; + + /** The scheduler to use for scheduling new tasks as they are needed */ + private final Scheduler scheduler; + + /** The metrics group used across jobs */ + private final JobManagerMetricGroup jobManagerMetricGroup; + + /** The execution context which is used to execute futures */ + private final Executor executionContext; + + private final OnCompletionActions jobCompletionActions; + + /** The execution graph of this job */ + private volatile ExecutionGraph executionGraph; + + /** The checkpoint recovery factory used by this job */ + private CheckpointRecoveryFactory checkpointRecoveryFactory; + + private ClassLoader userCodeLoader; + + private RestartStrategy restartStrategy; + + private MetricGroup jobMetrics; - /** - * The JM's Constructor - * - * @param jobGraph The representation of the job's execution plan - * @param configuration The job's configuration - * @param rpcService The RPC service at which the JM serves - * @param highAvailabilityService The cluster's HA service from the JM can elect and retrieve leaders. - */ public JobMaster( JobGraph jobGraph, Configuration configuration, RpcService rpcService, - HighAvailabilityServices highAvailabilityService) { - + HighAvailabilityServices highAvailabilityService, + BlobLibraryCacheManager libraryCacheManager, + RestartStrategyFactory restartStrategyFactory, + SavepointStore savepointStore, + Time timeout, + Scheduler scheduler, + JobManagerMetricGroup jobManagerMetricGroup, + OnCompletionActions jobCompletionActions) + { super(rpcService); - this.jobGraph = Preconditions.checkNotNull(jobGraph); - this.jobID = Preconditions.checkNotNull(jobGraph.getJobID()); - - this.configuration = Preconditions.checkNotNull(configuration); - - this.highAvailabilityServices = Preconditions.checkNotNull(highAvailabilityService); + this.jobGraph = checkNotNull(jobGraph); + this.configuration = checkNotNull(configuration); + this.highAvailabilityServices = checkNotNull(highAvailabilityService); + this.libraryCacheManager = checkNotNull(libraryCacheManager); + this.restartStrategyFactory = checkNotNull(restartStrategyFactory); + this.savepointStore = checkNotNull(savepointStore); + this.timeout = checkNotNull(timeout); + this.scheduler = checkNotNull(scheduler); + this.jobManagerMetricGroup = checkNotNull(jobManagerMetricGroup); + this.executionContext = checkNotNull(rpcService.getExecutor()); + this.jobCompletionActions = checkNotNull(jobCompletionActions); } public ResourceManagerGateway getResourceManager() { @@ -93,93 +156,294 @@ public ResourceManagerGateway getResourceManager() { } //---------------------------------------------------------------------------------------------- - // Initialization methods + // Lifecycle management //---------------------------------------------------------------------------------------------- + + /** + * Initializing the job execution environment, should be called before start. Any error occurred during + * initialization will be treated as job submission failure. + * + * @throws JobSubmissionException + */ + public void init() throws JobSubmissionException { + log.info("Initializing job {} ({}).", jobGraph.getName(), jobGraph.getJobID()); + + try { + // IMPORTANT: We need to make sure that the library registration is the first action, + // because this makes sure that the uploaded jar files are removed in case of + // unsuccessful + try { + libraryCacheManager.registerJob(jobGraph.getJobID(), jobGraph.getUserJarBlobKeys(), + jobGraph.getClasspaths()); + } catch (Throwable t) { + throw new JobSubmissionException(jobGraph.getJobID(), + "Cannot set up the user code libraries: " + t.getMessage(), t); + } + + userCodeLoader = libraryCacheManager.getClassLoader(jobGraph.getJobID()); + if (userCodeLoader == null) { + throw new JobSubmissionException(jobGraph.getJobID(), + "The user code class loader could not be initialized."); + } + + if (jobGraph.getNumberOfVertices() == 0) { + throw new JobSubmissionException(jobGraph.getJobID(), "The given job is empty"); + } + + final RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration = + jobGraph.getSerializedExecutionConfig() + .deserializeValue(userCodeLoader) + .getRestartStrategy(); + if (restartStrategyConfiguration != null) { + restartStrategy = RestartStrategyFactory.createRestartStrategy(restartStrategyConfiguration); + } else { + restartStrategy = restartStrategyFactory.createRestartStrategy(); + } + + log.info("Using restart strategy {} for {} ({}).", restartStrategy, jobGraph.getName(), jobGraph.getJobID()); + + if (jobManagerMetricGroup != null) { + jobMetrics = jobManagerMetricGroup.addJob(jobGraph); + } + if (jobMetrics == null) { + jobMetrics = new UnregisteredMetricsGroup(); + } + + try { + checkpointRecoveryFactory = highAvailabilityServices.getCheckpointRecoveryFactory(); + } catch (Exception e) { + log.error("Could not get the checkpoint recovery factory.", e); + throw new JobSubmissionException(jobGraph.getJobID(), "Could not get the checkpoint recovery factory.", e); + } + + } catch (Throwable t) { + log.error("Failed to initializing job {} ({})", jobGraph.getName(), jobGraph.getJobID(), t); + + libraryCacheManager.unregisterJob(jobGraph.getJobID()); + + if (t instanceof JobSubmissionException) { + throw (JobSubmissionException) t; + } else { + throw new JobSubmissionException(jobGraph.getJobID(), "Failed to initialize job " + + jobGraph.getName() + " (" + jobGraph.getJobID() + ")", t); + } + } + } + + @Override public void start() { super.start(); - - // register at the election once the JM starts - registerAtElectionService(); } + @Override + public void shutDown() { + super.shutDown(); + + suspendJob(new Exception("JobManager is shutting down.")); + } //---------------------------------------------------------------------------------------------- - // JobMaster Leadership methods + // RPC methods //---------------------------------------------------------------------------------------------- /** - * Retrieves the election service and contend for the leadership. + * Start to run the job, runtime data structures like ExecutionGraph will be constructed now and checkpoint + * being recovered. After this, we will begin to schedule the job. */ - private void registerAtElectionService() { - try { - leaderElectionService = highAvailabilityServices.getJobMasterLeaderElectionService(jobID); - leaderElectionService.start(new JobMasterLeaderContender()); - } catch (Exception e) { - throw new RuntimeException("Fail to register at the election of JobMaster", e); + @RpcMethod + public void startJob() { + log.info("Starting job {} ({}).", jobGraph.getName(), jobGraph.getJobID()); + + if (executionGraph != null) { + executionGraph = new ExecutionGraph( + ExecutionContext$.MODULE$.fromExecutor(executionContext), + jobGraph.getJobID(), + jobGraph.getName(), + jobGraph.getJobConfiguration(), + jobGraph.getSerializedExecutionConfig(), + new FiniteDuration(timeout.getSize(), timeout.getUnit()), + restartStrategy, + jobGraph.getUserJarBlobKeys(), + jobGraph.getClasspaths(), + userCodeLoader, + jobMetrics); + } else { + // TODO: update last active time in JobInfo } - } - /** - * Start the execution when the leadership is granted. - * - * @param newLeaderSessionID The identifier of the new leadership session - */ - public void grantJobMasterLeadership(final UUID newLeaderSessionID) { - runAsync(new Runnable() { - @Override - public void run() { - log.info("JobManager {} grants leadership with session id {}.", getAddress(), newLeaderSessionID); + try { + executionGraph.setScheduleMode(jobGraph.getScheduleMode()); + executionGraph.setQueuedSchedulingAllowed(jobGraph.getAllowQueuedScheduling()); + + try { + executionGraph.setJsonPlan(JsonPlanGenerator.generatePlan(jobGraph)); + } catch (Exception e) { + log.warn("Cannot create JSON plan for job {} ({})", jobGraph.getJobID(), jobGraph.getName(), e); + executionGraph.setJsonPlan("{}"); + } - // The operation may be blocking, but since JM is idle before it grants the leadership, it's okay that - // JM waits here for the operation's completeness. - leaderSessionID = newLeaderSessionID; - leaderElectionService.confirmLeaderSessionID(newLeaderSessionID); + // initialize the vertices that have a master initialization hook + // file output formats create directories here, input formats create splits + if (log.isDebugEnabled()) { + log.debug("Running initialization on master for job {} ({}).", jobGraph.getJobID(), jobGraph.getName()); + } + for (JobVertex vertex : jobGraph.getVertices()) { + final String executableClass = vertex.getInvokableClassName(); + if (executableClass == null || executableClass.length() == 0) { + throw new JobExecutionException(jobGraph.getJobID(), + "The vertex " + vertex.getID() + " (" + vertex.getName() + ") has no invokable class."); + } + if (vertex.getParallelism() == ExecutionConfig.PARALLELISM_AUTO_MAX) { + vertex.setParallelism(scheduler.getTotalNumberOfSlots()); + } + + try { + vertex.initializeOnMaster(userCodeLoader); + } catch (Throwable t) { + throw new JobExecutionException(jobGraph.getJobID(), + "Cannot initialize task '" + vertex.getName() + "': " + t.getMessage(), t); + } + } - // TODO:: execute the job when the leadership is granted. + // topologically sort the job vertices and attach the graph to the existing one + final List sortedTopology = jobGraph.getVerticesSortedTopologicallyFromSources(); + if (log.isDebugEnabled()) { + log.debug("Adding {} vertices from job graph {} ({}).", sortedTopology.size(), + jobGraph.getJobID(), jobGraph.getName()); } - }); - } + executionGraph.attachJobGraph(sortedTopology); - /** - * Stop the execution when the leadership is revoked. - */ - public void revokeJobMasterLeadership() { - runAsync(new Runnable() { - @Override - public void run() { - log.info("JobManager {} was revoked leadership.", getAddress()); + if (log.isDebugEnabled()) { + log.debug("Successfully created execution graph from job graph {} ({}).", + jobGraph.getJobID(), jobGraph.getName()); + } - // TODO:: cancel the job's execution and notify all listeners - cancelAndClearEverything(new Exception("JobManager is no longer the leader.")); + final JobSnapshottingSettings snapshotSettings = jobGraph.getSnapshotSettings(); + if (snapshotSettings != null) { + List triggerVertices = getExecutionJobVertexWithId( + executionGraph, snapshotSettings.getVerticesToTrigger()); + + List ackVertices = getExecutionJobVertexWithId( + executionGraph, snapshotSettings.getVerticesToAcknowledge()); + + List confirmVertices = getExecutionJobVertexWithId( + executionGraph, snapshotSettings.getVerticesToConfirm()); + + CompletedCheckpointStore completedCheckpoints = checkpointRecoveryFactory.createCheckpointStore( + jobGraph.getJobID(), userCodeLoader); + + CheckpointIDCounter checkpointIdCounter = checkpointRecoveryFactory.createCheckpointIDCounter( + jobGraph.getJobID()); + + // Checkpoint stats tracker + boolean isStatsDisabled = configuration.getBoolean( + ConfigConstants.JOB_MANAGER_WEB_CHECKPOINTS_DISABLE, + ConfigConstants.DEFAULT_JOB_MANAGER_WEB_CHECKPOINTS_DISABLE); + + final CheckpointStatsTracker checkpointStatsTracker; + if (isStatsDisabled) { + checkpointStatsTracker = new DisabledCheckpointStatsTracker(); + } else { + int historySize = configuration.getInteger( + ConfigConstants.JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE, + ConfigConstants.DEFAULT_JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE); + checkpointStatsTracker = new SimpleCheckpointStatsTracker(historySize, ackVertices, jobMetrics); + } + + executionGraph.enableSnapshotCheckpointing( + snapshotSettings.getCheckpointInterval(), + snapshotSettings.getCheckpointTimeout(), + snapshotSettings.getMinPauseBetweenCheckpoints(), + snapshotSettings.getMaxConcurrentCheckpoints(), + triggerVertices, + ackVertices, + confirmVertices, + checkpointIdCounter, + completedCheckpoints, + savepointStore, + checkpointStatsTracker); + } + + // TODO: register this class to execution graph as job status change listeners + + // TODO: register client as job / execution status change listeners if they are interested + + /* + TODO: decide whether we should take the savepoint before recovery + + if (isRecovery) { + // this is a recovery of a master failure (this master takes over) + executionGraph.restoreLatestCheckpointedState(); + } else { + if (snapshotSettings != null) { + String savepointPath = snapshotSettings.getSavepointPath(); + if (savepointPath != null) { + // got a savepoint + log.info("Starting job from savepoint {}.", savepointPath); + + // load the savepoint as a checkpoint into the system + final CompletedCheckpoint savepoint = SavepointLoader.loadAndValidateSavepoint( + jobGraph.getJobID(), executionGraph.getAllVertices(), savepointStore, savepointPath); + executionGraph.getCheckpointCoordinator().getCheckpointStore().addCheckpoint(savepoint); + + // Reset the checkpoint ID counter + long nextCheckpointId = savepoint.getCheckpointID() + 1; + log.info("Reset the checkpoint ID to " + nextCheckpointId); + executionGraph.getCheckpointCoordinator().getCheckpointIdCounter().setCount(nextCheckpointId); - leaderSessionID = null; + executionGraph.restoreLatestCheckpointedState(); + } + } } - }); - } + */ - /** - * Handles error occurring in the leader election service - * - * @param exception Exception thrown in the leader election service - */ - public void onJobMasterElectionError(final Exception exception) { - runAsync(new Runnable() { - @Override - public void run() { - log.error("Received an error from the LeaderElectionService.", exception); + } catch (Throwable t) { + log.error("Failed to start job {} ({})", jobGraph.getName(), jobGraph.getJobID(), t); - // TODO:: cancel the job's execution and shutdown the JM - cancelAndClearEverything(exception); + executionGraph.fail(t); + executionGraph = null; - leaderSessionID = null; + final Throwable rt; + if (t instanceof JobExecutionException) { + rt = (JobExecutionException) t; + } else { + rt = new JobExecutionException(jobGraph.getJobID(), + "Failed to start job " + jobGraph.getJobID() + " (" + jobGraph.getName() + ")", t); } - }); + // TODO: notify client about this failure + + jobCompletionActions.jobFailed(rt); + return; + } + + // start scheduling job in another thread + executionContext.execute(new Runnable() { + @Override + public void run() { + if (executionGraph != null) { + try { + executionGraph.scheduleForExecution(scheduler); + } catch (Throwable t) { + executionGraph.fail(t); + } + } + } + }); } - //---------------------------------------------------------------------------------------------- - // RPC methods - //---------------------------------------------------------------------------------------------- + /** + * Suspending job, all the running tasks will be cancelled, and runtime data will be cleared. + * + * @param cause The reason of why this job been suspended. + */ + @RpcMethod + public void suspendJob(final Throwable cause) { + if (executionGraph != null) { + executionGraph.suspend(cause); + executionGraph = null; + } + } /** * Updates the task execution state for a given task. @@ -208,37 +472,26 @@ public void registerAtResourceManager(final String address) { //---------------------------------------------------------------------------------------------- /** - * Cancel the current job and notify all listeners the job's cancellation. + * Converts JobVertexIDs to corresponding ExecutionJobVertexes * - * @param cause Cause for the cancelling. + * @param executionGraph The execution graph that holds the relationship + * @param vertexIDs The vertexIDs need to be converted + * @return The corresponding ExecutionJobVertexes + * @throws JobExecutionException */ - private void cancelAndClearEverything(Throwable cause) { - // currently, nothing to do here - } - - // ------------------------------------------------------------------------ - // Utility classes - // ------------------------------------------------------------------------ - private class JobMasterLeaderContender implements LeaderContender { - - @Override - public void grantLeadership(UUID leaderSessionID) { - JobMaster.this.grantJobMasterLeadership(leaderSessionID); - } - - @Override - public void revokeLeadership() { - JobMaster.this.revokeJobMasterLeadership(); - } - - @Override - public String getAddress() { - return JobMaster.this.getAddress(); - } - - @Override - public void handleError(Exception exception) { - onJobMasterElectionError(exception); + private static List getExecutionJobVertexWithId( + final ExecutionGraph executionGraph, final List vertexIDs) + throws JobExecutionException + { + final List ret = new ArrayList<>(vertexIDs.size()); + for (JobVertexID vertexID : vertexIDs) { + final ExecutionJobVertex executionJobVertex = executionGraph.getJobVertex(vertexID); + if (executionJobVertex == null) { + throw new JobExecutionException(executionGraph.getJobID(), + "The snapshot checkpointing settings refer to non-existent vertex " + vertexID); + } + ret.add(executionJobVertex); } + return ret; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java index 86bf17c13250d..b281ea8ce7132 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java @@ -28,6 +28,19 @@ */ public interface JobMasterGateway extends RpcGateway { + /** + * Making this job begins to run. + */ + void startJob(); + + /** + * Suspending job, all the running tasks will be cancelled, and runtime status will be cleared. Should re-submit + * the job before restarting it. + * + * @param cause The reason of why this job been suspended. + */ + void suspendJob(final Throwable cause); + /** * Updates the task execution state for a given task. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/MiniClusterJobDispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/MiniClusterJobDispatcher.java new file mode 100644 index 0000000000000..792bfd564c6b6 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/MiniClusterJobDispatcher.java @@ -0,0 +1,385 @@ +/* + * 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.jobmaster; + +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmanager.OnCompletionActions; +import org.apache.flink.runtime.rpc.RpcService; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + +/** + * The dispatcher that runs in the mini cluster, waits for jobs, and starts job masters + * upon receiving jobs. + */ +public class MiniClusterJobDispatcher { + + private static final Logger LOG = LoggerFactory.getLogger(MiniClusterJobDispatcher.class); + + // ------------------------------------------------------------------------ + + /** lock to ensure that this dispatcher executes only one job at a time */ + private final Object lock = new Object(); + + /** the configuration with which the mini cluster was started */ + private final Configuration configuration; + + /** the RPC service to use by the job managers */ + private final RpcService rpcService; + + /** services for discovery, leader election, and recovery */ + private final HighAvailabilityServices haServices; + + /** al the services that the JobManager needs, such as BLOB service, factories, etc */ + private final JobManagerServices jobManagerServices; + + /** The number of JobManagers to launch (more than one simulates a high-availability setup) */ + private final int numJobManagers; + + /** The runner for the job and master. non-null if a job is currently running */ + private volatile JobManagerRunner[] runners; + + /** flag marking the dispatcher as hut down */ + private volatile boolean shutdown; + + + /** + * Starts a mini cluster job dispatcher. + * + *

    The dispatcher kicks off one JobManager per job, a behavior similar to a + * non-highly-available setup. + * + * @param config The configuration of the mini cluster + * @param haServices Access to the discovery, leader election, and recovery services + * + * @throws Exception Thrown, if the services for the JobMaster could not be started. + */ + public MiniClusterJobDispatcher( + Configuration config, + RpcService rpcService, + HighAvailabilityServices haServices) throws Exception { + this(config, rpcService, haServices, 1); + } + + /** + * Starts a mini cluster job dispatcher. + * + *

    The dispatcher may kick off more than one JobManager per job, thus simulating + * a highly-available setup. + * + * @param config The configuration of the mini cluster + * @param haServices Access to the discovery, leader election, and recovery services + * @param numJobManagers The number of JobMasters to start for each job. + * + * @throws Exception Thrown, if the services for the JobMaster could not be started. + */ + public MiniClusterJobDispatcher( + Configuration config, + RpcService rpcService, + HighAvailabilityServices haServices, + int numJobManagers) throws Exception { + + checkArgument(numJobManagers >= 1); + this.configuration = checkNotNull(config); + this.rpcService = checkNotNull(rpcService); + this.haServices = checkNotNull(haServices); + this.numJobManagers = numJobManagers; + + LOG.info("Creating JobMaster services"); + this.jobManagerServices = JobManagerServices.fromConfiguration(config); + } + + // ------------------------------------------------------------------------ + // life cycle + // ------------------------------------------------------------------------ + + /** + * Shuts down the mini cluster dispatcher. If a job is currently running, that job will be + * terminally failed. + */ + public void shutdown() { + synchronized (lock) { + if (!shutdown) { + shutdown = true; + + LOG.info("Shutting down the dispatcher"); + + // in this shutdown code we copy the references to the stack first, + // to avoid concurrent modification + + JobManagerRunner[] runners = this.runners; + if (runners != null) { + this.runners = null; + + Exception shutdownException = new Exception("The MiniCluster is shutting down"); + for (JobManagerRunner runner : runners) { + runner.shutdown(shutdownException); + } + } + } + } + } + + // ------------------------------------------------------------------------ + // submitting jobs + // ------------------------------------------------------------------------ + + /** + * This method executes a job in detached mode. The method returns immediately after the job + * has been added to the + * + * @param job The Flink job to execute + * + * @throws JobExecutionException Thrown if anything went amiss during initial job launch, + * or if the job terminally failed. + */ + public void runDetached(JobGraph job) throws JobExecutionException { + checkNotNull(job); + + LOG.info("Received job for detached execution {} ({})", job.getName(), job.getJobID()); + + synchronized (lock) { + checkState(!shutdown, "mini cluster is shut down"); + checkState(runners == null, "mini cluster can only execute one job at a time"); + + OnCompletionActions onJobCompletion = new DetachedFinalizer(numJobManagers); + + this.runners = startJobRunners(job, onJobCompletion); + } + } + + /** + * This method runs a job in blocking mode. The method returns only after the job + * completed successfully, or after it failed terminally. + * + * @param job The Flink job to execute + * @return The result of the job execution + * + * @throws JobExecutionException Thrown if anything went amiss during initial job lauch, + * or if the job terminally failed. + */ + public JobExecutionResult runJobBlocking(JobGraph job) throws JobExecutionException, InterruptedException { + checkNotNull(job); + + LOG.info("Received job for blocking execution {} ({})", job.getName(), job.getJobID()); + final BlockingJobSync onJobCompletion = new BlockingJobSync(job.getJobID(), numJobManagers); + + synchronized (lock) { + checkState(!shutdown, "mini cluster is shut down"); + checkState(runners == null, "mini cluster can only execute one job at a time"); + + this.runners = startJobRunners(job, onJobCompletion); + } + + try { + return onJobCompletion.getResult(); + } + finally { + // always clear the status for the next job + runners = null; + } + } + + private JobManagerRunner[] startJobRunners(JobGraph job, OnCompletionActions onCompletion) throws JobExecutionException { + LOG.info("Starting {} JobMaster(s) for job {} ({})", numJobManagers, job.getName(), job.getJobID()); + + JobManagerRunner[] runners = new JobManagerRunner[numJobManagers]; + for (int i = 0; i < numJobManagers; i++) { + try { + runners[i] = new JobManagerRunner(job, configuration, + rpcService, haServices, jobManagerServices, onCompletion); + runners[i].start(); + } + catch (Throwable t) { + // shut down all the ones so far + Exception shutdownCause = new Exception("Failed to properly start all mini cluster JobManagers", t); + + for (int k = 0; k <= i; k++) { + try { + if (runners[i] != null) { + runners[i].shutdown(shutdownCause); + } + } catch (Throwable ignored) { + // silent shutdown + } + } + + throw new JobExecutionException(job.getJobID(), "Could not start the JobManager(s) for the job", t); + } + } + + return runners; + } + + // ------------------------------------------------------------------------ + // test methods to simulate job master failures + // ------------------------------------------------------------------------ + + public void killJobMaster(int which) { + checkArgument(which >= 0 && which < numJobManagers, "no such job master"); + checkState(!shutdown, "mini cluster is shut down"); + + JobManagerRunner[] runners = this.runners; + checkState(runners != null, "mini cluster it not executing a job right now"); + + runners[which].shutdown(new Throwable("kill JobManager")); + } + + // ------------------------------------------------------------------------ + // utility classes + // ------------------------------------------------------------------------ + + /** + * Simple class that waits for all runners to have reported that they are done. + * In the case of a high-availability test setup, there may be multiple runners. + * After that, it marks the mini cluster as ready to receive new jobs. + */ + private class DetachedFinalizer implements OnCompletionActions { + + private final AtomicInteger numJobManagersToWaitFor; + + private DetachedFinalizer(int numJobManagersToWaitFor) { + this.numJobManagersToWaitFor = new AtomicInteger(numJobManagersToWaitFor); + } + + @Override + public void jobFinished(JobExecutionResult result) { + decrementCheckAndCleanup(); + } + + @Override + public void jobFailed(Throwable cause) { + decrementCheckAndCleanup(); + } + + @Override + public void jobFinishedByOther() { + decrementCheckAndCleanup(); + } + + @Override + public void onFatalError(Throwable exception) { + decrementCheckAndCleanup(); + } + + private void decrementCheckAndCleanup() { + if (numJobManagersToWaitFor.decrementAndGet() == 0) { + MiniClusterJobDispatcher.this.runners = null; + } + } + } + + // ------------------------------------------------------------------------ + + /** + * This class is used to sync on blocking jobs across multiple runners. + * Only after all runners reported back that they are finished, the + * result will be released. + * + * That way it is guaranteed that after the blocking job submit call returns, + * the dispatcher is immediately free to accept another job. + */ + private static class BlockingJobSync implements OnCompletionActions { + + private final JobID jobId; + + private final CountDownLatch jobMastersToWaitFor; + + private volatile Throwable jobException; + + private volatile Throwable runnerException; + + private volatile JobExecutionResult result; + + BlockingJobSync(JobID jobId, int numJobMastersToWaitFor) { + this.jobId = jobId; + this.jobMastersToWaitFor = new CountDownLatch(numJobMastersToWaitFor); + } + + @Override + public void jobFinished(JobExecutionResult jobResult) { + this.result = jobResult; + jobMastersToWaitFor.countDown(); + } + + @Override + public void jobFailed(Throwable cause) { + jobException = cause; + jobMastersToWaitFor.countDown(); + } + + @Override + public void jobFinishedByOther() { + this.jobMastersToWaitFor.countDown(); + } + + @Override + public void onFatalError(Throwable exception) { + if (runnerException == null) { + runnerException = exception; + } + } + + public JobExecutionResult getResult() throws JobExecutionException, InterruptedException { + jobMastersToWaitFor.await(); + + final Throwable jobFailureCause = this.jobException; + final Throwable runnerException = this.runnerException; + final JobExecutionResult result = this.result; + + // (1) we check if teh job terminated with an exception + // (2) we check whether the job completed successfully + // (3) we check if we have exceptions from the JobManagers. the job may still have + // completed successfully in that case, if multiple JobMasters were running + // and other took over. only if all encounter a fatal error, the job cannot finish + + if (jobFailureCause != null) { + if (jobFailureCause instanceof JobExecutionException) { + throw (JobExecutionException) jobFailureCause; + } + else { + throw new JobExecutionException(jobId, "The job execution failed", jobFailureCause); + } + } + else if (result != null) { + return result; + } + else if (runnerException != null) { + throw new JobExecutionException(jobId, + "The job execution failed because all JobManagers encountered fatal errors", runnerException); + } + else { + throw new IllegalStateException("Bug: Job finished with neither error nor result."); + } + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/FatalErrorHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/FatalErrorHandler.java new file mode 100644 index 0000000000000..7721117a240b9 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/FatalErrorHandler.java @@ -0,0 +1,24 @@ +/* + * 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.rpc; + +public interface FatalErrorHandler { + + void onFatalError(Throwable exception); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java index cf709c861df77..9e3c3b9561ff1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java @@ -21,6 +21,7 @@ import akka.actor.ActorSystem; import com.typesafe.config.Config; import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.io.network.ConnectionManager; import org.apache.flink.runtime.io.network.LocalConnectionManager; @@ -28,6 +29,7 @@ import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; import org.apache.flink.runtime.io.network.netty.NettyConnectionManager; import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; +import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore; import org.apache.flink.runtime.query.KvStateRegistry; import org.apache.flink.runtime.query.netty.DisabledKvStateRequestStats; import org.apache.flink.runtime.query.netty.KvStateServer; @@ -340,6 +342,16 @@ public LeaderElectionService getResourceManagerLeaderElectionService() throws Ex public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception { return null; } + + @Override + public CheckpointRecoveryFactory getCheckpointRecoveryFactory() throws Exception { + return null; + } + + @Override + public SubmittedJobGraphStore getSubmittedJobGraphStore() throws Exception { + return null; + } }; // start all the TaskManager services (network stack, library cache, ...) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java index 2ac43be56b9be..1a5450dd80093 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java @@ -19,10 +19,13 @@ package org.apache.flink.runtime.highavailability; import org.apache.flink.api.common.JobID; -import org.apache.flink.hadoop.shaded.org.jboss.netty.util.internal.ConcurrentHashMap; +import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; +import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore; import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; +import java.util.concurrent.ConcurrentHashMap; + /** * A variant of the HighAvailabilityServices for testing. Each individual service can be set * to an arbitrary implementation, such as a mock or default service. @@ -37,6 +40,9 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices private volatile LeaderElectionService resourceManagerLeaderElectionService; + private volatile CheckpointRecoveryFactory checkpointRecoveryFactory; + + private volatile SubmittedJobGraphStore submittedJobGraphStore; // ------------------------------------------------------------------------ // Setters for mock / testing implementations @@ -58,6 +64,14 @@ public void setResourceManagerLeaderElectionService(LeaderElectionService leader this.resourceManagerLeaderElectionService = leaderElectionService; } + public void setCheckpointRecoveryFactory(CheckpointRecoveryFactory checkpointRecoveryFactory) { + this.checkpointRecoveryFactory = checkpointRecoveryFactory; + } + + public void setSubmittedJobGraphStore(SubmittedJobGraphStore submittedJobGraphStore) { + this.submittedJobGraphStore = submittedJobGraphStore; + } + // ------------------------------------------------------------------------ // HA Services Methods // ------------------------------------------------------------------------ @@ -103,4 +117,27 @@ public LeaderElectionService getResourceManagerLeaderElectionService() throws Ex throw new IllegalStateException("ResourceManagerLeaderElectionService has not been set"); } } + + @Override + public CheckpointRecoveryFactory getCheckpointRecoveryFactory() throws Exception { + CheckpointRecoveryFactory factory = checkpointRecoveryFactory; + + if (factory != null) { + return factory; + } else { + throw new IllegalStateException("CheckpointRecoveryFactory has not been set"); + } + } + + @Override + public SubmittedJobGraphStore getSubmittedJobGraphStore() throws Exception { + SubmittedJobGraphStore store = submittedJobGraphStore; + + if (store != null) { + return store; + } else { + throw new IllegalStateException("SubmittedJobGraphStore has not been set"); + + } + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerMockTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerMockTest.java new file mode 100644 index 0000000000000..dc3b5fda49b41 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerMockTest.java @@ -0,0 +1,254 @@ +/* + * 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.jobmaster; + +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmanager.OnCompletionActions; +import org.apache.flink.runtime.leaderelection.LeaderElectionService; +import org.apache.flink.runtime.rpc.RpcService; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.util.UUID; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@RunWith(PowerMockRunner.class) +@PrepareForTest(JobManagerRunner.class) +public class JobManagerRunnerMockTest { + + private JobManagerRunner runner; + + private JobMaster jobManager; + + private JobMasterGateway jobManagerGateway; + + private LeaderElectionService leaderElectionService; + + private TestingOnCompletionActions jobCompletion; + + @Before + public void setUp() throws Exception { + jobManager = mock(JobMaster.class); + jobManagerGateway = mock(JobMasterGateway.class); + when(jobManager.getSelf()).thenReturn(jobManagerGateway); + + PowerMockito.whenNew(JobMaster.class).withAnyArguments().thenReturn(jobManager); + + jobCompletion = new TestingOnCompletionActions(); + + leaderElectionService = mock(LeaderElectionService.class); + when(leaderElectionService.hasLeadership()).thenReturn(true); + + HighAvailabilityServices haServices = mock(HighAvailabilityServices.class); + when(haServices.getJobMasterLeaderElectionService(any(JobID.class))).thenReturn(leaderElectionService); + + runner = PowerMockito.spy(new JobManagerRunner( + new JobGraph("test"), + mock(Configuration.class), + mock(RpcService.class), + haServices, + mock(JobManagerServices.class), + jobCompletion)); + } + + @After + public void tearDown() throws Exception { + } + + @Test + public void testStartAndShutdown() throws Exception { + runner.start(); + verify(jobManager).init(); + verify(jobManager).start(); + verify(leaderElectionService).start(runner); + + assertTrue(!jobCompletion.isJobFinished()); + assertTrue(!jobCompletion.isJobFailed()); + + runner.shutdown(); + verify(leaderElectionService).stop(); + verify(jobManager).shutDown(); + } + + @Test + public void testShutdownBeforeGrantLeadership() throws Exception { + runner.start(); + verify(jobManager).init(); + verify(jobManager).start(); + verify(leaderElectionService).start(runner); + + runner.shutdown(); + verify(leaderElectionService).stop(); + verify(jobManager).shutDown(); + + assertTrue(!jobCompletion.isJobFinished()); + assertTrue(!jobCompletion.isJobFailed()); + + runner.grantLeadership(UUID.randomUUID()); + assertTrue(!jobCompletion.isJobFinished()); + assertTrue(!jobCompletion.isJobFailed()); + + } + + @Test + public void testJobFinishedByOtherBeforeGrantLeadership() throws Exception { + runner.start(); + + when(runner.isJobFinishedByOthers()).thenReturn(true); + runner.grantLeadership(UUID.randomUUID()); + + // runner should shutdown automatic and informed the job completion + verify(leaderElectionService).stop(); + verify(jobManager).shutDown(); + + assertTrue(jobCompletion.isJobFinished()); + assertTrue(jobCompletion.isJobFinishedByOther()); + } + + @Test + public void testJobFinished() throws Exception { + runner.start(); + + runner.grantLeadership(UUID.randomUUID()); + verify(jobManagerGateway).startJob(); + assertTrue(!jobCompletion.isJobFinished()); + + // runner been told by JobManager that job is finished + runner.jobFinished(mock(JobExecutionResult.class)); + + assertTrue(jobCompletion.isJobFinished()); + assertFalse(jobCompletion.isJobFinishedByOther()); + verify(leaderElectionService).stop(); + verify(jobManager).shutDown(); + assertTrue(runner.isShutdown()); + } + + @Test + public void testJobFailed() throws Exception { + runner.start(); + + runner.grantLeadership(UUID.randomUUID()); + verify(jobManagerGateway).startJob(); + assertTrue(!jobCompletion.isJobFinished()); + + // runner been told by JobManager that job is failed + runner.jobFailed(new Exception("failed manually")); + + assertTrue(jobCompletion.isJobFailed()); + verify(leaderElectionService).stop(); + verify(jobManager).shutDown(); + assertTrue(runner.isShutdown()); + } + + @Test + public void testLeadershipRevoked() throws Exception { + runner.start(); + + runner.grantLeadership(UUID.randomUUID()); + verify(jobManagerGateway).startJob(); + assertTrue(!jobCompletion.isJobFinished()); + + runner.revokeLeadership(); + verify(jobManagerGateway).suspendJob(any(Throwable.class)); + assertFalse(runner.isShutdown()); + } + + @Test + public void testRegainLeadership() throws Exception { + runner.start(); + + runner.grantLeadership(UUID.randomUUID()); + verify(jobManagerGateway).startJob(); + assertTrue(!jobCompletion.isJobFinished()); + + runner.revokeLeadership(); + verify(jobManagerGateway).suspendJob(any(Throwable.class)); + assertFalse(runner.isShutdown()); + + runner.grantLeadership(UUID.randomUUID()); + verify(jobManagerGateway, times(2)).startJob(); + } + + private static class TestingOnCompletionActions implements OnCompletionActions { + + private volatile JobExecutionResult result; + + private volatile Throwable failedCause; + + private volatile boolean finishedByOther; + + @Override + public void jobFinished(JobExecutionResult result) { + checkArgument(!isJobFinished(), "job finished already"); + checkArgument(!isJobFailed(), "job failed already"); + + this.result = result; + } + + @Override + public void jobFailed(Throwable cause) { + checkArgument(!isJobFinished(), "job finished already"); + checkArgument(!isJobFailed(), "job failed already"); + + this.failedCause = cause; + } + + @Override + public void jobFinishedByOther() { + checkArgument(!isJobFinished(), "job finished already"); + checkArgument(!isJobFailed(), "job failed already"); + + this.finishedByOther = true; + } + + @Override + public void onFatalError(Throwable exception) { + jobFailed(exception); + } + + boolean isJobFinished() { + return result != null || finishedByOther; + } + + boolean isJobFinishedByOther() { + return finishedByOther; + } + + boolean isJobFailed() { + return failedCause != null; + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcConnectionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcConnectionTest.java index 6363662dbf31a..e05c8d800b935 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcConnectionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcConnectionTest.java @@ -19,23 +19,21 @@ package org.apache.flink.runtime.rpc; import akka.actor.ActorSystem; -import akka.util.Timeout; +import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.rpc.akka.AkkaRpcService; import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException; import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; -import org.junit.AfterClass; import org.junit.Test; import scala.Option; import scala.Tuple2; -import scala.concurrent.Await; -import scala.concurrent.Future; -import scala.concurrent.duration.FiniteDuration; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -57,19 +55,20 @@ public void testConnectFailure() { // we start the RPC service with a very long timeout to ensure that the test // can only pass if the connection problem is not recognized merely via a timeout - rpcService = new AkkaRpcService(actorSystem, new Timeout(10000000, TimeUnit.SECONDS)); + rpcService = new AkkaRpcService(actorSystem, Time.of(10000000, TimeUnit.SECONDS)); Future future = rpcService.connect("foo.bar.com.test.invalid", TaskExecutorGateway.class); - Await.result(future, new FiniteDuration(10000000, TimeUnit.SECONDS)); + future.get(10000000, TimeUnit.SECONDS); fail("should never complete normally"); } catch (TimeoutException e) { fail("should not fail with a generic timeout exception"); } - catch (RpcConnectionException e) { + catch (ExecutionException e) { // that is what we want - assertTrue("wrong error message", e.getMessage().contains("foo.bar.com.test.invalid")); + assertTrue(e.getCause() instanceof RpcConnectionException); + assertTrue("wrong error message", e.getCause().getMessage().contains("foo.bar.com.test.invalid")); } catch (Throwable t) { fail("wrong exception: " + t); From cb501fb45318fe024f1328b3209ced84b3113124 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 21 Sep 2016 15:18:27 +0200 Subject: [PATCH 39/50] [FLINK-4580] [rpc] Report rpc invocation exceptions to the caller This closes #2526. --- .../flink/runtime/rpc/akka/AkkaRpcActor.java | 53 +++++++++++-------- .../runtime/rpc/akka/AkkaRpcActorTest.java | 34 ++++++++++++ 2 files changed, 66 insertions(+), 21 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java index 59daa46e9f6f6..1b456a76729bd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java @@ -35,6 +35,7 @@ import org.apache.flink.runtime.rpc.akka.messages.RpcInvocation; import org.apache.flink.runtime.rpc.akka.messages.RunAsync; +import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -86,11 +87,11 @@ public void onReceive(final Object message) { unstashAll(); getContext().become(new Procedure() { @Override - public void apply(Object message) throws Exception { - if (message.equals(Processing.STOP)) { + public void apply(Object msg) throws Exception { + if (msg.equals(Processing.STOP)) { getContext().unbecome(); } else { - handleMessage(message); + handleMessage(msg); } } }); @@ -130,21 +131,36 @@ private void handleMessage(Object message) { * @param rpcInvocation Rpc invocation message */ private void handleRpcInvocation(RpcInvocation rpcInvocation) { + Method rpcMethod = null; + try { String methodName = rpcInvocation.getMethodName(); Class[] parameterTypes = rpcInvocation.getParameterTypes(); - Method rpcMethod = lookupRpcMethod(methodName, parameterTypes); + rpcMethod = lookupRpcMethod(methodName, parameterTypes); + } catch(ClassNotFoundException e) { + LOG.error("Could not load method arguments.", e); + + RpcConnectionException rpcException = new RpcConnectionException("Could not load method arguments.", e); + getSender().tell(new Status.Failure(rpcException), getSelf()); + } catch (IOException e) { + LOG.error("Could not deserialize rpc invocation message.", e); + + RpcConnectionException rpcException = new RpcConnectionException("Could not deserialize rpc invocation message.", e); + getSender().tell(new Status.Failure(rpcException), getSelf()); + } catch (final NoSuchMethodException e) { + LOG.error("Could not find rpc method for rpc invocation.", e); + + RpcConnectionException rpcException = new RpcConnectionException("Could not find rpc method for rpc invocation.", e); + getSender().tell(new Status.Failure(rpcException), getSelf()); + } - if (rpcMethod.getReturnType().equals(Void.TYPE)) { - // No return value to send back - try { + if (rpcMethod != null) { + try { + if (rpcMethod.getReturnType().equals(Void.TYPE)) { + // No return value to send back rpcMethod.invoke(rpcEndpoint, rpcInvocation.getArgs()); - } catch (Throwable e) { - LOG.error("Error while executing remote procedure call {}.", rpcMethod, e); - } - } else { - try { + } else { Object result = rpcMethod.invoke(rpcEndpoint, rpcInvocation.getArgs()); if (result instanceof Future) { @@ -169,17 +185,12 @@ public Object call() throws Exception { // tell the sender the result of the computation getSender().tell(new Status.Success(result), getSelf()); } - } catch (Throwable e) { - // tell the sender about the failure - getSender().tell(new Status.Failure(e), getSelf()); } + } catch (Throwable e) { + LOG.error("Error while executing remote procedure call {}.", rpcMethod, e); + // tell the sender about the failure + getSender().tell(new Status.Failure(e), getSelf()); } - } catch(ClassNotFoundException e) { - LOG.error("Could not load method arguments.", e); - } catch (IOException e) { - LOG.error("Could not deserialize rpc invocation message.", e); - } catch (final NoSuchMethodException e) { - LOG.error("Could not find rpc method for rpc invocation: {}.", rpcInvocation, e); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java index 5624d12fc5255..1e8c9a64d3dac 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java @@ -119,10 +119,44 @@ public void testMessageStashing() throws Exception { rpcEndpoint.shutDown(); } + /** + * Tests that we receive a RpcConnectionException when calling a rpc method (with return type) + * on a wrong rpc endpoint. + * + * @throws Exception + */ + @Test + public void testWrongGatewayEndpointConnection() throws Exception { + DummyRpcEndpoint rpcEndpoint = new DummyRpcEndpoint(akkaRpcService); + + rpcEndpoint.start(); + + Future futureGateway = akkaRpcService.connect(rpcEndpoint.getAddress(), WrongRpcGateway.class); + + WrongRpcGateway gateway = Await.result(futureGateway, timeout.duration()); + + // since it is a tell operation we won't receive a RpcConnectionException, it's only logged + gateway.tell("foobar"); + + Future result = gateway.barfoo(); + + try { + Await.result(result, timeout.duration()); + fail("We expected a RpcConnectionException."); + } catch (RpcConnectionException rpcConnectionException) { + // we expect this exception here + } + } + private interface DummyRpcGateway extends RpcGateway { Future foobar(); } + private interface WrongRpcGateway extends RpcGateway { + Future barfoo(); + void tell(String message); + } + private static class DummyRpcEndpoint extends RpcEndpoint { private volatile int _foobar = 42; From b8419ec02bdc62b002a89fc227604db16d00b193 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 26 Sep 2016 18:01:47 +0200 Subject: [PATCH 40/50] [FLINK-4687] [rpc] Add getAddress to RpcService This closes #2551. --- .../org/apache/flink/runtime/rpc/RpcService.java | 8 ++++++++ .../flink/runtime/rpc/akka/AkkaRpcService.java | 16 ++++++++++++++++ .../flink/runtime/rpc/TestingRpcService.java | 5 +++-- .../runtime/rpc/TestingSerialRpcService.java | 6 ++++++ .../flink/runtime/rpc/akka/AkkaRpcActorTest.java | 8 ++++---- .../runtime/rpc/akka/AkkaRpcServiceTest.java | 5 +++++ 6 files changed, 42 insertions(+), 6 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java index 437e08b0a3e59..96844ed53fb79 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java @@ -32,6 +32,14 @@ */ public interface RpcService { + /** + * Return the address under which the rpc service can be reached. If the rpc service cannot be + * contacted remotely, then it will return an empty string. + * + * @return Address of the rpc service or empty string if local rpc service + */ + String getAddress(); + /** * Connect to a remote rpc server under the provided address. Returns a rpc gateway which can * be used to communicate with the rpc server. If the connection failed, then the returned diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java index cee19c4d00b97..6825557199d4e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java @@ -22,6 +22,7 @@ import akka.actor.ActorRef; import akka.actor.ActorSelection; import akka.actor.ActorSystem; +import akka.actor.Address; import akka.actor.Identify; import akka.actor.PoisonPill; import akka.actor.Props; @@ -75,6 +76,8 @@ public class AkkaRpcService implements RpcService { private final Set actors = new HashSet<>(4); private final long maximumFramesize; + private final String address; + private volatile boolean stopped; public AkkaRpcService(final ActorSystem actorSystem, final Time timeout) { @@ -87,6 +90,19 @@ public AkkaRpcService(final ActorSystem actorSystem, final Time timeout) { // only local communication maximumFramesize = Long.MAX_VALUE; } + + Address actorSystemAddress = AkkaUtils.getAddress(actorSystem); + + if (actorSystemAddress.host().isDefined()) { + address = actorSystemAddress.host().get(); + } else { + address = ""; + } + } + + @Override + public String getAddress() { + return address; } // this method does not mutate state and is thus thread-safe diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java index f1640565519c2..47c9e24cdd855 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java @@ -25,6 +25,7 @@ import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture; import org.apache.flink.runtime.rpc.akka.AkkaRpcService; +import java.net.UnknownHostException; import java.util.concurrent.ConcurrentHashMap; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -57,14 +58,14 @@ public class TestingRpcService extends AkkaRpcService { /** * Creates a new {@code TestingRpcService}. */ - public TestingRpcService() { + public TestingRpcService() throws UnknownHostException { this(new Configuration()); } /** * Creates a new {@code TestingRpcService}, using the given configuration. */ - public TestingRpcService(Configuration configuration) { + public TestingRpcService(Configuration configuration) throws UnknownHostException { super(AkkaUtils.createLocalActorSystem(configuration), Time.seconds(10)); this.registeredConnections = new ConcurrentHashMap<>(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java index c58ea205c953a..5b8e6e65cd061 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java @@ -30,6 +30,7 @@ import java.lang.reflect.InvocationHandler; import java.lang.reflect.Method; import java.lang.reflect.Proxy; +import java.net.InetAddress; import java.util.BitSet; import java.util.UUID; import java.util.concurrent.Callable; @@ -120,6 +121,11 @@ public > C startServer(S rpcEndpo return self; } + @Override + public String getAddress() { + return ""; + } + @Override public Future connect(String address, Class clazz) { RpcGateway gateway = registeredConnections.get(address); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java index 1e8c9a64d3dac..5d76024c364ec 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java @@ -133,7 +133,7 @@ public void testWrongGatewayEndpointConnection() throws Exception { Future futureGateway = akkaRpcService.connect(rpcEndpoint.getAddress(), WrongRpcGateway.class); - WrongRpcGateway gateway = Await.result(futureGateway, timeout.duration()); + WrongRpcGateway gateway = futureGateway.get(timeout.getSize(), timeout.getUnit()); // since it is a tell operation we won't receive a RpcConnectionException, it's only logged gateway.tell("foobar"); @@ -141,10 +141,10 @@ public void testWrongGatewayEndpointConnection() throws Exception { Future result = gateway.barfoo(); try { - Await.result(result, timeout.duration()); + result.get(timeout.getSize(), timeout.getUnit()); fail("We expected a RpcConnectionException."); - } catch (RpcConnectionException rpcConnectionException) { - // we expect this exception here + } catch (ExecutionException executionException) { + assertTrue(executionException.getCause() instanceof RpcConnectionException); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java index 5550cb5701aa2..3388011b46fc5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java @@ -115,4 +115,9 @@ public Integer call() throws Exception { assertEquals(expected, actual); assertTrue(latch.isTriggered()); } + + @Test + public void testGetAddress() { + assertEquals(AkkaUtils.getAddress(actorSystem).host().get(), akkaRpcService.getAddress()); + } } From b955465ff2f230da0ecd195d7d0e8312fdf0578e Mon Sep 17 00:00:00 2001 From: zhuhaifengleon Date: Mon, 26 Sep 2016 17:43:44 +0800 Subject: [PATCH 41/50] [FLINK-4530] [rpc] Generalize TaskExecutorToResourceManagerConnection to be reusable This closes #2520 --- .../JobMasterToResourceManagerConnection.java | 117 +++++++++++ .../registration/RegisteredRpcConnection.java | 192 ++++++++++++++++++ .../runtime/taskexecutor/TaskExecutor.java | 4 +- ...skExecutorToResourceManagerConnection.java | 127 ++---------- .../RegisteredRpcConnectionTest.java | 183 +++++++++++++++++ .../RetryingRegistrationTest.java | 6 +- 6 files changed, 519 insertions(+), 110 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterToResourceManagerConnection.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/registration/RegisteredRpcConnection.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/registration/RegisteredRpcConnectionTest.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterToResourceManagerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterToResourceManagerConnection.java new file mode 100644 index 0000000000000..71fce8c428251 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterToResourceManagerConnection.java @@ -0,0 +1,117 @@ +/* + * 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.jobmaster; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.registration.RegisteredRpcConnection; +import org.apache.flink.runtime.registration.RegistrationResponse; +import org.apache.flink.runtime.registration.RetryingRegistration; +import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.concurrent.Future; + +import org.slf4j.Logger; + +import java.util.UUID; +import java.util.concurrent.Executor; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * The connection between a JobMaster and the ResourceManager. + */ +public class JobMasterToResourceManagerConnection + extends RegisteredRpcConnection { + + /** the JobMaster whose connection to the ResourceManager this represents */ + private final JobMaster jobMaster; + + private final JobID jobID; + + private final UUID jobMasterLeaderId; + + public JobMasterToResourceManagerConnection( + Logger log, + JobID jobID, + JobMaster jobMaster, + UUID jobMasterLeaderId, + String resourceManagerAddress, + UUID resourceManagerLeaderId, + Executor executor) { + + super(log, resourceManagerAddress, resourceManagerLeaderId, executor); + this.jobMaster = checkNotNull(jobMaster); + this.jobID = checkNotNull(jobID); + this.jobMasterLeaderId = checkNotNull(jobMasterLeaderId); + } + + @Override + protected RetryingRegistration generateRegistration() { + return new JobMasterToResourceManagerConnection.ResourceManagerRegistration( + log, jobMaster.getRpcService(), + getTargetAddress(), getTargetLeaderId(), + jobMaster.getAddress(),jobID, jobMasterLeaderId); + } + + @Override + protected void onRegistrationSuccess(JobMasterRegistrationSuccess success) { + } + + @Override + protected void onRegistrationFailure(Throwable failure) { + } + + // ------------------------------------------------------------------------ + // Utilities + // ------------------------------------------------------------------------ + + private static class ResourceManagerRegistration + extends RetryingRegistration { + + private final String jobMasterAddress; + + private final JobID jobID; + + private final UUID jobMasterLeaderId; + + ResourceManagerRegistration( + Logger log, + RpcService rpcService, + String targetAddress, + UUID leaderId, + String jobMasterAddress, + JobID jobID, + UUID jobMasterLeaderId) { + + super(log, rpcService, "ResourceManager", ResourceManagerGateway.class, targetAddress, leaderId); + this.jobMasterAddress = checkNotNull(jobMasterAddress); + this.jobID = checkNotNull(jobID); + this.jobMasterLeaderId = checkNotNull(jobMasterLeaderId); + } + + @Override + protected Future invokeRegistration( + ResourceManagerGateway gateway, UUID leaderId, long timeoutMillis) throws Exception { + + Time timeout = Time.milliseconds(timeoutMillis); + return gateway.registerJobMaster(leaderId, jobMasterLeaderId,jobMasterAddress, jobID, timeout); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RegisteredRpcConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RegisteredRpcConnection.java new file mode 100644 index 0000000000000..76093b0fea26b --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RegisteredRpcConnection.java @@ -0,0 +1,192 @@ +/* + * 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.registration; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.rpc.RpcGateway; +import org.apache.flink.runtime.concurrent.AcceptFunction; +import org.apache.flink.runtime.concurrent.ApplyFunction; +import org.apache.flink.runtime.concurrent.Future; + +import org.slf4j.Logger; + +import java.util.UUID; +import java.util.concurrent.Executor; + +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + +/** + * This utility class implements the basis of RPC connecting from one component to another component, + * for example the RPC connection from TaskExecutor to ResourceManager. + * This {@code RegisteredRpcConnection} implements registration and get target gateway . + * + *

    The registration gives access to a future that is completed upon successful registration. + * The RPC connection can be closed, for example when the target where it tries to register + * at looses leader status. + * + * @param The type of the gateway to connect to. + * @param The type of the successful registration responses. + */ +public abstract class RegisteredRpcConnection { + + /** the logger for all log messages of this class */ + protected final Logger log; + + /** the target component leaderID, for example the ResourceManager leaderID */ + private final UUID targetLeaderId; + + /** the target component Address, for example the ResourceManager Address */ + private final String targetAddress; + + /** Execution context to be used to execute the on complete action of the ResourceManagerRegistration */ + private final Executor executor; + + /** the Registration of this RPC connection */ + private RetryingRegistration pendingRegistration; + + /** the gateway to register, it's null until the registration is completed */ + private volatile Gateway targetGateway; + + /** flag indicating that the RPC connection is closed */ + private volatile boolean closed; + + // ------------------------------------------------------------------------ + + public RegisteredRpcConnection( + Logger log, + String targetAddress, + UUID targetLeaderId, + Executor executor) + { + this.log = checkNotNull(log); + this.targetAddress = checkNotNull(targetAddress); + this.targetLeaderId = checkNotNull(targetLeaderId); + this.executor = checkNotNull(executor); + } + + // ------------------------------------------------------------------------ + // Life cycle + // ------------------------------------------------------------------------ + + @SuppressWarnings("unchecked") + public void start() { + checkState(!closed, "The RPC connection is already closed"); + checkState(!isConnected() && pendingRegistration == null, "The RPC connection is already started"); + + pendingRegistration = checkNotNull(generateRegistration()); + pendingRegistration.startRegistration(); + + Future> future = pendingRegistration.getFuture(); + + future.thenAcceptAsync(new AcceptFunction>() { + @Override + public void accept(Tuple2 result) { + targetGateway = result.f0; + onRegistrationSuccess(result.f1); + } + }, executor); + + // this future should only ever fail if there is a bug, not if the registration is declined + future.exceptionallyAsync(new ApplyFunction() { + @Override + public Void apply(Throwable failure) { + onRegistrationFailure(failure); + return null; + } + }, executor); + } + + /** + * This method generate a specific Registration, for example TaskExecutor Registration at the ResourceManager + */ + protected abstract RetryingRegistration generateRegistration(); + + /** + * This method handle the Registration Response + */ + protected abstract void onRegistrationSuccess(Success success); + + /** + * This method handle the Registration failure + */ + protected abstract void onRegistrationFailure(Throwable failure); + + /** + * close connection + */ + public void close() { + closed = true; + + // make sure we do not keep re-trying forever + if (pendingRegistration != null) { + pendingRegistration.cancel(); + } + } + + public boolean isClosed() { + return closed; + } + + // ------------------------------------------------------------------------ + // Properties + // ------------------------------------------------------------------------ + + public UUID getTargetLeaderId() { + return targetLeaderId; + } + + public String getTargetAddress() { + return targetAddress; + } + + /** + * Gets the RegisteredGateway. This returns null until the registration is completed. + */ + public Gateway getTargetGateway() { + return targetGateway; + } + + public boolean isConnected() { + return targetGateway != null; + } + + // ------------------------------------------------------------------------ + + @Override + public String toString() { + String connectionInfo = "(ADDRESS: " + targetAddress + " LEADERID: " + targetLeaderId + ")"; + + if (isConnected()) { + connectionInfo = "RPC connection to " + targetGateway.getClass().getSimpleName() + " " + connectionInfo; + } else { + connectionInfo = "RPC connection to " + connectionInfo; + } + + if (isClosed()) { + connectionInfo = connectionInfo + " is closed"; + } else if (isConnected()){ + connectionInfo = connectionInfo + " is established"; + } else { + connectionInfo = connectionInfo + " is connecting"; + } + + return connectionInfo; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java index 9e3c3b9561ff1..9d9ad2ac186fc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java @@ -178,12 +178,12 @@ public void notifyOfNewResourceManagerLeader(String newLeaderAddress, UUID newLe if (newLeaderAddress != null) { // the resource manager switched to a new leader log.info("ResourceManager leader changed from {} to {}. Registering at new leader.", - resourceManagerConnection.getResourceManagerAddress(), newLeaderAddress); + resourceManagerConnection.getTargetAddress(), newLeaderAddress); } else { // address null means that the current leader is lost without a new leader being there, yet log.info("Current ResourceManager {} lost leader status. Waiting for new ResourceManager leader.", - resourceManagerConnection.getResourceManagerAddress()); + resourceManagerConnection.getTargetAddress()); } // drop the current connection or connection attempt diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java index 647359d54cfba..b4b3baee17091 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java @@ -19,16 +19,14 @@ package org.apache.flink.runtime.taskexecutor; import org.apache.flink.api.common.time.Time; -import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.clusterframework.types.ResourceID; -import org.apache.flink.runtime.concurrent.AcceptFunction; -import org.apache.flink.runtime.concurrent.ApplyFunction; -import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.instance.InstanceID; +import org.apache.flink.runtime.registration.RegisteredRpcConnection; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.registration.RegistrationResponse; import org.apache.flink.runtime.registration.RetryingRegistration; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.concurrent.Future; import org.slf4j.Logger; @@ -36,115 +34,46 @@ import java.util.concurrent.Executor; import static org.apache.flink.util.Preconditions.checkNotNull; -import static org.apache.flink.util.Preconditions.checkState; /** * The connection between a TaskExecutor and the ResourceManager. */ -public class TaskExecutorToResourceManagerConnection { - - /** the logger for all log messages of this class */ - private final Logger log; +public class TaskExecutorToResourceManagerConnection + extends RegisteredRpcConnection { /** the TaskExecutor whose connection to the ResourceManager this represents */ private final TaskExecutor taskExecutor; - private final UUID resourceManagerLeaderId; - - private final String resourceManagerAddress; - - /** Execution context to be used to execute the on complete action of the ResourceManagerRegistration */ - private final Executor executor; - - private TaskExecutorToResourceManagerConnection.ResourceManagerRegistration pendingRegistration; - - private volatile ResourceManagerGateway registeredResourceManager; - private InstanceID registrationId; - /** flag indicating that the connection is closed */ - private volatile boolean closed; - - public TaskExecutorToResourceManagerConnection( - Logger log, - TaskExecutor taskExecutor, - String resourceManagerAddress, - UUID resourceManagerLeaderId, - Executor executor) { + Logger log, + TaskExecutor taskExecutor, + String resourceManagerAddress, + UUID resourceManagerLeaderId, + Executor executor) { - this.log = checkNotNull(log); + super(log, resourceManagerAddress, resourceManagerLeaderId, executor); this.taskExecutor = checkNotNull(taskExecutor); - this.resourceManagerAddress = checkNotNull(resourceManagerAddress); - this.resourceManagerLeaderId = checkNotNull(resourceManagerLeaderId); - this.executor = checkNotNull(executor); - } - - // ------------------------------------------------------------------------ - // Life cycle - // ------------------------------------------------------------------------ - - @SuppressWarnings("unchecked") - public void start() { - checkState(!closed, "The connection is already closed"); - checkState(!isRegistered() && pendingRegistration == null, "The connection is already started"); - - pendingRegistration = new TaskExecutorToResourceManagerConnection.ResourceManagerRegistration( - log, taskExecutor.getRpcService(), - resourceManagerAddress, resourceManagerLeaderId, - taskExecutor.getAddress(), taskExecutor.getResourceID()); - pendingRegistration.startRegistration(); - - Future> future = pendingRegistration.getFuture(); - - future.thenAcceptAsync(new AcceptFunction>() { - @Override - public void accept(Tuple2 result) { - registrationId = result.f1.getRegistrationId(); - registeredResourceManager = result.f0; - } - }, executor); - - // this future should only ever fail if there is a bug, not if the registration is declined - future.exceptionallyAsync(new ApplyFunction() { - @Override - public Void apply(Throwable failure) { - taskExecutor.onFatalErrorAsync(failure); - return null; - } - }, executor); - } - - public void close() { - closed = true; - - // make sure we do not keep re-trying forever - if (pendingRegistration != null) { - pendingRegistration.cancel(); - } } - public boolean isClosed() { - return closed; - } - // ------------------------------------------------------------------------ - // Properties - // ------------------------------------------------------------------------ - - public UUID getResourceManagerLeaderId() { - return resourceManagerLeaderId; + @Override + protected RetryingRegistration generateRegistration() { + return new TaskExecutorToResourceManagerConnection.ResourceManagerRegistration( + log, taskExecutor.getRpcService(), + getTargetAddress(), getTargetLeaderId(), + taskExecutor.getAddress(),taskExecutor.getResourceID()); } - public String getResourceManagerAddress() { - return resourceManagerAddress; + @Override + protected void onRegistrationSuccess(TaskExecutorRegistrationSuccess success) { + registrationId = success.getRegistrationId(); } - /** - * Gets the ResourceManagerGateway. This returns null until the registration is completed. - */ - public ResourceManagerGateway getResourceManager() { - return registeredResourceManager; + @Override + protected void onRegistrationFailure(Throwable failure) { + taskExecutor.onFatalErrorAsync(failure); } /** @@ -155,18 +84,6 @@ public InstanceID getRegistrationId() { return registrationId; } - public boolean isRegistered() { - return registeredResourceManager != null; - } - - // ------------------------------------------------------------------------ - - @Override - public String toString() { - return String.format("Connection to ResourceManager %s (leaderId=%s)", - resourceManagerAddress, resourceManagerLeaderId); - } - // ------------------------------------------------------------------------ // Utilities // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RegisteredRpcConnectionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RegisteredRpcConnectionTest.java new file mode 100644 index 0000000000000..8558205311007 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RegisteredRpcConnectionTest.java @@ -0,0 +1,183 @@ +/* + * 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.registration; + +import org.apache.flink.runtime.registration.RetryingRegistrationTest.TestRegistrationSuccess; +import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.rpc.TestingRpcService; +import org.apache.flink.util.TestLogger; +import org.junit.Test; +import org.slf4j.LoggerFactory; + +import java.util.UUID; +import java.util.concurrent.Executor; + +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyLong; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +/** + * Tests for RegisteredRpcConnection, validating the successful, failure and close behavior. + */ +public class RegisteredRpcConnectionTest extends TestLogger { + + @Test + public void testSuccessfulRpcConnection() throws Exception { + final String testRpcConnectionEndpointAddress = ""; + final UUID leaderId = UUID.randomUUID(); + final String connectionID = "Test RPC Connection ID"; + + // an endpoint that immediately returns success + TestRegistrationGateway testGateway = new TestRegistrationGateway(new RetryingRegistrationTest.TestRegistrationSuccess(connectionID)); + TestingRpcService rpcService = new TestingRpcService(); + + try { + rpcService.registerGateway(testRpcConnectionEndpointAddress, testGateway); + + TestRpcConnection connection = new TestRpcConnection(testRpcConnectionEndpointAddress, leaderId, rpcService.getExecutor(), rpcService); + connection.start(); + + //wait for connection established + Thread.sleep(RetryingRegistrationTest.TestRetryingRegistration.MAX_TIMEOUT); + + // validate correct invocation and result + assertTrue(connection.isConnected()); + assertEquals(testRpcConnectionEndpointAddress, connection.getTargetAddress()); + assertEquals(leaderId, connection.getTargetLeaderId()); + assertEquals(testGateway, connection.getTargetGateway()); + assertEquals(connectionID, connection.getConnectionId()); + } + finally { + testGateway.stop(); + rpcService.stopService(); + } + } + + @Test + public void testRpcConnectionFailures() throws Exception { + final String connectionFailureMessage = "Test RPC Connection failure"; + final String testRpcConnectionEndpointAddress = ""; + final UUID leaderId = UUID.randomUUID(); + + TestingRpcService rpcService = new TestingRpcService(); + + try { + // gateway that upon calls Throw an exception + TestRegistrationGateway testGateway = mock(TestRegistrationGateway.class); + when(testGateway.registrationCall(any(UUID.class), anyLong())).thenThrow( + new RuntimeException(connectionFailureMessage)); + + rpcService.registerGateway(testRpcConnectionEndpointAddress, testGateway); + + TestRpcConnection connection = new TestRpcConnection(testRpcConnectionEndpointAddress, leaderId, rpcService.getExecutor(), rpcService); + connection.start(); + + //wait for connection failure + Thread.sleep(RetryingRegistrationTest.TestRetryingRegistration.MAX_TIMEOUT); + + // validate correct invocation and result + assertFalse(connection.isConnected()); + assertEquals(testRpcConnectionEndpointAddress, connection.getTargetAddress()); + assertEquals(leaderId, connection.getTargetLeaderId()); + assertNull(connection.getTargetGateway()); + assertEquals(connectionFailureMessage, connection.getFailareMessage()); + } + finally { + rpcService.stopService(); + } + } + + @Test + public void testRpcConnectionClose() throws Exception { + final String testRpcConnectionEndpointAddress = ""; + final UUID leaderId = UUID.randomUUID(); + final String connectionID = "Test RPC Connection ID"; + + TestRegistrationGateway testGateway = new TestRegistrationGateway(new RetryingRegistrationTest.TestRegistrationSuccess(connectionID)); + TestingRpcService rpcService = new TestingRpcService(); + + try{ + rpcService.registerGateway(testRpcConnectionEndpointAddress, testGateway); + + TestRpcConnection connection = new TestRpcConnection(testRpcConnectionEndpointAddress, leaderId, rpcService.getExecutor(), rpcService); + connection.start(); + //close the connection + connection.close(); + + // validate connection is closed + assertEquals(testRpcConnectionEndpointAddress, connection.getTargetAddress()); + assertEquals(leaderId, connection.getTargetLeaderId()); + assertTrue(connection.isClosed()); + } + finally { + testGateway.stop(); + rpcService.stopService(); + } + } + + // ------------------------------------------------------------------------ + // test RegisteredRpcConnection + // ------------------------------------------------------------------------ + + private static class TestRpcConnection extends RegisteredRpcConnection { + + private final RpcService rpcService; + + private String connectionId; + + private String failureMessage; + + public TestRpcConnection(String targetAddress, + UUID targetLeaderId, + Executor executor, + RpcService rpcService) + { + super(LoggerFactory.getLogger(RegisteredRpcConnectionTest.class), targetAddress, targetLeaderId, executor); + this.rpcService = rpcService; + } + + @Override + protected RetryingRegistration generateRegistration() { + return new RetryingRegistrationTest.TestRetryingRegistration(rpcService, getTargetAddress(), getTargetLeaderId()); + } + + @Override + protected void onRegistrationSuccess(RetryingRegistrationTest.TestRegistrationSuccess success) { + connectionId = success.getCorrelationId(); + } + + @Override + protected void onRegistrationFailure(Throwable failure) { + failureMessage = failure.getMessage(); + } + + public String getConnectionId() { + return connectionId; + } + + public String getFailareMessage() { + return failureMessage; + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java index e56a9ec5432e2..6d6bbef478a4f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java @@ -298,12 +298,12 @@ public void testCancellation() throws Exception { // test registration // ------------------------------------------------------------------------ - private static class TestRegistrationSuccess extends RegistrationResponse.Success { + protected static class TestRegistrationSuccess extends RegistrationResponse.Success { private static final long serialVersionUID = 5542698790917150604L; private final String correlationId; - private TestRegistrationSuccess(String correlationId) { + public TestRegistrationSuccess(String correlationId) { this.correlationId = correlationId; } @@ -312,7 +312,7 @@ public String getCorrelationId() { } } - private static class TestRetryingRegistration extends RetryingRegistration { + protected static class TestRetryingRegistration extends RetryingRegistration { // we use shorter timeouts here to speed up the tests static final long INITIAL_TIMEOUT = 20; From df25f0a2164f65d8dbb2f7964d9cedabca751996 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=B7=98=E6=B1=9F?= Date: Fri, 2 Sep 2016 18:00:49 +0800 Subject: [PATCH 42/50] [FLINK-4505] [cluster mngt] Implement TaskManager component's startup The TaskManagerRunner now contains the startup logic for the TaskManager's components. --- .../runtime/taskexecutor/TaskExecutor.java | 766 +----------------- .../taskmanager/TaskManagerRunner.java | 749 +++++++++++++++++ .../taskexecutor/TaskExecutorTest.java | 53 +- 3 files changed, 804 insertions(+), 764 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerRunner.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java index 9d9ad2ac186fc..8ce278076c856 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java @@ -18,74 +18,29 @@ package org.apache.flink.runtime.taskexecutor; -import akka.actor.ActorSystem; -import com.typesafe.config.Config; import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; import org.apache.flink.runtime.clusterframework.types.AllocationID; -import org.apache.flink.runtime.io.network.ConnectionManager; -import org.apache.flink.runtime.io.network.LocalConnectionManager; -import org.apache.flink.runtime.io.network.TaskEventDispatcher; -import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; -import org.apache.flink.runtime.io.network.netty.NettyConnectionManager; -import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore; -import org.apache.flink.runtime.query.KvStateRegistry; -import org.apache.flink.runtime.query.netty.DisabledKvStateRequestStats; -import org.apache.flink.runtime.query.netty.KvStateServer; import org.apache.flink.runtime.resourcemanager.SlotRequestRegistered; import org.apache.flink.runtime.resourcemanager.SlotRequestReply; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; -import org.apache.flink.util.Preconditions; import org.jboss.netty.channel.ChannelException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.IllegalConfigurationException; -import org.apache.flink.core.memory.HeapMemorySegment; -import org.apache.flink.core.memory.HybridMemorySegment; -import org.apache.flink.core.memory.MemorySegmentFactory; -import org.apache.flink.core.memory.MemoryType; -import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.io.disk.iomanager.IOManager; -import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode; -import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.io.network.NetworkEnvironment; -import org.apache.flink.runtime.io.network.netty.NettyConfig; -import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener; -import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.memory.MemoryManager; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcMethod; import org.apache.flink.runtime.rpc.RpcService; -import org.apache.flink.runtime.rpc.akka.AkkaRpcService; -import org.apache.flink.runtime.taskmanager.MemoryLogger; -import org.apache.flink.runtime.util.EnvironmentInformation; -import org.apache.flink.runtime.util.LeaderRetrievalUtils; -import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration; -import org.apache.flink.util.MathUtils; -import org.apache.flink.util.NetUtils; - -import scala.Tuple2; -import scala.Option; -import scala.Some; -import scala.concurrent.duration.Duration; -import scala.concurrent.duration.FiniteDuration; - -import java.io.File; -import java.io.IOException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.net.BindException; -import java.net.InetAddress; -import java.net.InetSocketAddress; import java.util.UUID; -import java.util.concurrent.TimeUnit; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -98,12 +53,10 @@ public class TaskExecutor extends RpcEndpoint { private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class); - /** The unique resource ID of this TaskExecutor */ - private final ResourceID resourceID; - + /** The connection information of this task manager */ private final TaskManagerLocation taskManagerLocation; - /** The access to the leader election and metadata storage services */ + /** The access to the leader election and retrieval services */ private final HighAvailabilityServices haServices; /** The task manager configuration */ @@ -128,28 +81,26 @@ public class TaskExecutor extends RpcEndpoint { // ------------------------------------------------------------------------ public TaskExecutor( - TaskExecutorConfiguration taskExecutorConfig, - ResourceID resourceID, - TaskManagerLocation taskManagerLocation, - MemoryManager memoryManager, - IOManager ioManager, - NetworkEnvironment networkEnvironment, - RpcService rpcService, - HighAvailabilityServices haServices) { + TaskExecutorConfiguration taskExecutorConfig, + TaskManagerLocation taskManagerLocation, + RpcService rpcService, + MemoryManager memoryManager, + IOManager ioManager, + NetworkEnvironment networkEnvironment, + HighAvailabilityServices haServices) { super(rpcService); checkArgument(taskExecutorConfig.getNumberOfSlots() > 0, "The number of slots has to be larger than 0."); this.taskExecutorConfig = checkNotNull(taskExecutorConfig); - this.resourceID = checkNotNull(resourceID); this.taskManagerLocation = checkNotNull(taskManagerLocation); this.memoryManager = checkNotNull(memoryManager); this.ioManager = checkNotNull(ioManager); this.networkEnvironment = checkNotNull(networkEnvironment); this.haServices = checkNotNull(haServices); - this.numberOfSlots = taskExecutorConfig.getNumberOfSlots(); + this.numberOfSlots = taskExecutorConfig.getNumberOfSlots(); } // ------------------------------------------------------------------------ @@ -207,7 +158,6 @@ public void notifyOfNewResourceManagerLeader(String newLeaderAddress, UUID newLe } } - /** * Requests a slot from the TaskManager * * @param allocationID id for the request @@ -220,126 +170,11 @@ public SlotRequestReply requestSlot(AllocationID allocationID, UUID resourceMana } /** - * Starts and runs the TaskManager. - *

    - * This method first tries to select the network interface to use for the TaskManager - * communication. The network interface is used both for the actor communication - * (coordination) as well as for the data exchange between task managers. Unless - * the hostname/interface is explicitly configured in the configuration, this - * method will try out various interfaces and methods to connect to the JobManager - * and select the one where the connection attempt is successful. - *

    - * After selecting the network interface, this method brings up an actor system - * for the TaskManager and its actors, starts the TaskManager's services - * (library cache, shuffle network stack, ...), and starts the TaskManager itself. - * - * @param configuration The configuration for the TaskManager. - * @param resourceID The id of the resource which the task manager will run on. - */ - public static void selectNetworkInterfaceAndRunTaskManager( - Configuration configuration, - ResourceID resourceID) throws Exception { - - final InetSocketAddress taskManagerAddress = selectNetworkInterfaceAndPort(configuration); - - runTaskManager(taskManagerAddress.getHostName(), resourceID, taskManagerAddress.getPort(), configuration); - } - - private static InetSocketAddress selectNetworkInterfaceAndPort(Configuration configuration) - throws Exception { - String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null); - if (taskManagerHostname != null) { - LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname); - } else { - LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration); - FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration); - - InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout); - taskManagerHostname = taskManagerAddress.getHostName(); - LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.", - taskManagerHostname, taskManagerAddress.getHostAddress()); - } - - // if no task manager port has been configured, use 0 (system will pick any free port) - final int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0); - if (actorSystemPort < 0 || actorSystemPort > 65535) { - throw new IllegalConfigurationException("Invalid value for '" + - ConfigConstants.TASK_MANAGER_IPC_PORT_KEY + - "' (port for the TaskManager actor system) : " + actorSystemPort + - " - Leave config parameter empty or use 0 to let the system choose a port automatically."); - } - - return new InetSocketAddress(taskManagerHostname, actorSystemPort); - } - - /** - * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its - * actors, starts the TaskManager's services (library cache, shuffle network stack, ...), - * and starts the TaskManager itself. - *

    - * This method will also spawn a process reaper for the TaskManager (kill the process if - * the actor fails) and optionally start the JVM memory logging thread. - * - * @param taskManagerHostname The hostname/address of the interface where the actor system - * will communicate. - * @param resourceID The id of the resource which the task manager will run on. - * @param actorSystemPort The port at which the actor system will communicate. - * @param configuration The configuration for the TaskManager. - */ - private static void runTaskManager( - String taskManagerHostname, - ResourceID resourceID, - int actorSystemPort, - final Configuration configuration) throws Exception { - - LOG.info("Starting TaskManager"); - - // Bring up the TaskManager actor system first, bind it to the given address. - - LOG.info("Starting TaskManager actor system at " + - NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort)); - - final ActorSystem taskManagerSystem; - try { - Tuple2 address = new Tuple2(taskManagerHostname, actorSystemPort); - Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address)); - LOG.debug("Using akka configuration\n " + akkaConfig); - taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig); - } catch (Throwable t) { - if (t instanceof ChannelException) { - Throwable cause = t.getCause(); - if (cause != null && t.getCause() instanceof BindException) { - String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort); - throw new IOException("Unable to bind TaskManager actor system to address " + - address + " - " + cause.getMessage(), t); - } - } - throw new Exception("Could not create TaskManager actor system", t); - } - - // start akka rpc service based on actor system - final Time timeout = Time.milliseconds(AkkaUtils.getTimeout(configuration).toMillis()); - final AkkaRpcService akkaRpcService = new AkkaRpcService(taskManagerSystem, timeout); - - // start high availability service to implement getResourceManagerLeaderRetriever method only - final HighAvailabilityServices haServices = new HighAvailabilityServices() { - @Override - public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception { - return LeaderRetrievalUtils.createLeaderRetrievalService(configuration); - } - - @Override public LeaderRetrievalService getJobMasterLeaderRetriever(JobID jobID) throws Exception { return null; } @Override - public LeaderElectionService getResourceManagerLeaderElectionService() throws Exception { - return null; - } - - @Override - public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception { return null; } @@ -350,552 +185,12 @@ public CheckpointRecoveryFactory getCheckpointRecoveryFactory() throws Exception @Override public SubmittedJobGraphStore getSubmittedJobGraphStore() throws Exception { - return null; - } - }; - - // start all the TaskManager services (network stack, library cache, ...) - // and the TaskManager actor - try { - LOG.info("Starting TaskManager actor"); - TaskExecutor taskExecutor = startTaskManagerComponentsAndActor( - configuration, - resourceID, - akkaRpcService, - taskManagerHostname, - haServices, - false); - - taskExecutor.start(); - - // if desired, start the logging daemon that periodically logs the memory usage information - if (LOG.isInfoEnabled() && configuration.getBoolean( - ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD, - ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) { - LOG.info("Starting periodic memory usage logger"); - - long interval = configuration.getLong( - ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS, - ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS); - - MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem); - logger.start(); - } - - // block until everything is done - taskManagerSystem.awaitTermination(); - } catch (Throwable t) { - LOG.error("Error while starting up taskManager", t); - try { - taskManagerSystem.shutdown(); - } catch (Throwable tt) { - LOG.warn("Could not cleanly shut down actor system", tt); - } - throw t; - } - } - - // -------------------------------------------------------------------------- - // Starting and running the TaskManager - // -------------------------------------------------------------------------- - - /** - * @param configuration The configuration for the TaskManager. - * @param resourceID The id of the resource which the task manager will run on. - * @param rpcService The rpc service which is used to start and connect to the TaskManager RpcEndpoint . - * @param taskManagerHostname The hostname/address that describes the TaskManager's data location. - * @param haServices Optionally, a high availability service can be provided. If none is given, - * then a HighAvailabilityServices is constructed from the configuration. - * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack. - * @return An ActorRef to the TaskManager actor. - * @throws IllegalConfigurationException Thrown, if the given config contains illegal values. - * @throws IOException Thrown, if any of the I/O components (such as buffer pools, - * I/O manager, ...) cannot be properly started. - * @throws Exception Thrown is some other error occurs while parsing the configuration - * or starting the TaskManager components. - */ - public static TaskExecutor startTaskManagerComponentsAndActor( - Configuration configuration, - ResourceID resourceID, - RpcService rpcService, - String taskManagerHostname, - HighAvailabilityServices haServices, - boolean localTaskManagerCommunication) throws Exception { - - final TaskExecutorConfiguration taskExecutorConfig = parseTaskManagerConfiguration( - configuration, taskManagerHostname, localTaskManagerCommunication); - - TaskManagerComponents taskManagerComponents = createTaskManagerComponents( - resourceID, - InetAddress.getByName(taskManagerHostname), - taskExecutorConfig, - configuration); - - final TaskExecutor taskExecutor = new TaskExecutor( - taskExecutorConfig, - resourceID, - taskManagerComponents.getTaskManagerLocation(), - taskManagerComponents.getMemoryManager(), - taskManagerComponents.getIOManager(), - taskManagerComponents.getNetworkEnvironment(), - rpcService, - haServices); - - return taskExecutor; - } - - /** - * Creates and returns the task manager components. - * - * @param resourceID resource ID of the task manager - * @param taskManagerAddress address of the task manager - * @param taskExecutorConfig task manager configuration - * @param configuration of Flink - * @return task manager components - * @throws Exception - */ - private static TaskExecutor.TaskManagerComponents createTaskManagerComponents( - ResourceID resourceID, - InetAddress taskManagerAddress, - TaskExecutorConfiguration taskExecutorConfig, - Configuration configuration) throws Exception { - MemoryType memType = taskExecutorConfig.getNetworkConfig().memoryType(); - - // pre-start checks - checkTempDirs(taskExecutorConfig.getTmpDirPaths()); - - NetworkEnvironmentConfiguration networkEnvironmentConfiguration = taskExecutorConfig.getNetworkConfig(); - - NetworkBufferPool networkBufferPool = new NetworkBufferPool( - networkEnvironmentConfiguration.numNetworkBuffers(), - networkEnvironmentConfiguration.networkBufferSize(), - networkEnvironmentConfiguration.memoryType()); - - ConnectionManager connectionManager; - - if (networkEnvironmentConfiguration.nettyConfig().isDefined()) { - connectionManager = new NettyConnectionManager(networkEnvironmentConfiguration.nettyConfig().get()); - } else { - connectionManager = new LocalConnectionManager(); - } - - ResultPartitionManager resultPartitionManager = new ResultPartitionManager(); - TaskEventDispatcher taskEventDispatcher = new TaskEventDispatcher(); - - KvStateRegistry kvStateRegistry = new KvStateRegistry(); - - KvStateServer kvStateServer; - - if (networkEnvironmentConfiguration.nettyConfig().isDefined()) { - NettyConfig nettyConfig = networkEnvironmentConfiguration.nettyConfig().get(); - - int numNetworkThreads = networkEnvironmentConfiguration.queryServerNetworkThreads() == 0 ? - nettyConfig.getNumberOfSlots() : networkEnvironmentConfiguration.queryServerNetworkThreads(); - - int numQueryThreads = networkEnvironmentConfiguration.queryServerQueryThreads() == 0 ? - nettyConfig.getNumberOfSlots() : networkEnvironmentConfiguration.queryServerQueryThreads(); - - kvStateServer = new KvStateServer( - taskManagerAddress, - networkEnvironmentConfiguration.queryServerPort(), - numNetworkThreads, - numQueryThreads, - kvStateRegistry, - new DisabledKvStateRequestStats()); - } else { - kvStateServer = null; - } - - // we start the network first, to make sure it can allocate its buffers first - final NetworkEnvironment network = new NetworkEnvironment( - networkBufferPool, - connectionManager, - resultPartitionManager, - taskEventDispatcher, - kvStateRegistry, - kvStateServer, - networkEnvironmentConfiguration.ioMode(), - networkEnvironmentConfiguration.partitionRequestInitialBackoff(), - networkEnvironmentConfiguration.partitinRequestMaxBackoff()); - - network.start(); - - TaskManagerLocation taskManagerLocation = new TaskManagerLocation( - resourceID, - taskManagerAddress, - network.getConnectionManager().getDataPort()); - - // computing the amount of memory to use depends on how much memory is available - // it strictly needs to happen AFTER the network stack has been initialized - - // check if a value has been configured - long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L); - checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory, - ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, - "MemoryManager needs at least one MB of memory. " + - "If you leave this config parameter empty, the system automatically " + - "pick a fraction of the available memory."); - - final long memorySize; - boolean preAllocateMemory = configuration.getBoolean( - ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY, - ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE); - if (configuredMemory > 0) { - if (preAllocateMemory) { - LOG.info("Using {} MB for managed memory." , configuredMemory); - } else { - LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory); - } - memorySize = configuredMemory << 20; // megabytes to bytes - } else { - float fraction = configuration.getFloat( - ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY, - ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION); - checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction, - ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY, - "MemoryManager fraction of the free memory must be between 0.0 and 1.0"); - - if (memType == MemoryType.HEAP) { - long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction); - if (preAllocateMemory) { - LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." , - fraction , relativeMemSize >> 20); - } else { - LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " + - "memory will be allocated lazily." , fraction , relativeMemSize >> 20); - } - memorySize = relativeMemSize; - } else if (memType == MemoryType.OFF_HEAP) { - // The maximum heap memory has been adjusted according to the fraction - long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory(); - long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction); - if (preAllocateMemory) { - LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." , - fraction, directMemorySize >> 20); - } else { - LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," + - " memory will be allocated lazily.", fraction, directMemorySize >> 20); - } - memorySize = directMemorySize; - } else { - throw new RuntimeException("No supported memory type detected."); - } - } - - // now start the memory manager - final MemoryManager memoryManager; - try { - memoryManager = new MemoryManager( - memorySize, - taskExecutorConfig.getNumberOfSlots(), - taskExecutorConfig.getNetworkConfig().networkBufferSize(), - memType, - preAllocateMemory); - } catch (OutOfMemoryError e) { - if (memType == MemoryType.HEAP) { - throw new Exception("OutOfMemory error (" + e.getMessage() + - ") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e); - } else if (memType == MemoryType.OFF_HEAP) { - throw new Exception("OutOfMemory error (" + e.getMessage() + - ") while allocating the TaskManager off-heap memory (" + memorySize + - " bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e); - } else { - throw e; - } - } - - // start the I/O manager, it will create some temp directories. - final IOManager ioManager = new IOManagerAsync(taskExecutorConfig.getTmpDirPaths()); - - return new TaskExecutor.TaskManagerComponents(taskManagerLocation, memoryManager, ioManager, network); - } - - // -------------------------------------------------------------------------- - // Parsing and checking the TaskManager Configuration - // -------------------------------------------------------------------------- - - /** - * Utility method to extract TaskManager config parameters from the configuration and to - * sanity check them. - * - * @param configuration The configuration. - * @param taskManagerHostname The host name under which the TaskManager communicates. - * @param localTaskManagerCommunication True, to skip initializing the network stack. - * Use only in cases where only one task manager runs. - * @return TaskExecutorConfiguration that wrappers InstanceConnectionInfo, NetworkEnvironmentConfiguration, etc. - */ - private static TaskExecutorConfiguration parseTaskManagerConfiguration( - Configuration configuration, - String taskManagerHostname, - boolean localTaskManagerCommunication) throws Exception { - - // ------- read values from the config and check them --------- - // (a lot of them) - - // ----> hosts / ports for communication and data exchange - - int dataport = configuration.getInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, - ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT); - if (dataport == 0) { - dataport = NetUtils.getAvailablePort(); - } - checkConfigParameter(dataport > 0, dataport, ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, - "Leave config parameter empty or use 0 to let the system choose a port automatically."); - - InetAddress taskManagerAddress = InetAddress.getByName(taskManagerHostname); - final InetSocketAddress taskManagerInetSocketAddress = new InetSocketAddress(taskManagerAddress, dataport); - - // ----> memory / network stack (shuffles/broadcasts), task slots, temp directories - - // we need this because many configs have been written with a "-1" entry - int slots = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1); - if (slots == -1) { - slots = 1; - } - checkConfigParameter(slots >= 1, slots, ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, - "Number of task slots must be at least one."); - - final int numNetworkBuffers = configuration.getInteger( - ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, - ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS); - checkConfigParameter(numNetworkBuffers > 0, numNetworkBuffers, - ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, ""); - - final int pageSize = configuration.getInteger( - ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY, - ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE); - // check page size of for minimum size - checkConfigParameter(pageSize >= MemoryManager.MIN_PAGE_SIZE, pageSize, - ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY, - "Minimum memory segment size is " + MemoryManager.MIN_PAGE_SIZE); - // check page size for power of two - checkConfigParameter(MathUtils.isPowerOf2(pageSize), pageSize, - ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY, - "Memory segment size must be a power of 2."); - - // check whether we use heap or off-heap memory - final MemoryType memType; - if (configuration.getBoolean(ConfigConstants.TASK_MANAGER_MEMORY_OFF_HEAP_KEY, false)) { - memType = MemoryType.OFF_HEAP; - } else { - memType = MemoryType.HEAP; - } - - // initialize the memory segment factory accordingly - if (memType == MemoryType.HEAP) { - if (!MemorySegmentFactory.initializeIfNotInitialized(HeapMemorySegment.FACTORY)) { - throw new Exception("Memory type is set to heap memory, but memory segment " + - "factory has been initialized for off-heap memory segments"); - } - } else { - if (!MemorySegmentFactory.initializeIfNotInitialized(HybridMemorySegment.FACTORY)) { - throw new Exception("Memory type is set to off-heap memory, but memory segment " + - "factory has been initialized for heap memory segments"); - } - } - - final String[] tmpDirs = configuration.getString( - ConfigConstants.TASK_MANAGER_TMP_DIR_KEY, - ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH).split(",|" + File.pathSeparator); - - final NettyConfig nettyConfig; - if (!localTaskManagerCommunication) { - nettyConfig = new NettyConfig( - taskManagerInetSocketAddress.getAddress(), - taskManagerInetSocketAddress.getPort(), - pageSize, - slots, - configuration); - } else { - nettyConfig = null; - } - - // Default spill I/O mode for intermediate results - final String syncOrAsync = configuration.getString( - ConfigConstants.TASK_MANAGER_NETWORK_DEFAULT_IO_MODE, - ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_DEFAULT_IO_MODE); - - final IOMode ioMode; - if (syncOrAsync.equals("async")) { - ioMode = IOManager.IOMode.ASYNC; - } else { - ioMode = IOManager.IOMode.SYNC; - } - - final int queryServerPort = configuration.getInteger( - ConfigConstants.QUERYABLE_STATE_SERVER_PORT, - ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_PORT); - - final int queryServerNetworkThreads = configuration.getInteger( - ConfigConstants.QUERYABLE_STATE_SERVER_NETWORK_THREADS, - ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_NETWORK_THREADS); - - final int queryServerQueryThreads = configuration.getInteger( - ConfigConstants.QUERYABLE_STATE_SERVER_QUERY_THREADS, - ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_QUERY_THREADS); - - final NetworkEnvironmentConfiguration networkConfig = new NetworkEnvironmentConfiguration( - numNetworkBuffers, - pageSize, - memType, - ioMode, - queryServerPort, - queryServerNetworkThreads, - queryServerQueryThreads, - Option.apply(nettyConfig), - 500, - 30000); - - // ----> timeouts, library caching, profiling - - final FiniteDuration timeout; - try { - timeout = AkkaUtils.getTimeout(configuration); - } catch (Exception e) { - throw new IllegalArgumentException( - "Invalid format for '" + ConfigConstants.AKKA_ASK_TIMEOUT + - "'.Use formats like '50 s' or '1 min' to specify the timeout."); - } - LOG.info("Messages between TaskManager and JobManager have a max timeout of " + timeout); - - final long cleanupInterval = configuration.getLong( - ConfigConstants.LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL, - ConfigConstants.DEFAULT_LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL) * 1000; - - final FiniteDuration finiteRegistrationDuration; - try { - Duration maxRegistrationDuration = Duration.create(configuration.getString( - ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION, - ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_DURATION)); - if (maxRegistrationDuration.isFinite()) { - finiteRegistrationDuration = new FiniteDuration(maxRegistrationDuration.toSeconds(), TimeUnit.SECONDS); - } else { - finiteRegistrationDuration = null; - } - } catch (NumberFormatException e) { - throw new IllegalArgumentException("Invalid format for parameter " + - ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION, e); - } - - final FiniteDuration initialRegistrationPause; - try { - Duration pause = Duration.create(configuration.getString( - ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, - ConfigConstants.DEFAULT_TASK_MANAGER_INITIAL_REGISTRATION_PAUSE)); - if (pause.isFinite()) { - initialRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS); - } else { - throw new IllegalArgumentException("The initial registration pause must be finite: " + pause); - } - } catch (NumberFormatException e) { - throw new IllegalArgumentException("Invalid format for parameter " + - ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e); - } - - final FiniteDuration maxRegistrationPause; - try { - Duration pause = Duration.create(configuration.getString( - ConfigConstants.TASK_MANAGER_MAX_REGISTARTION_PAUSE, - ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_PAUSE)); - if (pause.isFinite()) { - maxRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS); - } else { - throw new IllegalArgumentException("The maximum registration pause must be finite: " + pause); - } - } catch (NumberFormatException e) { - throw new IllegalArgumentException("Invalid format for parameter " + - ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e); - } - - final FiniteDuration refusedRegistrationPause; - try { - Duration pause = Duration.create(configuration.getString( - ConfigConstants.TASK_MANAGER_REFUSED_REGISTRATION_PAUSE, - ConfigConstants.DEFAULT_TASK_MANAGER_REFUSED_REGISTRATION_PAUSE)); - if (pause.isFinite()) { - refusedRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS); - } else { - throw new IllegalArgumentException("The refused registration pause must be finite: " + pause); - } - } catch (NumberFormatException e) { - throw new IllegalArgumentException("Invalid format for parameter " + - ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e); - } - - return new TaskExecutorConfiguration( - tmpDirs, - cleanupInterval, - networkConfig, - timeout, - finiteRegistrationDuration, - slots, - configuration, - initialRegistrationPause, - maxRegistrationPause, - refusedRegistrationPause); - } - - /** - * Validates a condition for a config parameter and displays a standard exception, if the - * the condition does not hold. - * - * @param condition The condition that must hold. If the condition is false, an exception is thrown. - * @param parameter The parameter value. Will be shown in the exception message. - * @param name The name of the config parameter. Will be shown in the exception message. - * @param errorMessage The optional custom error message to append to the exception message. - */ - private static void checkConfigParameter( - boolean condition, - Object parameter, - String name, - String errorMessage) { - if (!condition) { - throw new IllegalConfigurationException("Invalid configuration value for " + name + " : " + parameter + " - " + errorMessage); - } - } - - /** - * Validates that all the directories denoted by the strings do actually exist, are proper - * directories (not files), and are writable. - * - * @param tmpDirs The array of directory paths to check. - * @throws Exception Thrown if any of the directories does not exist or is not writable - * or is a file, rather than a directory. - */ - private static void checkTempDirs(String[] tmpDirs) throws IOException { - for (String dir : tmpDirs) { - if (dir != null && !dir.equals("")) { - File file = new File(dir); - if (!file.exists()) { - throw new IOException("Temporary file directory " + file.getAbsolutePath() + " does not exist."); - } - if (!file.isDirectory()) { - throw new IOException("Temporary file directory " + file.getAbsolutePath() + " is not a directory."); - } - if (!file.canWrite()) { - throw new IOException("Temporary file directory " + file.getAbsolutePath() + " is not writable."); - } - - if (LOG.isInfoEnabled()) { - long totalSpaceGb = file.getTotalSpace() >> 30; - long usableSpaceGb = file.getUsableSpace() >> 30; - double usablePercentage = (double)usableSpaceGb / totalSpaceGb * 100; - String path = file.getAbsolutePath(); - LOG.info(String.format("Temporary file directory '%s': total %d GB, " + "usable %d GB (%.2f%% usable)", - path, totalSpaceGb, usableSpaceGb, usablePercentage)); - } - } else { - throw new IllegalArgumentException("Temporary file directory #$id is null."); - } - } - } - // ------------------------------------------------------------------------ // Properties // ------------------------------------------------------------------------ public ResourceID getResourceID() { - return resourceID; + return taskManagerLocation.getResourceID(); } // ------------------------------------------------------------------------ @@ -959,37 +254,4 @@ public void handleError(Exception exception) { } } - private static class TaskManagerComponents { - private final TaskManagerLocation taskManagerLocation; - private final MemoryManager memoryManager; - private final IOManager ioManager; - private final NetworkEnvironment networkEnvironment; - - private TaskManagerComponents( - TaskManagerLocation taskManagerLocation, - MemoryManager memoryManager, - IOManager ioManager, - NetworkEnvironment networkEnvironment) { - this.taskManagerLocation = Preconditions.checkNotNull(taskManagerLocation); - this.memoryManager = Preconditions.checkNotNull(memoryManager); - this.ioManager = Preconditions.checkNotNull(ioManager); - this.networkEnvironment = Preconditions.checkNotNull(networkEnvironment); - } - - public MemoryManager getMemoryManager() { - return memoryManager; - } - - public IOManager getIOManager() { - return ioManager; - } - - public NetworkEnvironment getNetworkEnvironment() { - return networkEnvironment; - } - - public TaskManagerLocation getTaskManagerLocation() { - return taskManagerLocation; - } - } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerRunner.java new file mode 100644 index 0000000000000..4f756fbae6d7a --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerRunner.java @@ -0,0 +1,749 @@ +/* + * 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.taskmanager; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.IllegalConfigurationException; +import org.apache.flink.core.memory.HeapMemorySegment; +import org.apache.flink.core.memory.HybridMemorySegment; +import org.apache.flink.core.memory.MemorySegmentFactory; +import org.apache.flink.core.memory.MemoryType; +import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; +import org.apache.flink.runtime.io.network.ConnectionManager; +import org.apache.flink.runtime.io.network.LocalConnectionManager; +import org.apache.flink.runtime.io.network.NetworkEnvironment; +import org.apache.flink.runtime.io.network.TaskEventDispatcher; +import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; +import org.apache.flink.runtime.io.network.netty.NettyConfig; +import org.apache.flink.runtime.io.network.netty.NettyConnectionManager; +import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; +import org.apache.flink.runtime.leaderelection.LeaderElectionService; +import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; +import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.query.KvStateRegistry; +import org.apache.flink.runtime.query.netty.DisabledKvStateRequestStats; +import org.apache.flink.runtime.query.netty.KvStateServer; +import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.rpc.akka.AkkaRpcService; +import org.apache.flink.runtime.taskexecutor.TaskExecutor; +import org.apache.flink.runtime.taskexecutor.TaskExecutorConfiguration; +import org.apache.flink.runtime.util.EnvironmentInformation; +import org.apache.flink.runtime.util.LeaderRetrievalUtils; +import org.apache.flink.util.MathUtils; +import org.apache.flink.util.NetUtils; + +import akka.actor.ActorSystem; +import akka.util.Timeout; +import com.typesafe.config.Config; +import org.apache.flink.util.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.Option; +import scala.Some; +import scala.Tuple2; +import scala.concurrent.duration.Duration; +import scala.concurrent.duration.FiniteDuration; + +import java.io.File; +import java.io.IOException; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.util.concurrent.TimeUnit; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * This class is the executable entry point for the task manager in yarn or standalone mode. + * It constructs the related components (network, I/O manager, memory manager, RPC service, HA service) + * and starts them. + */ +public class TaskManagerRunner { + + private static final Logger LOG = LoggerFactory.getLogger(TaskManagerRunner.class); + + /** + * Constructs related components of the TaskManager and starts them. + * + * @param configuration The configuration for the TaskManager. + * @param resourceID The id of the resource which the task manager will run on. + * @param rpcService Optionally, The rpc service which is used to start and connect to the TaskManager RpcEndpoint . + * If none is given, then a RpcService is constructed from the configuration. + * @param taskManagerHostname Optionally, The hostname/address that describes the TaskManager's data location. + * If none is given, it can be got from the configuration. + * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack. + * @param haServices Optionally, a high availability service can be provided. If none is given, + * then a HighAvailabilityServices is constructed from the configuration. + */ + public static void createAndStartComponents( + final Configuration configuration, + final ResourceID resourceID, + RpcService rpcService, + String taskManagerHostname, + boolean localTaskManagerCommunication, + HighAvailabilityServices haServices) throws Exception { + + checkNotNull(configuration); + checkNotNull(resourceID); + + if (taskManagerHostname == null || taskManagerHostname.isEmpty()) { + taskManagerHostname = selectNetworkInterface(configuration); + } + + if (rpcService == null) { + // if no task manager port has been configured, use 0 (system will pick any free port) + final int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0); + if (actorSystemPort < 0 || actorSystemPort > 65535) { + throw new IllegalConfigurationException("Invalid value for '" + + ConfigConstants.TASK_MANAGER_IPC_PORT_KEY + + "' (port for the TaskManager actor system) : " + actorSystemPort + + " - Leave config parameter empty or use 0 to let the system choose a port automatically."); + } + rpcService = createRpcService(configuration, taskManagerHostname, actorSystemPort); + } + + if(haServices == null) { + // start high availability service to implement getResourceManagerLeaderRetriever method only + haServices = new HighAvailabilityServices() { + @Override + public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception { + return LeaderRetrievalUtils.createLeaderRetrievalService(configuration); + } + + @Override + public LeaderElectionService getResourceManagerLeaderElectionService() throws Exception { + return null; + } + + @Override + public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception { + return null; + } + }; + } + + createAndStartTaskManagerComponents( + configuration, + resourceID, + rpcService, + taskManagerHostname, + haServices, + localTaskManagerCommunication); + } + + /** + *

    + * This method tries to select the network interface to use for the TaskManager + * communication. The network interface is used both for the actor communication + * (coordination) as well as for the data exchange between task managers. Unless + * the hostname/interface is explicitly configured in the configuration, this + * method will try out various interfaces and methods to connect to the JobManager + * and select the one where the connection attempt is successful. + *

    + * + * @param configuration The configuration for the TaskManager. + * @return The host name under which the TaskManager communicates. + */ + private static String selectNetworkInterface(Configuration configuration) throws Exception { + String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null); + if (taskManagerHostname != null) { + LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname); + } else { + LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration); + FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration); + + InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout); + taskManagerHostname = taskManagerAddress.getHostName(); + LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.", + taskManagerHostname, taskManagerAddress.getHostAddress()); + } + + return taskManagerHostname; + } + + /** + * Utility method to create RPC service from configuration and hostname, port. + * + * @param configuration The configuration for the TaskManager. + * @param taskManagerHostname The hostname/address that describes the TaskManager's data location. + * @param actorSystemPort If true, the TaskManager will not initiate the TCP network stack. + * @return The rpc service which is used to start and connect to the TaskManager RpcEndpoint . + * @throws java.io.IOException Thrown, if the actor system can not bind to the address + * @throws java.lang.Exception Thrown is some other error occurs while creating akka actor system + */ + private static RpcService createRpcService(Configuration configuration, String taskManagerHostname, int actorSystemPort) + throws Exception{ + + // Bring up the TaskManager actor system first, bind it to the given address. + + LOG.info("Starting TaskManager actor system at " + + NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort)); + + final ActorSystem taskManagerSystem; + try { + Tuple2 address = new Tuple2(taskManagerHostname, actorSystemPort); + Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address)); + LOG.debug("Using akka configuration\n " + akkaConfig); + taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig); + } catch (Throwable t) { + if (t instanceof org.jboss.netty.channel.ChannelException) { + Throwable cause = t.getCause(); + if (cause != null && t.getCause() instanceof java.net.BindException) { + String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort); + throw new IOException("Unable to bind TaskManager actor system to address " + + address + " - " + cause.getMessage(), t); + } + } + throw new Exception("Could not create TaskManager actor system", t); + } + + // start akka rpc service based on actor system + final Timeout timeout = new Timeout(AkkaUtils.getTimeout(configuration).toMillis(), TimeUnit.MILLISECONDS); + final AkkaRpcService akkaRpcService = new AkkaRpcService(taskManagerSystem, timeout); + + return akkaRpcService; + } + + /** + * @param configuration The configuration for the TaskManager. + * @param resourceID The id of the resource which the task manager will run on. + * @param rpcService The rpc service which is used to start and connect to the TaskManager RpcEndpoint . + * @param taskManagerHostname The hostname/address that describes the TaskManager's data location. + * @param haServices Optionally, a high availability service can be provided. If none is given, + * then a HighAvailabilityServices is constructed from the configuration. + * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack. + * @throws IllegalConfigurationException Thrown, if the given config contains illegal values. + * @throws IOException Thrown, if any of the I/O components (such as buffer pools, I/O manager, ...) + * cannot be properly started. + * @throws Exception Thrown is some other error occurs while parsing the configuration or + * starting the TaskManager components. + */ + private static void createAndStartTaskManagerComponents( + Configuration configuration, + ResourceID resourceID, + RpcService rpcService, + String taskManagerHostname, + HighAvailabilityServices haServices, + boolean localTaskManagerCommunication) throws Exception { + + final TaskExecutorConfiguration taskManagerConfig = parseTaskManagerConfiguration( + configuration, taskManagerHostname, localTaskManagerCommunication); + + TaskManagerComponents taskManagerComponents = createTaskManagerComponents( + resourceID, + InetAddress.getByName(taskManagerHostname), + taskManagerConfig, + configuration); + + final TaskExecutor taskExecutor = new TaskExecutor( + taskManagerConfig, + taskManagerComponents.getTaskManagerLocation(), + rpcService, taskManagerComponents.getMemoryManager(), + taskManagerComponents.getIOManager(), + taskManagerComponents.getNetworkEnvironment(), + haServices); + + taskExecutor.start(); + } + + /** + * Creates and returns the task manager components. + * + * @param resourceID resource ID of the task manager + * @param taskManagerAddress address of the task manager + * @param taskExecutorConfig task manager configuration + * @param configuration of Flink + * @return task manager components + * @throws Exception + */ + private static TaskManagerComponents createTaskManagerComponents( + ResourceID resourceID, + InetAddress taskManagerAddress, + TaskExecutorConfiguration taskExecutorConfig, + Configuration configuration) throws Exception { + + MemoryType memType = taskExecutorConfig.getNetworkConfig().memoryType(); + + // pre-start checks + checkTempDirs(taskExecutorConfig.getTmpDirPaths()); + + NetworkEnvironmentConfiguration networkEnvironmentConfiguration = taskExecutorConfig.getNetworkConfig(); + + NetworkBufferPool networkBufferPool = new NetworkBufferPool( + networkEnvironmentConfiguration.numNetworkBuffers(), + networkEnvironmentConfiguration.networkBufferSize(), + networkEnvironmentConfiguration.memoryType()); + + ConnectionManager connectionManager; + + if (networkEnvironmentConfiguration.nettyConfig().isDefined()) { + connectionManager = new NettyConnectionManager(networkEnvironmentConfiguration.nettyConfig().get()); + } else { + connectionManager = new LocalConnectionManager(); + } + + ResultPartitionManager resultPartitionManager = new ResultPartitionManager(); + TaskEventDispatcher taskEventDispatcher = new TaskEventDispatcher(); + + KvStateRegistry kvStateRegistry = new KvStateRegistry(); + + KvStateServer kvStateServer; + + if (networkEnvironmentConfiguration.nettyConfig().isDefined()) { + NettyConfig nettyConfig = networkEnvironmentConfiguration.nettyConfig().get(); + + int numNetworkThreads = networkEnvironmentConfiguration.queryServerNetworkThreads() == 0 ? + nettyConfig.getNumberOfSlots() : networkEnvironmentConfiguration.queryServerNetworkThreads(); + + int numQueryThreads = networkEnvironmentConfiguration.queryServerQueryThreads() == 0 ? + nettyConfig.getNumberOfSlots() : networkEnvironmentConfiguration.queryServerQueryThreads(); + + kvStateServer = new KvStateServer( + taskManagerAddress, + networkEnvironmentConfiguration.queryServerPort(), + numNetworkThreads, + numQueryThreads, + kvStateRegistry, + new DisabledKvStateRequestStats()); + } else { + kvStateServer = null; + } + + // we start the network first, to make sure it can allocate its buffers first + final NetworkEnvironment network = new NetworkEnvironment( + networkBufferPool, + connectionManager, + resultPartitionManager, + taskEventDispatcher, + kvStateRegistry, + kvStateServer, + networkEnvironmentConfiguration.ioMode(), + networkEnvironmentConfiguration.partitionRequestInitialBackoff(), + networkEnvironmentConfiguration.partitinRequestMaxBackoff()); + + network.start(); + + final TaskManagerLocation taskManagerLocation = new TaskManagerLocation( + resourceID, + taskManagerAddress, + network.getConnectionManager().getDataPort()); + + // computing the amount of memory to use depends on how much memory is available + // it strictly needs to happen AFTER the network stack has been initialized + + // check if a value has been configured + long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L); + checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory, + ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, + "MemoryManager needs at least one MB of memory. " + + "If you leave this config parameter empty, the system automatically " + + "pick a fraction of the available memory."); + + final long memorySize; + boolean preAllocateMemory = configuration.getBoolean( + ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY, + ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE); + if (configuredMemory > 0) { + if (preAllocateMemory) { + LOG.info("Using {} MB for managed memory." , configuredMemory); + } else { + LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory); + } + memorySize = configuredMemory << 20; // megabytes to bytes + } else { + float fraction = configuration.getFloat( + ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY, + ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION); + checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction, + ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY, + "MemoryManager fraction of the free memory must be between 0.0 and 1.0"); + + if (memType == MemoryType.HEAP) { + long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction); + if (preAllocateMemory) { + LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." , + fraction , relativeMemSize >> 20); + } else { + LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " + + "memory will be allocated lazily." , fraction , relativeMemSize >> 20); + } + memorySize = relativeMemSize; + } else if (memType == MemoryType.OFF_HEAP) { + // The maximum heap memory has been adjusted according to the fraction + long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory(); + long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction); + if (preAllocateMemory) { + LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." , + fraction, directMemorySize >> 20); + } else { + LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," + + " memory will be allocated lazily.", fraction, directMemorySize >> 20); + } + memorySize = directMemorySize; + } else { + throw new RuntimeException("No supported memory type detected."); + } + } + + // now start the memory manager + final MemoryManager memoryManager; + try { + memoryManager = new MemoryManager( + memorySize, + taskExecutorConfig.getNumberOfSlots(), + taskExecutorConfig.getNetworkConfig().networkBufferSize(), + memType, + preAllocateMemory); + } catch (OutOfMemoryError e) { + if (memType == MemoryType.HEAP) { + throw new Exception("OutOfMemory error (" + e.getMessage() + + ") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e); + } else if (memType == MemoryType.OFF_HEAP) { + throw new Exception("OutOfMemory error (" + e.getMessage() + + ") while allocating the TaskManager off-heap memory (" + memorySize + + " bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e); + } else { + throw e; + } + } + + // start the I/O manager, it will create some temp directories. + final IOManager ioManager = new IOManagerAsync(taskExecutorConfig.getTmpDirPaths()); + + return new TaskManagerComponents(taskManagerLocation, memoryManager, ioManager, network); + } + + // -------------------------------------------------------------------------- + // Parsing and checking the TaskManager Configuration + // -------------------------------------------------------------------------- + + /** + * Utility method to extract TaskManager config parameters from the configuration and to + * sanity check them. + * + * @param configuration The configuration. + * @param taskManagerHostname The host name under which the TaskManager communicates. + * @param localTaskManagerCommunication True, to skip initializing the network stack. + * Use only in cases where only one task manager runs. + * @return TaskExecutorConfiguration that wrappers InstanceConnectionInfo, NetworkEnvironmentConfiguration, etc. + */ + private static TaskExecutorConfiguration parseTaskManagerConfiguration( + Configuration configuration, + String taskManagerHostname, + boolean localTaskManagerCommunication) throws Exception { + + // ------- read values from the config and check them --------- + // (a lot of them) + + // ----> hosts / ports for communication and data exchange + + int dataport = configuration.getInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, + ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT); + + checkConfigParameter(dataport > 0, dataport, ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, + "Leave config parameter empty or use 0 to let the system choose a port automatically."); + + InetAddress taskManagerAddress = InetAddress.getByName(taskManagerHostname); + final InetSocketAddress taskManagerInetSocketAddress = new InetSocketAddress(taskManagerAddress, dataport); + + // ----> memory / network stack (shuffles/broadcasts), task slots, temp directories + + // we need this because many configs have been written with a "-1" entry + int slots = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1); + if (slots == -1) { + slots = 1; + } + + checkConfigParameter(slots >= 1, slots, ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, + "Number of task slots must be at least one."); + + final int numNetworkBuffers = configuration.getInteger( + ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, + ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS); + + checkConfigParameter(numNetworkBuffers > 0, numNetworkBuffers, + ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, ""); + + final int pageSize = configuration.getInteger( + ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY, + ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE); + + // check page size of for minimum size + checkConfigParameter(pageSize >= MemoryManager.MIN_PAGE_SIZE, pageSize, + ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY, + "Minimum memory segment size is " + MemoryManager.MIN_PAGE_SIZE); + + // check page size for power of two + checkConfigParameter(MathUtils.isPowerOf2(pageSize), pageSize, + ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY, + "Memory segment size must be a power of 2."); + + // check whether we use heap or off-heap memory + final MemoryType memType; + if (configuration.getBoolean(ConfigConstants.TASK_MANAGER_MEMORY_OFF_HEAP_KEY, false)) { + memType = MemoryType.OFF_HEAP; + } else { + memType = MemoryType.HEAP; + } + + // initialize the memory segment factory accordingly + if (memType == MemoryType.HEAP) { + if (!MemorySegmentFactory.initializeIfNotInitialized(HeapMemorySegment.FACTORY)) { + throw new Exception("Memory type is set to heap memory, but memory segment " + + "factory has been initialized for off-heap memory segments"); + } + } else { + if (!MemorySegmentFactory.initializeIfNotInitialized(HybridMemorySegment.FACTORY)) { + throw new Exception("Memory type is set to off-heap memory, but memory segment " + + "factory has been initialized for heap memory segments"); + } + } + + final String[] tmpDirs = configuration.getString( + ConfigConstants.TASK_MANAGER_TMP_DIR_KEY, + ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH).split(",|" + File.pathSeparator); + + final NettyConfig nettyConfig; + if (!localTaskManagerCommunication) { + nettyConfig = new NettyConfig(taskManagerInetSocketAddress.getAddress(), + taskManagerInetSocketAddress.getPort(), pageSize, slots, configuration); + } else { + nettyConfig = null; + } + + // Default spill I/O mode for intermediate results + final String syncOrAsync = configuration.getString( + ConfigConstants.TASK_MANAGER_NETWORK_DEFAULT_IO_MODE, + ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_DEFAULT_IO_MODE); + + final IOManager.IOMode ioMode; + if (syncOrAsync.equals("async")) { + ioMode = IOManager.IOMode.ASYNC; + } else { + ioMode = IOManager.IOMode.SYNC; + } + + final int queryServerPort = configuration.getInteger( + ConfigConstants.QUERYABLE_STATE_SERVER_PORT, + ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_PORT); + + final int queryServerNetworkThreads = configuration.getInteger( + ConfigConstants.QUERYABLE_STATE_SERVER_NETWORK_THREADS, + ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_NETWORK_THREADS); + + final int queryServerQueryThreads = configuration.getInteger( + ConfigConstants.QUERYABLE_STATE_SERVER_QUERY_THREADS, + ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_QUERY_THREADS); + + final NetworkEnvironmentConfiguration networkConfig = new NetworkEnvironmentConfiguration( + numNetworkBuffers, + pageSize, + memType, + ioMode, + queryServerPort, + queryServerNetworkThreads, + queryServerQueryThreads, + Option.apply(nettyConfig), + 500, + 3000); + + // ----> timeouts, library caching, profiling + + final FiniteDuration timeout; + try { + timeout = AkkaUtils.getTimeout(configuration); + } catch (Exception e) { + throw new IllegalArgumentException( + "Invalid format for '" + ConfigConstants.AKKA_ASK_TIMEOUT + + "'.Use formats like '50 s' or '1 min' to specify the timeout."); + } + LOG.info("Messages between TaskManager and JobManager have a max timeout of " + timeout); + + final long cleanupInterval = configuration.getLong( + ConfigConstants.LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL, + ConfigConstants.DEFAULT_LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL) * 1000; + + final FiniteDuration finiteRegistrationDuration; + try { + Duration maxRegistrationDuration = Duration.create(configuration.getString( + ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION, + ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_DURATION)); + if (maxRegistrationDuration.isFinite()) { + finiteRegistrationDuration = new FiniteDuration(maxRegistrationDuration.toSeconds(), TimeUnit.SECONDS); + } else { + finiteRegistrationDuration = null; + } + } catch (NumberFormatException e) { + throw new IllegalArgumentException("Invalid format for parameter " + + ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION, e); + } + + final FiniteDuration initialRegistrationPause; + try { + Duration pause = Duration.create(configuration.getString( + ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, + ConfigConstants.DEFAULT_TASK_MANAGER_INITIAL_REGISTRATION_PAUSE)); + if (pause.isFinite()) { + initialRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS); + } else { + throw new IllegalArgumentException("The initial registration pause must be finite: " + pause); + } + } catch (NumberFormatException e) { + throw new IllegalArgumentException("Invalid format for parameter " + + ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e); + } + + final FiniteDuration maxRegistrationPause; + try { + Duration pause = Duration.create(configuration.getString( + ConfigConstants.TASK_MANAGER_MAX_REGISTARTION_PAUSE, + ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_PAUSE)); + if (pause.isFinite()) { + maxRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS); + } else { + throw new IllegalArgumentException("The maximum registration pause must be finite: " + pause); + } + } catch (NumberFormatException e) { + throw new IllegalArgumentException("Invalid format for parameter " + + ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e); + } + + final FiniteDuration refusedRegistrationPause; + try { + Duration pause = Duration.create(configuration.getString( + ConfigConstants.TASK_MANAGER_REFUSED_REGISTRATION_PAUSE, + ConfigConstants.DEFAULT_TASK_MANAGER_REFUSED_REGISTRATION_PAUSE)); + if (pause.isFinite()) { + refusedRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS); + } else { + throw new IllegalArgumentException("The refused registration pause must be finite: " + pause); + } + } catch (NumberFormatException e) { + throw new IllegalArgumentException("Invalid format for parameter " + + ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e); + } + + return new TaskExecutorConfiguration( + tmpDirs, + cleanupInterval, + networkConfig, + timeout, + finiteRegistrationDuration, + slots, + configuration, + initialRegistrationPause, + maxRegistrationPause, + refusedRegistrationPause); + } + + /** + * Validates a condition for a config parameter and displays a standard exception, if the + * the condition does not hold. + * + * @param condition The condition that must hold. If the condition is false, an exception is thrown. + * @param parameter The parameter value. Will be shown in the exception message. + * @param name The name of the config parameter. Will be shown in the exception message. + * @param errorMessage The optional custom error message to append to the exception message. + */ + private static void checkConfigParameter( + boolean condition, + Object parameter, + String name, + String errorMessage) { + if (!condition) { + throw new IllegalConfigurationException("Invalid configuration value for " + name + " : " + parameter + " - " + errorMessage); + } + } + + /** + * Validates that all the directories denoted by the strings do actually exist, are proper + * directories (not files), and are writable. + * + * @param tmpDirs The array of directory paths to check. + * @throws Exception Thrown if any of the directories does not exist or is not writable + * or is a file, rather than a directory. + */ + private static void checkTempDirs(String[] tmpDirs) throws IOException { + for (String dir : tmpDirs) { + if (dir != null && !dir.equals("")) { + File file = new File(dir); + if (!file.exists()) { + throw new IOException("Temporary file directory " + file.getAbsolutePath() + " does not exist."); + } + if (!file.isDirectory()) { + throw new IOException("Temporary file directory " + file.getAbsolutePath() + " is not a directory."); + } + if (!file.canWrite()) { + throw new IOException("Temporary file directory " + file.getAbsolutePath() + " is not writable."); + } + + if (LOG.isInfoEnabled()) { + long totalSpaceGb = file.getTotalSpace() >> 30; + long usableSpaceGb = file.getUsableSpace() >> 30; + double usablePercentage = (double)usableSpaceGb / totalSpaceGb * 100; + String path = file.getAbsolutePath(); + LOG.info(String.format("Temporary file directory '%s': total %d GB, " + "usable %d GB (%.2f%% usable)", + path, totalSpaceGb, usableSpaceGb, usablePercentage)); + } + } else { + throw new IllegalArgumentException("Temporary file directory #$id is null."); + } + } + } + + private static class TaskManagerComponents { + private final TaskManagerLocation taskManagerLocation; + private final MemoryManager memoryManager; + private final IOManager ioManager; + private final NetworkEnvironment networkEnvironment; + + private TaskManagerComponents( + TaskManagerLocation taskManagerLocation, + MemoryManager memoryManager, + IOManager ioManager, + NetworkEnvironment networkEnvironment) { + + this.taskManagerLocation = Preconditions.checkNotNull(taskManagerLocation); + this.memoryManager = Preconditions.checkNotNull(memoryManager); + this.ioManager = Preconditions.checkNotNull(ioManager); + this.networkEnvironment = Preconditions.checkNotNull(networkEnvironment); + } + + public MemoryManager getMemoryManager() { + return memoryManager; + } + + public IOManager getIOManager() { + return ioManager; + } + + public NetworkEnvironment getNetworkEnvironment() { + return networkEnvironment; + } + + public TaskManagerLocation getTaskManagerLocation() { + return taskManagerLocation; + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java index 09aab183782e2..26218dd22387c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java @@ -19,17 +19,22 @@ package org.apache.flink.runtime.taskexecutor; import org.apache.flink.api.common.time.Time; -import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.highavailability.NonHaServices; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.io.network.NetworkEnvironment; import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService; +import org.apache.flink.runtime.memory.MemoryManager; import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.rpc.TestingSerialRpcService; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.util.TestLogger; import org.junit.Test; +import org.powermock.api.mockito.PowerMockito; import java.util.UUID; import static org.junit.Assert.*; @@ -42,19 +47,31 @@ public void testImmediatelyRegistersIfLeaderIsKnown() throws Exception { final ResourceID resourceID = ResourceID.generate(); final String resourceManagerAddress = "/resource/manager/address/one"; - final TestingRpcService rpc = new TestingRpcService(); + final TestingSerialRpcService rpc = new TestingSerialRpcService(); try { // register a mock resource manager gateway ResourceManagerGateway rmGateway = mock(ResourceManagerGateway.class); + TaskExecutorConfiguration taskExecutorConfiguration = mock(TaskExecutorConfiguration.class); + PowerMockito.when(taskExecutorConfiguration.getNumberOfSlots()).thenReturn(1); rpc.registerGateway(resourceManagerAddress, rmGateway); + TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class); + when(taskManagerLocation.getResourceID()).thenReturn(resourceID); + NonHaServices haServices = new NonHaServices(resourceManagerAddress); - TaskExecutor taskManager = TaskExecutor.startTaskManagerComponentsAndActor( - new Configuration(), resourceID, rpc, "localhost", haServices, true); - String taskManagerAddress = taskManager.getAddress(); + + TaskExecutor taskManager = new TaskExecutor( + taskExecutorConfiguration, + taskManagerLocation, + rpc, mock(MemoryManager.class), + mock(IOManager.class), + mock(NetworkEnvironment.class), + haServices); + taskManager.start(); + String taskManagerAddress = taskManager.getAddress(); - verify(rmGateway, timeout(5000)).registerTaskExecutor( + verify(rmGateway).registerTaskExecutor( any(UUID.class), eq(taskManagerAddress), eq(resourceID), any(Time.class)); } finally { @@ -71,7 +88,7 @@ public void testTriggerRegistrationOnLeaderChange() throws Exception { final UUID leaderId1 = UUID.randomUUID(); final UUID leaderId2 = UUID.randomUUID(); - final TestingRpcService rpc = new TestingRpcService(); + final TestingSerialRpcService rpc = new TestingSerialRpcService(); try { // register the mock resource manager gateways ResourceManagerGateway rmGateway1 = mock(ResourceManagerGateway.class); @@ -84,10 +101,22 @@ public void testTriggerRegistrationOnLeaderChange() throws Exception { TestingHighAvailabilityServices haServices = new TestingHighAvailabilityServices(); haServices.setResourceManagerLeaderRetriever(testLeaderService); - TaskExecutor taskManager = TaskExecutor.startTaskManagerComponentsAndActor( - new Configuration(), resourceID, rpc, "localhost", haServices, true); - String taskManagerAddress = taskManager.getAddress(); + TaskExecutorConfiguration taskExecutorConfiguration = mock(TaskExecutorConfiguration.class); + PowerMockito.when(taskExecutorConfiguration.getNumberOfSlots()).thenReturn(1); + + TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class); + when(taskManagerLocation.getResourceID()).thenReturn(resourceID); + + TaskExecutor taskManager = new TaskExecutor( + taskExecutorConfiguration, + taskManagerLocation, + rpc, mock(MemoryManager.class), + mock(IOManager.class), + mock(NetworkEnvironment.class), + haServices); + taskManager.start(); + String taskManagerAddress = taskManager.getAddress(); // no connection initially, since there is no leader assertNull(taskManager.getResourceManagerConnection()); @@ -95,7 +124,7 @@ public void testTriggerRegistrationOnLeaderChange() throws Exception { // define a leader and see that a registration happens testLeaderService.notifyListener(address1, leaderId1); - verify(rmGateway1, timeout(5000)).registerTaskExecutor( + verify(rmGateway1).registerTaskExecutor( eq(leaderId1), eq(taskManagerAddress), eq(resourceID), any(Time.class)); assertNotNull(taskManager.getResourceManagerConnection()); @@ -105,7 +134,7 @@ public void testTriggerRegistrationOnLeaderChange() throws Exception { // set a new leader, see that a registration happens testLeaderService.notifyListener(address2, leaderId2); - verify(rmGateway2, timeout(5000)).registerTaskExecutor( + verify(rmGateway2).registerTaskExecutor( eq(leaderId2), eq(taskManagerAddress), eq(resourceID), any(Time.class)); assertNotNull(taskManager.getResourceManagerConnection()); } From 5513fe61dc8699a617c163ee2a555e43dc6422e4 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 21 Sep 2016 12:33:15 +0200 Subject: [PATCH 43/50] [FLINK-4505] [cluster mngt] Separate TaskManager service configuration from TaskManagerConfiguration; Implement TaskManagerRunner Refactors the startup logic so that is easier to reuse. This closes #2461. --- .../HighAvailabilityServicesUtils.java | 41 + .../flink/runtime/rpc/RpcServiceUtils.java | 73 ++ .../runtime/rpc/akka/AkkaRpcService.java | 2 + .../runtime/taskexecutor/TaskExecutor.java | 51 +- .../TaskExecutorConfiguration.java | 142 ---- .../TaskManagerConfiguration.java | 205 +++++ .../taskexecutor/TaskManagerRunner.java | 172 ++++ .../taskexecutor/TaskManagerServices.java | 320 ++++++++ .../TaskManagerServicesConfiguration.java | 325 ++++++++ .../taskmanager/TaskManagerRunner.java | 749 ------------------ .../runtime/util/LeaderRetrievalUtils.java | 7 + .../apache/flink/runtime/akka/AkkaUtils.scala | 4 + .../NetworkEnvironmentConfiguration.scala | 2 +- .../runtime/taskmanager/TaskManager.scala | 6 +- .../io/network/NetworkEnvironmentTest.java | 4 +- .../runtime/rpc/TestingSerialRpcService.java | 1 - .../taskexecutor/TaskExecutorTest.java | 29 +- ...kManagerComponentsStartupShutdownTest.java | 3 +- .../TaskManagerConfigurationTest.java | 1 - 19 files changed, 1195 insertions(+), 942 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServicesUtils.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcServiceUtils.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorConfiguration.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerRunner.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServicesUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServicesUtils.java new file mode 100644 index 0000000000000..f3da84732b178 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServicesUtils.java @@ -0,0 +1,41 @@ +/* + * 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.highavailability; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; +import org.apache.flink.runtime.util.LeaderRetrievalUtils; + +public class HighAvailabilityServicesUtils { + + public static HighAvailabilityServices createHighAvailabilityServices(Configuration configuration) throws Exception { + HighAvailabilityMode highAvailabilityMode = LeaderRetrievalUtils.getRecoveryMode(configuration); + + switch(highAvailabilityMode) { + case NONE: + final String resourceManagerAddress = null; + return new NonHaServices(resourceManagerAddress); + case ZOOKEEPER: + throw new UnsupportedOperationException("ZooKeeper high availability services " + + "have not been implemented yet."); + default: + throw new Exception("Recovery mode " + highAvailabilityMode + " is not supported."); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcServiceUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcServiceUtils.java new file mode 100644 index 0000000000000..d40e336c3a346 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcServiceUtils.java @@ -0,0 +1,73 @@ +/* + * 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.rpc; + +import akka.actor.ActorSystem; +import com.typesafe.config.Config; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.rpc.akka.AkkaRpcService; +import org.apache.flink.util.NetUtils; +import org.jboss.netty.channel.ChannelException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; + +public class RpcServiceUtils { + private static final Logger LOG = LoggerFactory.getLogger(RpcServiceUtils.class); + + /** + * Utility method to create RPC service from configuration and hostname, port. + * + * @param hostname The hostname/address that describes the TaskManager's data location. + * @param port If true, the TaskManager will not initiate the TCP network stack. + * @param configuration The configuration for the TaskManager. + * @return The rpc service which is used to start and connect to the TaskManager RpcEndpoint . + * @throws IOException Thrown, if the actor system can not bind to the address + * @throws Exception Thrown is some other error occurs while creating akka actor system + */ + public static RpcService createRpcService(String hostname, int port, Configuration configuration) throws Exception { + LOG.info("Starting AkkaRpcService at {}.", NetUtils.hostAndPortToUrlString(hostname, port)); + + final ActorSystem actorSystem; + + try { + Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, hostname, port); + + LOG.debug("Using akka configuration \n {}.", akkaConfig); + + actorSystem = AkkaUtils.createActorSystem(akkaConfig); + } catch (Throwable t) { + if (t instanceof ChannelException) { + Throwable cause = t.getCause(); + if (cause != null && t.getCause() instanceof java.net.BindException) { + String address = NetUtils.hostAndPortToUrlString(hostname, port); + throw new IOException("Unable to bind AkkaRpcService actor system to address " + + address + " - " + cause.getMessage(), t); + } + } + throw new Exception("Could not create TaskManager actor system", t); + } + + final Time timeout = Time.milliseconds(AkkaUtils.getTimeout(configuration).toMillis()); + return new AkkaRpcService(actorSystem, timeout); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java index 6825557199d4e..fb7896aa15842 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java @@ -93,6 +93,8 @@ public AkkaRpcService(final ActorSystem actorSystem, final Time timeout) { Address actorSystemAddress = AkkaUtils.getAddress(actorSystem); + + if (actorSystemAddress.host().isDefined()) { address = actorSystemAddress.host().get(); } else { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java index 8ce278076c856..7df0a911e7735 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java @@ -18,16 +18,14 @@ package org.apache.flink.runtime.taskexecutor; -import org.apache.flink.api.common.time.Time; -import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; import org.apache.flink.runtime.clusterframework.types.AllocationID; -import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.resourcemanager.SlotRequestRegistered; import org.apache.flink.runtime.resourcemanager.SlotRequestReply; +import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; -import org.jboss.netty.channel.ChannelException; import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.network.NetworkEnvironment; @@ -39,7 +37,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.net.BindException; + import java.util.UUID; import static org.apache.flink.util.Preconditions.checkArgument; @@ -60,7 +58,7 @@ public class TaskExecutor extends RpcEndpoint { private final HighAvailabilityServices haServices; /** The task manager configuration */ - private final TaskExecutorConfiguration taskExecutorConfig; + private final TaskManagerConfiguration taskManagerConfiguration; /** The I/O manager component in the task manager */ private final IOManager ioManager; @@ -71,9 +69,15 @@ public class TaskExecutor extends RpcEndpoint { /** The network component in the task manager */ private final NetworkEnvironment networkEnvironment; + /** The metric registry in the task manager */ + private final MetricRegistry metricRegistry; + /** The number of slots in the task manager, should be 1 for YARN */ private final int numberOfSlots; + /** The fatal error handler to use in case of a fatal error */ + private final FatalErrorHandler fatalErrorHandler; + // --------- resource manager -------- private TaskExecutorToResourceManagerConnection resourceManagerConnection; @@ -81,26 +85,30 @@ public class TaskExecutor extends RpcEndpoint { // ------------------------------------------------------------------------ public TaskExecutor( - TaskExecutorConfiguration taskExecutorConfig, + TaskManagerConfiguration taskManagerConfiguration, TaskManagerLocation taskManagerLocation, RpcService rpcService, MemoryManager memoryManager, IOManager ioManager, NetworkEnvironment networkEnvironment, - HighAvailabilityServices haServices) { + HighAvailabilityServices haServices, + MetricRegistry metricRegistry, + FatalErrorHandler fatalErrorHandler) { super(rpcService); - checkArgument(taskExecutorConfig.getNumberOfSlots() > 0, "The number of slots has to be larger than 0."); + checkArgument(taskManagerConfiguration.getNumberSlots() > 0, "The number of slots has to be larger than 0."); - this.taskExecutorConfig = checkNotNull(taskExecutorConfig); + this.taskManagerConfiguration = checkNotNull(taskManagerConfiguration); this.taskManagerLocation = checkNotNull(taskManagerLocation); this.memoryManager = checkNotNull(memoryManager); this.ioManager = checkNotNull(ioManager); this.networkEnvironment = checkNotNull(networkEnvironment); this.haServices = checkNotNull(haServices); + this.metricRegistry = checkNotNull(metricRegistry); + this.fatalErrorHandler = checkNotNull(fatalErrorHandler); - this.numberOfSlots = taskExecutorConfig.getNumberOfSlots(); + this.numberOfSlots = taskManagerConfiguration.getNumberSlots(); } // ------------------------------------------------------------------------ @@ -158,6 +166,7 @@ public void notifyOfNewResourceManagerLeader(String newLeaderAddress, UUID newLe } } + /** * Requests a slot from the TaskManager * * @param allocationID id for the request @@ -169,22 +178,6 @@ public SlotRequestReply requestSlot(AllocationID allocationID, UUID resourceMana return new SlotRequestRegistered(allocationID); } - /** - public LeaderRetrievalService getJobMasterLeaderRetriever(JobID jobID) throws Exception { - return null; - } - - @Override - return null; - } - - @Override - public CheckpointRecoveryFactory getCheckpointRecoveryFactory() throws Exception { - return null; - } - - @Override - public SubmittedJobGraphStore getSubmittedJobGraphStore() throws Exception { // ------------------------------------------------------------------------ // Properties // ------------------------------------------------------------------------ @@ -222,7 +215,7 @@ public void run() { void onFatalError(Throwable t) { // to be determined, probably delegate to a fatal error handler that // would either log (mini cluster) ot kill the process (yarn, mesos, ...) - log.error("FATAL ERROR", t); + fatalErrorHandler.onFatalError(t); } // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorConfiguration.java deleted file mode 100644 index c97c893530a45..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorConfiguration.java +++ /dev/null @@ -1,142 +0,0 @@ -/* - * 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.taskexecutor; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration; - -import scala.concurrent.duration.FiniteDuration; - -import java.io.Serializable; -import java.util.concurrent.TimeUnit; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * {@link TaskExecutor} Configuration - */ -public class TaskExecutorConfiguration implements Serializable { - - private static final long serialVersionUID = 1L; - - private final String[] tmpDirPaths; - - private final long cleanupInterval; - - private final int numberOfSlots; - - private final Configuration configuration; - - private final FiniteDuration timeout; - private final FiniteDuration maxRegistrationDuration; - private final FiniteDuration initialRegistrationPause; - private final FiniteDuration maxRegistrationPause; - private final FiniteDuration refusedRegistrationPause; - - private final NetworkEnvironmentConfiguration networkConfig; - - public TaskExecutorConfiguration( - String[] tmpDirPaths, - long cleanupInterval, - NetworkEnvironmentConfiguration networkConfig, - FiniteDuration timeout, - FiniteDuration maxRegistrationDuration, - int numberOfSlots, - Configuration configuration) { - - this (tmpDirPaths, - cleanupInterval, - networkConfig, - timeout, - maxRegistrationDuration, - numberOfSlots, - configuration, - new FiniteDuration(500, TimeUnit.MILLISECONDS), - new FiniteDuration(30, TimeUnit.SECONDS), - new FiniteDuration(10, TimeUnit.SECONDS)); - } - - public TaskExecutorConfiguration( - String[] tmpDirPaths, - long cleanupInterval, - NetworkEnvironmentConfiguration networkConfig, - FiniteDuration timeout, - FiniteDuration maxRegistrationDuration, - int numberOfSlots, - Configuration configuration, - FiniteDuration initialRegistrationPause, - FiniteDuration maxRegistrationPause, - FiniteDuration refusedRegistrationPause) { - - this.tmpDirPaths = checkNotNull(tmpDirPaths); - this.cleanupInterval = checkNotNull(cleanupInterval); - this.networkConfig = checkNotNull(networkConfig); - this.timeout = checkNotNull(timeout); - this.maxRegistrationDuration = maxRegistrationDuration; - this.numberOfSlots = checkNotNull(numberOfSlots); - this.configuration = checkNotNull(configuration); - this.initialRegistrationPause = checkNotNull(initialRegistrationPause); - this.maxRegistrationPause = checkNotNull(maxRegistrationPause); - this.refusedRegistrationPause = checkNotNull(refusedRegistrationPause); - } - - // -------------------------------------------------------------------------------------------- - // Properties - // -------------------------------------------------------------------------------------------- - - public String[] getTmpDirPaths() { - return tmpDirPaths; - } - - public long getCleanupInterval() { - return cleanupInterval; - } - - public NetworkEnvironmentConfiguration getNetworkConfig() { return networkConfig; } - - public FiniteDuration getTimeout() { - return timeout; - } - - public FiniteDuration getMaxRegistrationDuration() { - return maxRegistrationDuration; - } - - public int getNumberOfSlots() { - return numberOfSlots; - } - - public Configuration getConfiguration() { - return configuration; - } - - public FiniteDuration getInitialRegistrationPause() { - return initialRegistrationPause; - } - - public FiniteDuration getMaxRegistrationPause() { - return maxRegistrationPause; - } - - public FiniteDuration getRefusedRegistrationPause() { - return refusedRegistrationPause; - } - -} - diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java new file mode 100644 index 0000000000000..32eb8c17325f3 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java @@ -0,0 +1,205 @@ +/* + * 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.taskexecutor; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.util.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.concurrent.duration.Duration; + +import java.io.File; + +/** + * Configuration object for {@link TaskExecutor}. + */ +public class TaskManagerConfiguration { + + private static final Logger LOG = LoggerFactory.getLogger(TaskManagerConfiguration.class); + + private final int numberSlots; + + private final String[] tmpDirPaths; + + private final Time timeout; + private final Time maxRegistrationDuration; + private final Time initialRegistrationPause; + private final Time maxRegistrationPause; + private final Time refusedRegistrationPause; + + private final long cleanupInterval; + + public TaskManagerConfiguration( + int numberSlots, + String[] tmpDirPaths, + Time timeout, + Time maxRegistrationDuration, + Time initialRegistrationPause, + Time maxRegistrationPause, + Time refusedRegistrationPause, + long cleanupInterval) { + + this.numberSlots = numberSlots; + this.tmpDirPaths = Preconditions.checkNotNull(tmpDirPaths); + this.timeout = Preconditions.checkNotNull(timeout); + this.maxRegistrationDuration = Preconditions.checkNotNull(maxRegistrationDuration); + this.initialRegistrationPause = Preconditions.checkNotNull(initialRegistrationPause); + this.maxRegistrationPause = Preconditions.checkNotNull(maxRegistrationPause); + this.refusedRegistrationPause = Preconditions.checkNotNull(refusedRegistrationPause); + this.cleanupInterval = Preconditions.checkNotNull(cleanupInterval); + } + + public int getNumberSlots() { + return numberSlots; + } + + public String[] getTmpDirPaths() { + return tmpDirPaths; + } + + public Time getTimeout() { + return timeout; + } + + public Time getMaxRegistrationDuration() { + return maxRegistrationDuration; + } + + public Time getInitialRegistrationPause() { + return initialRegistrationPause; + } + + public Time getMaxRegistrationPause() { + return maxRegistrationPause; + } + + public Time getRefusedRegistrationPause() { + return refusedRegistrationPause; + } + + public long getCleanupInterval() { + return cleanupInterval; + } + + // -------------------------------------------------------------------------------------------- + // Static factory methods + // -------------------------------------------------------------------------------------------- + + public static TaskManagerConfiguration fromConfiguration(Configuration configuration) { + int numberSlots = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1); + + if (numberSlots == -1) { + numberSlots = 1; + } + + final String[] tmpDirPaths = configuration.getString( + ConfigConstants.TASK_MANAGER_TMP_DIR_KEY, + ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH).split(",|" + File.pathSeparator); + + final Time timeout; + + try { + timeout = Time.milliseconds(AkkaUtils.getTimeout(configuration).toMillis()); + } catch (Exception e) { + throw new IllegalArgumentException( + "Invalid format for '" + ConfigConstants.AKKA_ASK_TIMEOUT + + "'.Use formats like '50 s' or '1 min' to specify the timeout."); + } + + LOG.info("Messages have a max timeout of " + timeout); + + final long cleanupInterval = configuration.getLong( + ConfigConstants.LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL, + ConfigConstants.DEFAULT_LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL) * 1000; + + final Time finiteRegistrationDuration; + + try { + Duration maxRegistrationDuration = Duration.create(configuration.getString( + ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION, + ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_DURATION)); + if (maxRegistrationDuration.isFinite()) { + finiteRegistrationDuration = Time.seconds(maxRegistrationDuration.toSeconds()); + } else { + finiteRegistrationDuration = null; + } + } catch (NumberFormatException e) { + throw new IllegalArgumentException("Invalid format for parameter " + + ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION, e); + } + + final Time initialRegistrationPause; + try { + Duration pause = Duration.create(configuration.getString( + ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, + ConfigConstants.DEFAULT_TASK_MANAGER_INITIAL_REGISTRATION_PAUSE)); + if (pause.isFinite()) { + initialRegistrationPause = Time.seconds(pause.toSeconds()); + } else { + throw new IllegalArgumentException("The initial registration pause must be finite: " + pause); + } + } catch (NumberFormatException e) { + throw new IllegalArgumentException("Invalid format for parameter " + + ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e); + } + + final Time maxRegistrationPause; + try { + Duration pause = Duration.create(configuration.getString( + ConfigConstants.TASK_MANAGER_MAX_REGISTARTION_PAUSE, + ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_PAUSE)); + if (pause.isFinite()) { + maxRegistrationPause = Time.seconds(pause.toSeconds()); + } else { + throw new IllegalArgumentException("The maximum registration pause must be finite: " + pause); + } + } catch (NumberFormatException e) { + throw new IllegalArgumentException("Invalid format for parameter " + + ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e); + } + + final Time refusedRegistrationPause; + try { + Duration pause = Duration.create(configuration.getString( + ConfigConstants.TASK_MANAGER_REFUSED_REGISTRATION_PAUSE, + ConfigConstants.DEFAULT_TASK_MANAGER_REFUSED_REGISTRATION_PAUSE)); + if (pause.isFinite()) { + refusedRegistrationPause = Time.seconds(pause.toSeconds()); + } else { + throw new IllegalArgumentException("The refused registration pause must be finite: " + pause); + } + } catch (NumberFormatException e) { + throw new IllegalArgumentException("Invalid format for parameter " + + ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e); + } + + return new TaskManagerConfiguration( + numberSlots, + tmpDirPaths, + timeout, + finiteRegistrationDuration, + initialRegistrationPause, + maxRegistrationPause, + refusedRegistrationPause, + cleanupInterval); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java new file mode 100644 index 0000000000000..8ac0ddd5b4e06 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java @@ -0,0 +1,172 @@ +/* + * 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.taskexecutor; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.rpc.RpcServiceUtils; +import org.apache.flink.runtime.util.LeaderRetrievalUtils; + +import org.apache.flink.util.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.net.InetAddress; +import java.util.concurrent.Executor; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * This class is the executable entry point for the task manager in yarn or standalone mode. + * It constructs the related components (network, I/O manager, memory manager, RPC service, HA service) + * and starts them. + */ +public class TaskManagerRunner implements FatalErrorHandler { + + private static final Logger LOG = LoggerFactory.getLogger(TaskManagerRunner.class); + + private final Object lock = new Object(); + + private final Configuration configuration; + + private final ResourceID resourceID; + + private final RpcService rpcService; + + private final HighAvailabilityServices highAvailabilityServices; + + /** Executor used to run future callbacks */ + private final Executor executor; + + private final TaskExecutor taskManager; + + public TaskManagerRunner( + Configuration configuration, + ResourceID resourceID, + RpcService rpcService, + HighAvailabilityServices highAvailabilityServices, + Executor executor) throws Exception { + + this.configuration = Preconditions.checkNotNull(configuration); + this.resourceID = Preconditions.checkNotNull(resourceID); + this.rpcService = Preconditions.checkNotNull(rpcService); + this.highAvailabilityServices = Preconditions.checkNotNull(highAvailabilityServices); + this.executor = rpcService.getExecutor(); + + InetAddress remoteAddress = InetAddress.getByName(rpcService.getAddress()); + + TaskManagerServicesConfiguration taskManagerServicesConfiguration = TaskManagerServicesConfiguration.fromConfiguration( + configuration, + remoteAddress, + false); + + TaskManagerServices taskManagerServices = TaskManagerServices.fromConfiguration(taskManagerServicesConfiguration, resourceID); + + TaskManagerConfiguration taskManagerConfiguration = TaskManagerConfiguration.fromConfiguration(configuration); + + this.taskManager = new TaskExecutor( + taskManagerConfiguration, + taskManagerServices.getTaskManagerLocation(), + rpcService, + taskManagerServices.getMemoryManager(), + taskManagerServices.getIOManager(), + taskManagerServices.getNetworkEnvironment(), + highAvailabilityServices, + taskManagerServices.getMetricRegistry(), + this); + } + + // -------------------------------------------------------------------------------------------- + // Lifecycle management + // -------------------------------------------------------------------------------------------- + + public void start() { + taskManager.start(); + } + + public void shutDown(Throwable cause) { + shutDownInternally(); + } + + protected void shutDownInternally() { + synchronized(lock) { + taskManager.shutDown(); + } + } + + // -------------------------------------------------------------------------------------------- + // FatalErrorHandler methods + // -------------------------------------------------------------------------------------------- + + @Override + public void onFatalError(Throwable exception) { + LOG.error("Fatal error occurred while executing the TaskManager. Shutting it down...", exception); + shutDown(exception); + } + + // -------------------------------------------------------------------------------------------- + // Static utilities + // -------------------------------------------------------------------------------------------- + + /** + * Create a RPC service for the task manager. + * + * @param configuration The configuration for the TaskManager. + * @param haServices to use for the task manager hostname retrieval + */ + public static RpcService createRpcService( + final Configuration configuration, + final HighAvailabilityServices haServices) throws Exception { + + checkNotNull(configuration); + checkNotNull(haServices); + + String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null); + + if (taskManagerHostname != null) { + LOG.info("Using configured hostname/address for TaskManager: {}.", taskManagerHostname); + } else { + Time lookupTimeout = Time.milliseconds(AkkaUtils.getLookupTimeout(configuration).toMillis()); + + InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress( + haServices.getResourceManagerLeaderRetriever(), + lookupTimeout); + + taskManagerHostname = taskManagerAddress.getHostName(); + + LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.", + taskManagerHostname, taskManagerAddress.getHostAddress()); + } + + final int rpcPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0); + + Preconditions.checkState(rpcPort < 0 || rpcPort >65535, "Invalid value for " + + "'%s' (port for the TaskManager actor system) : %d - Leave config parameter empty or " + + "use 0 to let the system choose port automatically.", + ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, rpcPort); + + return RpcServiceUtils.createRpcService(taskManagerHostname, rpcPort, configuration); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java new file mode 100644 index 0000000000000..ff7f7d5a639b7 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java @@ -0,0 +1,320 @@ +/* + * 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.taskexecutor; + +import org.apache.flink.core.memory.MemoryType; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; +import org.apache.flink.runtime.io.network.ConnectionManager; +import org.apache.flink.runtime.io.network.LocalConnectionManager; +import org.apache.flink.runtime.io.network.NetworkEnvironment; +import org.apache.flink.runtime.io.network.TaskEventDispatcher; +import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; +import org.apache.flink.runtime.io.network.netty.NettyConfig; +import org.apache.flink.runtime.io.network.netty.NettyConnectionManager; +import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; +import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.query.KvStateRegistry; +import org.apache.flink.runtime.query.netty.DisabledKvStateRequestStats; +import org.apache.flink.runtime.query.netty.KvStateServer; +import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; +import org.apache.flink.runtime.util.EnvironmentInformation; +import org.apache.flink.util.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; + +/** + * Container for {@link TaskExecutor} services such as the {@link MemoryManager}, {@link IOManager}, + * {@link NetworkEnvironment} and the {@link MetricRegistry}. + */ +public class TaskManagerServices { + private static final Logger LOG = LoggerFactory.getLogger(TaskManagerServices.class); + + /** TaskManager services */ + private final TaskManagerLocation taskManagerLocation; + private final MemoryManager memoryManager; + private final IOManager ioManager; + private final NetworkEnvironment networkEnvironment; + private final MetricRegistry metricRegistry; + + private TaskManagerServices( + TaskManagerLocation taskManagerLocation, + MemoryManager memoryManager, + IOManager ioManager, + NetworkEnvironment networkEnvironment, + MetricRegistry metricRegistry) { + + this.taskManagerLocation = Preconditions.checkNotNull(taskManagerLocation); + this.memoryManager = Preconditions.checkNotNull(memoryManager); + this.ioManager = Preconditions.checkNotNull(ioManager); + this.networkEnvironment = Preconditions.checkNotNull(networkEnvironment); + this.metricRegistry = Preconditions.checkNotNull(metricRegistry); + } + + // -------------------------------------------------------------------------------------------- + // Getter/Setter + // -------------------------------------------------------------------------------------------- + + public MemoryManager getMemoryManager() { + return memoryManager; + } + + public IOManager getIOManager() { + return ioManager; + } + + public NetworkEnvironment getNetworkEnvironment() { + return networkEnvironment; + } + + public TaskManagerLocation getTaskManagerLocation() { + return taskManagerLocation; + } + + public MetricRegistry getMetricRegistry() { + return metricRegistry; + } + + // -------------------------------------------------------------------------------------------- + // Static factory methods for task manager services + // -------------------------------------------------------------------------------------------- + + /** + * Creates and returns the task manager services. + * + * @param resourceID resource ID of the task manager + * @param taskManagerServicesConfiguration task manager configuration + * @return task manager components + * @throws Exception + */ + public static TaskManagerServices fromConfiguration( + TaskManagerServicesConfiguration taskManagerServicesConfiguration, + ResourceID resourceID) throws Exception { + + final NetworkEnvironment network = createNetworkEnvironment(taskManagerServicesConfiguration); + + network.start(); + + final TaskManagerLocation taskManagerLocation = new TaskManagerLocation( + resourceID, + taskManagerServicesConfiguration.getTaskManagerAddress(), + network.getConnectionManager().getDataPort()); + + // this call has to happen strictly after the network stack has been initialized + final MemoryManager memoryManager = createMemoryManager(taskManagerServicesConfiguration); + + // start the I/O manager, it will create some temp directories. + final IOManager ioManager = new IOManagerAsync(taskManagerServicesConfiguration.getTmpDirPaths()); + + MetricRegistry metricsRegistry = new MetricRegistry(taskManagerServicesConfiguration.getMetricRegistryConfiguration()); + + return new TaskManagerServices(taskManagerLocation, memoryManager, ioManager, network, metricsRegistry); + } + + /** + * Creates a {@link MemoryManager} from the given {@link TaskManagerServicesConfiguration}. + * + * @param taskManagerServicesConfiguration to create the memory manager from + * @return Memory manager + * @throws Exception + */ + private static MemoryManager createMemoryManager(TaskManagerServicesConfiguration taskManagerServicesConfiguration) throws Exception { + // computing the amount of memory to use depends on how much memory is available + // it strictly needs to happen AFTER the network stack has been initialized + + MemoryType memType = taskManagerServicesConfiguration.getNetworkConfig().memoryType(); + + // check if a value has been configured + long configuredMemory = taskManagerServicesConfiguration.getConfiguredMemory(); + + final long memorySize; + + boolean preAllocateMemory = taskManagerServicesConfiguration.isPreAllocateMemory(); + + if (configuredMemory > 0) { + if (preAllocateMemory) { + LOG.info("Using {} MB for managed memory." , configuredMemory); + } else { + LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory); + } + memorySize = configuredMemory << 20; // megabytes to bytes + } else { + float memoryFraction = taskManagerServicesConfiguration.getMemoryFraction(); + + if (memType == MemoryType.HEAP) { + long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * memoryFraction); + if (preAllocateMemory) { + LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." , + memoryFraction , relativeMemSize >> 20); + } else { + LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " + + "memory will be allocated lazily." , memoryFraction , relativeMemSize >> 20); + } + memorySize = relativeMemSize; + } else if (memType == MemoryType.OFF_HEAP) { + // The maximum heap memory has been adjusted according to the fraction + long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory(); + long directMemorySize = (long) (maxMemory / (1.0 - memoryFraction) * memoryFraction); + if (preAllocateMemory) { + LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." , + memoryFraction, directMemorySize >> 20); + } else { + LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," + + " memory will be allocated lazily.", memoryFraction, directMemorySize >> 20); + } + memorySize = directMemorySize; + } else { + throw new RuntimeException("No supported memory type detected."); + } + } + + // now start the memory manager + final MemoryManager memoryManager; + try { + memoryManager = new MemoryManager( + memorySize, + taskManagerServicesConfiguration.getNumberOfSlots(), + taskManagerServicesConfiguration.getNetworkConfig().networkBufferSize(), + memType, + preAllocateMemory); + } catch (OutOfMemoryError e) { + if (memType == MemoryType.HEAP) { + throw new Exception("OutOfMemory error (" + e.getMessage() + + ") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e); + } else if (memType == MemoryType.OFF_HEAP) { + throw new Exception("OutOfMemory error (" + e.getMessage() + + ") while allocating the TaskManager off-heap memory (" + memorySize + + " bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e); + } else { + throw e; + } + } + return memoryManager; + } + + /** + * Creates the {@link NetworkEnvironment} from the given {@link TaskManagerServicesConfiguration}. + * + * @param taskManagerServicesConfiguration to construct the network environment from + * @return Network environment + * @throws IOException + */ + private static NetworkEnvironment createNetworkEnvironment(TaskManagerServicesConfiguration taskManagerServicesConfiguration) throws IOException { + // pre-start checks + checkTempDirs(taskManagerServicesConfiguration.getTmpDirPaths()); + + NetworkEnvironmentConfiguration networkEnvironmentConfiguration = taskManagerServicesConfiguration.getNetworkConfig(); + + NetworkBufferPool networkBufferPool = new NetworkBufferPool( + networkEnvironmentConfiguration.numNetworkBuffers(), + networkEnvironmentConfiguration.networkBufferSize(), + networkEnvironmentConfiguration.memoryType()); + + ConnectionManager connectionManager; + + if (networkEnvironmentConfiguration.nettyConfig() != null) { + connectionManager = new NettyConnectionManager(networkEnvironmentConfiguration.nettyConfig()); + } else { + connectionManager = new LocalConnectionManager(); + } + + ResultPartitionManager resultPartitionManager = new ResultPartitionManager(); + TaskEventDispatcher taskEventDispatcher = new TaskEventDispatcher(); + + KvStateRegistry kvStateRegistry = new KvStateRegistry(); + + KvStateServer kvStateServer; + + if (networkEnvironmentConfiguration.nettyConfig() != null) { + NettyConfig nettyConfig = networkEnvironmentConfiguration.nettyConfig(); + + int numNetworkThreads = networkEnvironmentConfiguration.queryServerNetworkThreads() == 0 ? + nettyConfig.getNumberOfSlots() : networkEnvironmentConfiguration.queryServerNetworkThreads(); + + int numQueryThreads = networkEnvironmentConfiguration.queryServerQueryThreads() == 0 ? + nettyConfig.getNumberOfSlots() : networkEnvironmentConfiguration.queryServerQueryThreads(); + + kvStateServer = new KvStateServer( + taskManagerServicesConfiguration.getTaskManagerAddress(), + networkEnvironmentConfiguration.queryServerPort(), + numNetworkThreads, + numQueryThreads, + kvStateRegistry, + new DisabledKvStateRequestStats()); + } else { + kvStateServer = null; + } + + // we start the network first, to make sure it can allocate its buffers first + final NetworkEnvironment network = new NetworkEnvironment( + networkBufferPool, + connectionManager, + resultPartitionManager, + taskEventDispatcher, + kvStateRegistry, + kvStateServer, + networkEnvironmentConfiguration.ioMode(), + networkEnvironmentConfiguration.partitionRequestInitialBackoff(), + networkEnvironmentConfiguration.partitinRequestMaxBackoff()); + + return network; + } + + /** + * Validates that all the directories denoted by the strings do actually exist, are proper + * directories (not files), and are writable. + * + * @param tmpDirs The array of directory paths to check. + * @throws Exception Thrown if any of the directories does not exist or is not writable + * or is a file, rather than a directory. + */ + private static void checkTempDirs(String[] tmpDirs) throws IOException { + for (String dir : tmpDirs) { + if (dir != null && !dir.equals("")) { + File file = new File(dir); + if (!file.exists()) { + throw new IOException("Temporary file directory " + file.getAbsolutePath() + " does not exist."); + } + if (!file.isDirectory()) { + throw new IOException("Temporary file directory " + file.getAbsolutePath() + " is not a directory."); + } + if (!file.canWrite()) { + throw new IOException("Temporary file directory " + file.getAbsolutePath() + " is not writable."); + } + + if (LOG.isInfoEnabled()) { + long totalSpaceGb = file.getTotalSpace() >> 30; + long usableSpaceGb = file.getUsableSpace() >> 30; + double usablePercentage = (double)usableSpaceGb / totalSpaceGb * 100; + String path = file.getAbsolutePath(); + LOG.info(String.format("Temporary file directory '%s': total %d GB, " + "usable %d GB (%.2f%% usable)", + path, totalSpaceGb, usableSpaceGb, usablePercentage)); + } + } else { + throw new IllegalArgumentException("Temporary file directory #$id is null."); + } + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java new file mode 100644 index 0000000000000..66d969ad005a8 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java @@ -0,0 +1,325 @@ +/* + * 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.taskexecutor; + +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.IllegalConfigurationException; +import org.apache.flink.core.memory.HeapMemorySegment; +import org.apache.flink.core.memory.HybridMemorySegment; +import org.apache.flink.core.memory.MemorySegmentFactory; +import org.apache.flink.core.memory.MemoryType; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.io.network.netty.NettyConfig; +import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; +import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration; +import org.apache.flink.util.MathUtils; + +import java.io.File; +import java.net.InetAddress; +import java.net.InetSocketAddress; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Configuration for the task manager services such as the network environment, the memory manager, + * the io manager and the metric registry + */ +public class TaskManagerServicesConfiguration { + + private final InetAddress taskManagerAddress; + + private final String[] tmpDirPaths; + + private final int numberOfSlots; + + private final NetworkEnvironmentConfiguration networkConfig; + + private final long configuredMemory; + + private final boolean preAllocateMemory; + + private final float memoryFraction; + + private final MetricRegistryConfiguration metricRegistryConfiguration; + + public TaskManagerServicesConfiguration( + InetAddress taskManagerAddress, + String[] tmpDirPaths, + NetworkEnvironmentConfiguration networkConfig, + int numberOfSlots, + long configuredMemory, + boolean preAllocateMemory, + float memoryFraction, + MetricRegistryConfiguration metricRegistryConfiguration) { + + this.taskManagerAddress = checkNotNull(taskManagerAddress); + this.tmpDirPaths = checkNotNull(tmpDirPaths); + this.networkConfig = checkNotNull(networkConfig); + this.numberOfSlots = checkNotNull(numberOfSlots); + + this.configuredMemory = configuredMemory; + this.preAllocateMemory = preAllocateMemory; + this.memoryFraction = memoryFraction; + + this.metricRegistryConfiguration = checkNotNull(metricRegistryConfiguration); + } + + // -------------------------------------------------------------------------------------------- + // Getter/Setter + // -------------------------------------------------------------------------------------------- + + + public InetAddress getTaskManagerAddress() { + return taskManagerAddress; + } + + public String[] getTmpDirPaths() { + return tmpDirPaths; + } + + public NetworkEnvironmentConfiguration getNetworkConfig() { return networkConfig; } + + public int getNumberOfSlots() { + return numberOfSlots; + } + + public float getMemoryFraction() { + return memoryFraction; + } + + public long getConfiguredMemory() { + return configuredMemory; + } + + public boolean isPreAllocateMemory() { + return preAllocateMemory; + } + + public MetricRegistryConfiguration getMetricRegistryConfiguration() { + return metricRegistryConfiguration; + } + + // -------------------------------------------------------------------------------------------- + // Parsing of Flink configuration + // -------------------------------------------------------------------------------------------- + + /** + * Utility method to extract TaskManager config parameters from the configuration and to + * sanity check them. + * + * @param configuration The configuration. + * @param remoteAddress identifying the IP address under which the TaskManager will be accessible + * @param localCommunication True, to skip initializing the network stack. + * Use only in cases where only one task manager runs. + * @return TaskExecutorConfiguration that wrappers InstanceConnectionInfo, NetworkEnvironmentConfiguration, etc. + */ + public static TaskManagerServicesConfiguration fromConfiguration( + Configuration configuration, + InetAddress remoteAddress, + boolean localCommunication) throws Exception { + + // we need this because many configs have been written with a "-1" entry + int slots = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1); + if (slots == -1) { + slots = 1; + } + + final String[] tmpDirs = configuration.getString( + ConfigConstants.TASK_MANAGER_TMP_DIR_KEY, + ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH).split(",|" + File.pathSeparator); + + final NetworkEnvironmentConfiguration networkConfig = parseNetworkEnvironmentConfiguration( + configuration, + localCommunication, + remoteAddress, + slots); + + // extract memory settings + long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L); + checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory, + ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, + "MemoryManager needs at least one MB of memory. " + + "If you leave this config parameter empty, the system automatically " + + "pick a fraction of the available memory."); + + boolean preAllocateMemory = configuration.getBoolean( + ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY, + ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE); + + float memoryFraction = configuration.getFloat( + ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY, + ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION); + checkConfigParameter(memoryFraction > 0.0f && memoryFraction < 1.0f, memoryFraction, + ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY, + "MemoryManager fraction of the free memory must be between 0.0 and 1.0"); + + final MetricRegistryConfiguration metricRegistryConfiguration = MetricRegistryConfiguration.fromConfiguration(configuration); + + return new TaskManagerServicesConfiguration( + remoteAddress, + tmpDirs, + networkConfig, + slots, + configuredMemory, + preAllocateMemory, + memoryFraction, + metricRegistryConfiguration); + } + + // -------------------------------------------------------------------------- + // Parsing and checking the TaskManager Configuration + // -------------------------------------------------------------------------- + + /** + * Creates the {@link NetworkEnvironmentConfiguration} from the given {@link Configuration}. + * + * @param configuration to create the network environment configuration from + * @param localTaskManagerCommunication true if task manager communication is local + * @param taskManagerAddress address of the task manager + * @param slots to start the task manager with + * @return Network environment configuration + */ + private static NetworkEnvironmentConfiguration parseNetworkEnvironmentConfiguration( + Configuration configuration, + boolean localTaskManagerCommunication, + InetAddress taskManagerAddress, + int slots) throws Exception { + + // ----> hosts / ports for communication and data exchange + + int dataport = configuration.getInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, + ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT); + + checkConfigParameter(dataport > 0, dataport, ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, + "Leave config parameter empty or use 0 to let the system choose a port automatically."); + + checkConfigParameter(slots >= 1, slots, ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, + "Number of task slots must be at least one."); + + final int numNetworkBuffers = configuration.getInteger( + ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, + ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS); + + checkConfigParameter(numNetworkBuffers > 0, numNetworkBuffers, + ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, ""); + + final int pageSize = configuration.getInteger( + ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY, + ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE); + + // check page size of for minimum size + checkConfigParameter(pageSize >= MemoryManager.MIN_PAGE_SIZE, pageSize, + ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY, + "Minimum memory segment size is " + MemoryManager.MIN_PAGE_SIZE); + + // check page size for power of two + checkConfigParameter(MathUtils.isPowerOf2(pageSize), pageSize, + ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY, + "Memory segment size must be a power of 2."); + + // check whether we use heap or off-heap memory + final MemoryType memType; + if (configuration.getBoolean(ConfigConstants.TASK_MANAGER_MEMORY_OFF_HEAP_KEY, false)) { + memType = MemoryType.OFF_HEAP; + } else { + memType = MemoryType.HEAP; + } + + // initialize the memory segment factory accordingly + if (memType == MemoryType.HEAP) { + if (!MemorySegmentFactory.initializeIfNotInitialized(HeapMemorySegment.FACTORY)) { + throw new Exception("Memory type is set to heap memory, but memory segment " + + "factory has been initialized for off-heap memory segments"); + } + } else { + if (!MemorySegmentFactory.initializeIfNotInitialized(HybridMemorySegment.FACTORY)) { + throw new Exception("Memory type is set to off-heap memory, but memory segment " + + "factory has been initialized for heap memory segments"); + } + } + + final NettyConfig nettyConfig; + if (!localTaskManagerCommunication) { + final InetSocketAddress taskManagerInetSocketAddress = new InetSocketAddress(taskManagerAddress, dataport); + + nettyConfig = new NettyConfig(taskManagerInetSocketAddress.getAddress(), + taskManagerInetSocketAddress.getPort(), pageSize, slots, configuration); + } else { + nettyConfig = null; + } + + // Default spill I/O mode for intermediate results + final String syncOrAsync = configuration.getString( + ConfigConstants.TASK_MANAGER_NETWORK_DEFAULT_IO_MODE, + ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_DEFAULT_IO_MODE); + + final IOManager.IOMode ioMode; + if (syncOrAsync.equals("async")) { + ioMode = IOManager.IOMode.ASYNC; + } else { + ioMode = IOManager.IOMode.SYNC; + } + + final int queryServerPort = configuration.getInteger( + ConfigConstants.QUERYABLE_STATE_SERVER_PORT, + ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_PORT); + + final int queryServerNetworkThreads = configuration.getInteger( + ConfigConstants.QUERYABLE_STATE_SERVER_NETWORK_THREADS, + ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_NETWORK_THREADS); + + final int queryServerQueryThreads = configuration.getInteger( + ConfigConstants.QUERYABLE_STATE_SERVER_QUERY_THREADS, + ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_QUERY_THREADS); + + return new NetworkEnvironmentConfiguration( + numNetworkBuffers, + pageSize, + memType, + ioMode, + queryServerPort, + queryServerNetworkThreads, + queryServerQueryThreads, + nettyConfig, + 500, + 3000); + } + + /** + * Validates a condition for a config parameter and displays a standard exception, if the + * the condition does not hold. + * + * @param condition The condition that must hold. If the condition is false, an exception is thrown. + * @param parameter The parameter value. Will be shown in the exception message. + * @param name The name of the config parameter. Will be shown in the exception message. + * @param errorMessage The optional custom error message to append to the exception message. + */ + private static void checkConfigParameter( + boolean condition, + Object parameter, + String name, + String errorMessage) { + if (!condition) { + throw new IllegalConfigurationException("Invalid configuration value for " + name + " : " + parameter + " - " + errorMessage); + } + } +} + diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerRunner.java deleted file mode 100644 index 4f756fbae6d7a..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerRunner.java +++ /dev/null @@ -1,749 +0,0 @@ -/* - * 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.taskmanager; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.IllegalConfigurationException; -import org.apache.flink.core.memory.HeapMemorySegment; -import org.apache.flink.core.memory.HybridMemorySegment; -import org.apache.flink.core.memory.MemorySegmentFactory; -import org.apache.flink.core.memory.MemoryType; -import org.apache.flink.runtime.akka.AkkaUtils; -import org.apache.flink.runtime.clusterframework.types.ResourceID; -import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.io.disk.iomanager.IOManager; -import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; -import org.apache.flink.runtime.io.network.ConnectionManager; -import org.apache.flink.runtime.io.network.LocalConnectionManager; -import org.apache.flink.runtime.io.network.NetworkEnvironment; -import org.apache.flink.runtime.io.network.TaskEventDispatcher; -import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; -import org.apache.flink.runtime.io.network.netty.NettyConfig; -import org.apache.flink.runtime.io.network.netty.NettyConnectionManager; -import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; -import org.apache.flink.runtime.leaderelection.LeaderElectionService; -import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; -import org.apache.flink.runtime.memory.MemoryManager; -import org.apache.flink.runtime.query.KvStateRegistry; -import org.apache.flink.runtime.query.netty.DisabledKvStateRequestStats; -import org.apache.flink.runtime.query.netty.KvStateServer; -import org.apache.flink.runtime.rpc.RpcService; -import org.apache.flink.runtime.rpc.akka.AkkaRpcService; -import org.apache.flink.runtime.taskexecutor.TaskExecutor; -import org.apache.flink.runtime.taskexecutor.TaskExecutorConfiguration; -import org.apache.flink.runtime.util.EnvironmentInformation; -import org.apache.flink.runtime.util.LeaderRetrievalUtils; -import org.apache.flink.util.MathUtils; -import org.apache.flink.util.NetUtils; - -import akka.actor.ActorSystem; -import akka.util.Timeout; -import com.typesafe.config.Config; -import org.apache.flink.util.Preconditions; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import scala.Option; -import scala.Some; -import scala.Tuple2; -import scala.concurrent.duration.Duration; -import scala.concurrent.duration.FiniteDuration; - -import java.io.File; -import java.io.IOException; -import java.net.InetAddress; -import java.net.InetSocketAddress; -import java.util.concurrent.TimeUnit; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * This class is the executable entry point for the task manager in yarn or standalone mode. - * It constructs the related components (network, I/O manager, memory manager, RPC service, HA service) - * and starts them. - */ -public class TaskManagerRunner { - - private static final Logger LOG = LoggerFactory.getLogger(TaskManagerRunner.class); - - /** - * Constructs related components of the TaskManager and starts them. - * - * @param configuration The configuration for the TaskManager. - * @param resourceID The id of the resource which the task manager will run on. - * @param rpcService Optionally, The rpc service which is used to start and connect to the TaskManager RpcEndpoint . - * If none is given, then a RpcService is constructed from the configuration. - * @param taskManagerHostname Optionally, The hostname/address that describes the TaskManager's data location. - * If none is given, it can be got from the configuration. - * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack. - * @param haServices Optionally, a high availability service can be provided. If none is given, - * then a HighAvailabilityServices is constructed from the configuration. - */ - public static void createAndStartComponents( - final Configuration configuration, - final ResourceID resourceID, - RpcService rpcService, - String taskManagerHostname, - boolean localTaskManagerCommunication, - HighAvailabilityServices haServices) throws Exception { - - checkNotNull(configuration); - checkNotNull(resourceID); - - if (taskManagerHostname == null || taskManagerHostname.isEmpty()) { - taskManagerHostname = selectNetworkInterface(configuration); - } - - if (rpcService == null) { - // if no task manager port has been configured, use 0 (system will pick any free port) - final int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0); - if (actorSystemPort < 0 || actorSystemPort > 65535) { - throw new IllegalConfigurationException("Invalid value for '" + - ConfigConstants.TASK_MANAGER_IPC_PORT_KEY + - "' (port for the TaskManager actor system) : " + actorSystemPort + - " - Leave config parameter empty or use 0 to let the system choose a port automatically."); - } - rpcService = createRpcService(configuration, taskManagerHostname, actorSystemPort); - } - - if(haServices == null) { - // start high availability service to implement getResourceManagerLeaderRetriever method only - haServices = new HighAvailabilityServices() { - @Override - public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception { - return LeaderRetrievalUtils.createLeaderRetrievalService(configuration); - } - - @Override - public LeaderElectionService getResourceManagerLeaderElectionService() throws Exception { - return null; - } - - @Override - public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception { - return null; - } - }; - } - - createAndStartTaskManagerComponents( - configuration, - resourceID, - rpcService, - taskManagerHostname, - haServices, - localTaskManagerCommunication); - } - - /** - *

    - * This method tries to select the network interface to use for the TaskManager - * communication. The network interface is used both for the actor communication - * (coordination) as well as for the data exchange between task managers. Unless - * the hostname/interface is explicitly configured in the configuration, this - * method will try out various interfaces and methods to connect to the JobManager - * and select the one where the connection attempt is successful. - *

    - * - * @param configuration The configuration for the TaskManager. - * @return The host name under which the TaskManager communicates. - */ - private static String selectNetworkInterface(Configuration configuration) throws Exception { - String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null); - if (taskManagerHostname != null) { - LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname); - } else { - LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration); - FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration); - - InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout); - taskManagerHostname = taskManagerAddress.getHostName(); - LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.", - taskManagerHostname, taskManagerAddress.getHostAddress()); - } - - return taskManagerHostname; - } - - /** - * Utility method to create RPC service from configuration and hostname, port. - * - * @param configuration The configuration for the TaskManager. - * @param taskManagerHostname The hostname/address that describes the TaskManager's data location. - * @param actorSystemPort If true, the TaskManager will not initiate the TCP network stack. - * @return The rpc service which is used to start and connect to the TaskManager RpcEndpoint . - * @throws java.io.IOException Thrown, if the actor system can not bind to the address - * @throws java.lang.Exception Thrown is some other error occurs while creating akka actor system - */ - private static RpcService createRpcService(Configuration configuration, String taskManagerHostname, int actorSystemPort) - throws Exception{ - - // Bring up the TaskManager actor system first, bind it to the given address. - - LOG.info("Starting TaskManager actor system at " + - NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort)); - - final ActorSystem taskManagerSystem; - try { - Tuple2 address = new Tuple2(taskManagerHostname, actorSystemPort); - Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address)); - LOG.debug("Using akka configuration\n " + akkaConfig); - taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig); - } catch (Throwable t) { - if (t instanceof org.jboss.netty.channel.ChannelException) { - Throwable cause = t.getCause(); - if (cause != null && t.getCause() instanceof java.net.BindException) { - String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort); - throw new IOException("Unable to bind TaskManager actor system to address " + - address + " - " + cause.getMessage(), t); - } - } - throw new Exception("Could not create TaskManager actor system", t); - } - - // start akka rpc service based on actor system - final Timeout timeout = new Timeout(AkkaUtils.getTimeout(configuration).toMillis(), TimeUnit.MILLISECONDS); - final AkkaRpcService akkaRpcService = new AkkaRpcService(taskManagerSystem, timeout); - - return akkaRpcService; - } - - /** - * @param configuration The configuration for the TaskManager. - * @param resourceID The id of the resource which the task manager will run on. - * @param rpcService The rpc service which is used to start and connect to the TaskManager RpcEndpoint . - * @param taskManagerHostname The hostname/address that describes the TaskManager's data location. - * @param haServices Optionally, a high availability service can be provided. If none is given, - * then a HighAvailabilityServices is constructed from the configuration. - * @param localTaskManagerCommunication If true, the TaskManager will not initiate the TCP network stack. - * @throws IllegalConfigurationException Thrown, if the given config contains illegal values. - * @throws IOException Thrown, if any of the I/O components (such as buffer pools, I/O manager, ...) - * cannot be properly started. - * @throws Exception Thrown is some other error occurs while parsing the configuration or - * starting the TaskManager components. - */ - private static void createAndStartTaskManagerComponents( - Configuration configuration, - ResourceID resourceID, - RpcService rpcService, - String taskManagerHostname, - HighAvailabilityServices haServices, - boolean localTaskManagerCommunication) throws Exception { - - final TaskExecutorConfiguration taskManagerConfig = parseTaskManagerConfiguration( - configuration, taskManagerHostname, localTaskManagerCommunication); - - TaskManagerComponents taskManagerComponents = createTaskManagerComponents( - resourceID, - InetAddress.getByName(taskManagerHostname), - taskManagerConfig, - configuration); - - final TaskExecutor taskExecutor = new TaskExecutor( - taskManagerConfig, - taskManagerComponents.getTaskManagerLocation(), - rpcService, taskManagerComponents.getMemoryManager(), - taskManagerComponents.getIOManager(), - taskManagerComponents.getNetworkEnvironment(), - haServices); - - taskExecutor.start(); - } - - /** - * Creates and returns the task manager components. - * - * @param resourceID resource ID of the task manager - * @param taskManagerAddress address of the task manager - * @param taskExecutorConfig task manager configuration - * @param configuration of Flink - * @return task manager components - * @throws Exception - */ - private static TaskManagerComponents createTaskManagerComponents( - ResourceID resourceID, - InetAddress taskManagerAddress, - TaskExecutorConfiguration taskExecutorConfig, - Configuration configuration) throws Exception { - - MemoryType memType = taskExecutorConfig.getNetworkConfig().memoryType(); - - // pre-start checks - checkTempDirs(taskExecutorConfig.getTmpDirPaths()); - - NetworkEnvironmentConfiguration networkEnvironmentConfiguration = taskExecutorConfig.getNetworkConfig(); - - NetworkBufferPool networkBufferPool = new NetworkBufferPool( - networkEnvironmentConfiguration.numNetworkBuffers(), - networkEnvironmentConfiguration.networkBufferSize(), - networkEnvironmentConfiguration.memoryType()); - - ConnectionManager connectionManager; - - if (networkEnvironmentConfiguration.nettyConfig().isDefined()) { - connectionManager = new NettyConnectionManager(networkEnvironmentConfiguration.nettyConfig().get()); - } else { - connectionManager = new LocalConnectionManager(); - } - - ResultPartitionManager resultPartitionManager = new ResultPartitionManager(); - TaskEventDispatcher taskEventDispatcher = new TaskEventDispatcher(); - - KvStateRegistry kvStateRegistry = new KvStateRegistry(); - - KvStateServer kvStateServer; - - if (networkEnvironmentConfiguration.nettyConfig().isDefined()) { - NettyConfig nettyConfig = networkEnvironmentConfiguration.nettyConfig().get(); - - int numNetworkThreads = networkEnvironmentConfiguration.queryServerNetworkThreads() == 0 ? - nettyConfig.getNumberOfSlots() : networkEnvironmentConfiguration.queryServerNetworkThreads(); - - int numQueryThreads = networkEnvironmentConfiguration.queryServerQueryThreads() == 0 ? - nettyConfig.getNumberOfSlots() : networkEnvironmentConfiguration.queryServerQueryThreads(); - - kvStateServer = new KvStateServer( - taskManagerAddress, - networkEnvironmentConfiguration.queryServerPort(), - numNetworkThreads, - numQueryThreads, - kvStateRegistry, - new DisabledKvStateRequestStats()); - } else { - kvStateServer = null; - } - - // we start the network first, to make sure it can allocate its buffers first - final NetworkEnvironment network = new NetworkEnvironment( - networkBufferPool, - connectionManager, - resultPartitionManager, - taskEventDispatcher, - kvStateRegistry, - kvStateServer, - networkEnvironmentConfiguration.ioMode(), - networkEnvironmentConfiguration.partitionRequestInitialBackoff(), - networkEnvironmentConfiguration.partitinRequestMaxBackoff()); - - network.start(); - - final TaskManagerLocation taskManagerLocation = new TaskManagerLocation( - resourceID, - taskManagerAddress, - network.getConnectionManager().getDataPort()); - - // computing the amount of memory to use depends on how much memory is available - // it strictly needs to happen AFTER the network stack has been initialized - - // check if a value has been configured - long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L); - checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory, - ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, - "MemoryManager needs at least one MB of memory. " + - "If you leave this config parameter empty, the system automatically " + - "pick a fraction of the available memory."); - - final long memorySize; - boolean preAllocateMemory = configuration.getBoolean( - ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY, - ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE); - if (configuredMemory > 0) { - if (preAllocateMemory) { - LOG.info("Using {} MB for managed memory." , configuredMemory); - } else { - LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory); - } - memorySize = configuredMemory << 20; // megabytes to bytes - } else { - float fraction = configuration.getFloat( - ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY, - ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION); - checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction, - ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY, - "MemoryManager fraction of the free memory must be between 0.0 and 1.0"); - - if (memType == MemoryType.HEAP) { - long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction); - if (preAllocateMemory) { - LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." , - fraction , relativeMemSize >> 20); - } else { - LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " + - "memory will be allocated lazily." , fraction , relativeMemSize >> 20); - } - memorySize = relativeMemSize; - } else if (memType == MemoryType.OFF_HEAP) { - // The maximum heap memory has been adjusted according to the fraction - long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory(); - long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction); - if (preAllocateMemory) { - LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." , - fraction, directMemorySize >> 20); - } else { - LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," + - " memory will be allocated lazily.", fraction, directMemorySize >> 20); - } - memorySize = directMemorySize; - } else { - throw new RuntimeException("No supported memory type detected."); - } - } - - // now start the memory manager - final MemoryManager memoryManager; - try { - memoryManager = new MemoryManager( - memorySize, - taskExecutorConfig.getNumberOfSlots(), - taskExecutorConfig.getNetworkConfig().networkBufferSize(), - memType, - preAllocateMemory); - } catch (OutOfMemoryError e) { - if (memType == MemoryType.HEAP) { - throw new Exception("OutOfMemory error (" + e.getMessage() + - ") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e); - } else if (memType == MemoryType.OFF_HEAP) { - throw new Exception("OutOfMemory error (" + e.getMessage() + - ") while allocating the TaskManager off-heap memory (" + memorySize + - " bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e); - } else { - throw e; - } - } - - // start the I/O manager, it will create some temp directories. - final IOManager ioManager = new IOManagerAsync(taskExecutorConfig.getTmpDirPaths()); - - return new TaskManagerComponents(taskManagerLocation, memoryManager, ioManager, network); - } - - // -------------------------------------------------------------------------- - // Parsing and checking the TaskManager Configuration - // -------------------------------------------------------------------------- - - /** - * Utility method to extract TaskManager config parameters from the configuration and to - * sanity check them. - * - * @param configuration The configuration. - * @param taskManagerHostname The host name under which the TaskManager communicates. - * @param localTaskManagerCommunication True, to skip initializing the network stack. - * Use only in cases where only one task manager runs. - * @return TaskExecutorConfiguration that wrappers InstanceConnectionInfo, NetworkEnvironmentConfiguration, etc. - */ - private static TaskExecutorConfiguration parseTaskManagerConfiguration( - Configuration configuration, - String taskManagerHostname, - boolean localTaskManagerCommunication) throws Exception { - - // ------- read values from the config and check them --------- - // (a lot of them) - - // ----> hosts / ports for communication and data exchange - - int dataport = configuration.getInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, - ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT); - - checkConfigParameter(dataport > 0, dataport, ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, - "Leave config parameter empty or use 0 to let the system choose a port automatically."); - - InetAddress taskManagerAddress = InetAddress.getByName(taskManagerHostname); - final InetSocketAddress taskManagerInetSocketAddress = new InetSocketAddress(taskManagerAddress, dataport); - - // ----> memory / network stack (shuffles/broadcasts), task slots, temp directories - - // we need this because many configs have been written with a "-1" entry - int slots = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1); - if (slots == -1) { - slots = 1; - } - - checkConfigParameter(slots >= 1, slots, ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, - "Number of task slots must be at least one."); - - final int numNetworkBuffers = configuration.getInteger( - ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, - ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS); - - checkConfigParameter(numNetworkBuffers > 0, numNetworkBuffers, - ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, ""); - - final int pageSize = configuration.getInteger( - ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY, - ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE); - - // check page size of for minimum size - checkConfigParameter(pageSize >= MemoryManager.MIN_PAGE_SIZE, pageSize, - ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY, - "Minimum memory segment size is " + MemoryManager.MIN_PAGE_SIZE); - - // check page size for power of two - checkConfigParameter(MathUtils.isPowerOf2(pageSize), pageSize, - ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY, - "Memory segment size must be a power of 2."); - - // check whether we use heap or off-heap memory - final MemoryType memType; - if (configuration.getBoolean(ConfigConstants.TASK_MANAGER_MEMORY_OFF_HEAP_KEY, false)) { - memType = MemoryType.OFF_HEAP; - } else { - memType = MemoryType.HEAP; - } - - // initialize the memory segment factory accordingly - if (memType == MemoryType.HEAP) { - if (!MemorySegmentFactory.initializeIfNotInitialized(HeapMemorySegment.FACTORY)) { - throw new Exception("Memory type is set to heap memory, but memory segment " + - "factory has been initialized for off-heap memory segments"); - } - } else { - if (!MemorySegmentFactory.initializeIfNotInitialized(HybridMemorySegment.FACTORY)) { - throw new Exception("Memory type is set to off-heap memory, but memory segment " + - "factory has been initialized for heap memory segments"); - } - } - - final String[] tmpDirs = configuration.getString( - ConfigConstants.TASK_MANAGER_TMP_DIR_KEY, - ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH).split(",|" + File.pathSeparator); - - final NettyConfig nettyConfig; - if (!localTaskManagerCommunication) { - nettyConfig = new NettyConfig(taskManagerInetSocketAddress.getAddress(), - taskManagerInetSocketAddress.getPort(), pageSize, slots, configuration); - } else { - nettyConfig = null; - } - - // Default spill I/O mode for intermediate results - final String syncOrAsync = configuration.getString( - ConfigConstants.TASK_MANAGER_NETWORK_DEFAULT_IO_MODE, - ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_DEFAULT_IO_MODE); - - final IOManager.IOMode ioMode; - if (syncOrAsync.equals("async")) { - ioMode = IOManager.IOMode.ASYNC; - } else { - ioMode = IOManager.IOMode.SYNC; - } - - final int queryServerPort = configuration.getInteger( - ConfigConstants.QUERYABLE_STATE_SERVER_PORT, - ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_PORT); - - final int queryServerNetworkThreads = configuration.getInteger( - ConfigConstants.QUERYABLE_STATE_SERVER_NETWORK_THREADS, - ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_NETWORK_THREADS); - - final int queryServerQueryThreads = configuration.getInteger( - ConfigConstants.QUERYABLE_STATE_SERVER_QUERY_THREADS, - ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_QUERY_THREADS); - - final NetworkEnvironmentConfiguration networkConfig = new NetworkEnvironmentConfiguration( - numNetworkBuffers, - pageSize, - memType, - ioMode, - queryServerPort, - queryServerNetworkThreads, - queryServerQueryThreads, - Option.apply(nettyConfig), - 500, - 3000); - - // ----> timeouts, library caching, profiling - - final FiniteDuration timeout; - try { - timeout = AkkaUtils.getTimeout(configuration); - } catch (Exception e) { - throw new IllegalArgumentException( - "Invalid format for '" + ConfigConstants.AKKA_ASK_TIMEOUT + - "'.Use formats like '50 s' or '1 min' to specify the timeout."); - } - LOG.info("Messages between TaskManager and JobManager have a max timeout of " + timeout); - - final long cleanupInterval = configuration.getLong( - ConfigConstants.LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL, - ConfigConstants.DEFAULT_LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL) * 1000; - - final FiniteDuration finiteRegistrationDuration; - try { - Duration maxRegistrationDuration = Duration.create(configuration.getString( - ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION, - ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_DURATION)); - if (maxRegistrationDuration.isFinite()) { - finiteRegistrationDuration = new FiniteDuration(maxRegistrationDuration.toSeconds(), TimeUnit.SECONDS); - } else { - finiteRegistrationDuration = null; - } - } catch (NumberFormatException e) { - throw new IllegalArgumentException("Invalid format for parameter " + - ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION, e); - } - - final FiniteDuration initialRegistrationPause; - try { - Duration pause = Duration.create(configuration.getString( - ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, - ConfigConstants.DEFAULT_TASK_MANAGER_INITIAL_REGISTRATION_PAUSE)); - if (pause.isFinite()) { - initialRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS); - } else { - throw new IllegalArgumentException("The initial registration pause must be finite: " + pause); - } - } catch (NumberFormatException e) { - throw new IllegalArgumentException("Invalid format for parameter " + - ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e); - } - - final FiniteDuration maxRegistrationPause; - try { - Duration pause = Duration.create(configuration.getString( - ConfigConstants.TASK_MANAGER_MAX_REGISTARTION_PAUSE, - ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_PAUSE)); - if (pause.isFinite()) { - maxRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS); - } else { - throw new IllegalArgumentException("The maximum registration pause must be finite: " + pause); - } - } catch (NumberFormatException e) { - throw new IllegalArgumentException("Invalid format for parameter " + - ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e); - } - - final FiniteDuration refusedRegistrationPause; - try { - Duration pause = Duration.create(configuration.getString( - ConfigConstants.TASK_MANAGER_REFUSED_REGISTRATION_PAUSE, - ConfigConstants.DEFAULT_TASK_MANAGER_REFUSED_REGISTRATION_PAUSE)); - if (pause.isFinite()) { - refusedRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS); - } else { - throw new IllegalArgumentException("The refused registration pause must be finite: " + pause); - } - } catch (NumberFormatException e) { - throw new IllegalArgumentException("Invalid format for parameter " + - ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e); - } - - return new TaskExecutorConfiguration( - tmpDirs, - cleanupInterval, - networkConfig, - timeout, - finiteRegistrationDuration, - slots, - configuration, - initialRegistrationPause, - maxRegistrationPause, - refusedRegistrationPause); - } - - /** - * Validates a condition for a config parameter and displays a standard exception, if the - * the condition does not hold. - * - * @param condition The condition that must hold. If the condition is false, an exception is thrown. - * @param parameter The parameter value. Will be shown in the exception message. - * @param name The name of the config parameter. Will be shown in the exception message. - * @param errorMessage The optional custom error message to append to the exception message. - */ - private static void checkConfigParameter( - boolean condition, - Object parameter, - String name, - String errorMessage) { - if (!condition) { - throw new IllegalConfigurationException("Invalid configuration value for " + name + " : " + parameter + " - " + errorMessage); - } - } - - /** - * Validates that all the directories denoted by the strings do actually exist, are proper - * directories (not files), and are writable. - * - * @param tmpDirs The array of directory paths to check. - * @throws Exception Thrown if any of the directories does not exist or is not writable - * or is a file, rather than a directory. - */ - private static void checkTempDirs(String[] tmpDirs) throws IOException { - for (String dir : tmpDirs) { - if (dir != null && !dir.equals("")) { - File file = new File(dir); - if (!file.exists()) { - throw new IOException("Temporary file directory " + file.getAbsolutePath() + " does not exist."); - } - if (!file.isDirectory()) { - throw new IOException("Temporary file directory " + file.getAbsolutePath() + " is not a directory."); - } - if (!file.canWrite()) { - throw new IOException("Temporary file directory " + file.getAbsolutePath() + " is not writable."); - } - - if (LOG.isInfoEnabled()) { - long totalSpaceGb = file.getTotalSpace() >> 30; - long usableSpaceGb = file.getUsableSpace() >> 30; - double usablePercentage = (double)usableSpaceGb / totalSpaceGb * 100; - String path = file.getAbsolutePath(); - LOG.info(String.format("Temporary file directory '%s': total %d GB, " + "usable %d GB (%.2f%% usable)", - path, totalSpaceGb, usableSpaceGb, usablePercentage)); - } - } else { - throw new IllegalArgumentException("Temporary file directory #$id is null."); - } - } - } - - private static class TaskManagerComponents { - private final TaskManagerLocation taskManagerLocation; - private final MemoryManager memoryManager; - private final IOManager ioManager; - private final NetworkEnvironment networkEnvironment; - - private TaskManagerComponents( - TaskManagerLocation taskManagerLocation, - MemoryManager memoryManager, - IOManager ioManager, - NetworkEnvironment networkEnvironment) { - - this.taskManagerLocation = Preconditions.checkNotNull(taskManagerLocation); - this.memoryManager = Preconditions.checkNotNull(memoryManager); - this.ioManager = Preconditions.checkNotNull(ioManager); - this.networkEnvironment = Preconditions.checkNotNull(networkEnvironment); - } - - public MemoryManager getMemoryManager() { - return memoryManager; - } - - public IOManager getIOManager() { - return ioManager; - } - - public NetworkEnvironment getNetworkEnvironment() { - return networkEnvironment; - } - - public TaskManagerLocation getTaskManagerLocation() { - return taskManagerLocation; - } - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderRetrievalUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderRetrievalUtils.java index b6d9306f6831d..42655a2ad52d5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderRetrievalUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderRetrievalUtils.java @@ -22,6 +22,7 @@ import akka.actor.ActorSystem; import akka.dispatch.Mapper; import akka.dispatch.OnComplete; +import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.IllegalConfigurationException; @@ -169,6 +170,12 @@ public static LeaderConnectionInfo retrieveLeaderConnectionInfo( } } + public static InetAddress findConnectingAddress( + LeaderRetrievalService leaderRetrievalService, + Time timeout) throws LeaderRetrievalException { + return findConnectingAddress(leaderRetrievalService, new FiniteDuration(timeout.getSize(), timeout.getUnit())); + } + public static InetAddress findConnectingAddress( LeaderRetrievalService leaderRetrievalService, FiniteDuration timeout) throws LeaderRetrievalException { diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala index bd3af33297637..84f5ac77e6401 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala @@ -94,6 +94,10 @@ object AkkaUtils { createActorSystem(getDefaultAkkaConfig) } + def getAkkaConfig(configuration: Configuration, hostname: String, port: Int): Config = { + getAkkaConfig(configuration, if (hostname == null) Some((hostname, port)) else None) + } + /** * Creates an akka config with the provided configuration values. If the listening address is * specified, then the actor system will listen on the respective address. diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/NetworkEnvironmentConfiguration.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/NetworkEnvironmentConfiguration.scala index 893eaa8d0a28a..97aae34b7bcfd 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/NetworkEnvironmentConfiguration.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/NetworkEnvironmentConfiguration.scala @@ -30,6 +30,6 @@ case class NetworkEnvironmentConfiguration( queryServerPort: Int, queryServerNetworkThreads: Int, queryServerQueryThreads: Int, - nettyConfig: Option[NettyConfig] = None, + nettyConfig: NettyConfig = null, partitionRequestInitialBackoff: Int = 500, partitinRequestMaxBackoff: Int = 3000) diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala index c6dcbb0dbaf9e..21964b53a2e00 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala @@ -1925,7 +1925,7 @@ object TaskManager { netConfig.networkBufferSize, netConfig.memoryType) - val connectionManager = netConfig.nettyConfig match { + val connectionManager = Option(netConfig.nettyConfig) match { case Some(nettyConfig) => new NettyConnectionManager(nettyConfig) case None => new LocalConnectionManager() } @@ -1935,7 +1935,7 @@ object TaskManager { val kvStateRegistry = new KvStateRegistry() - val kvStateServer = netConfig.nettyConfig match { + val kvStateServer = Option(netConfig.nettyConfig) match { case Some(nettyConfig) => val numNetworkThreads = if (netConfig.queryServerNetworkThreads == 0) { @@ -2267,7 +2267,7 @@ object TaskManager { queryServerPort, queryServerNetworkThreads, queryServerQueryThreads, - nettyConfig) + nettyConfig.getOrElse(null)) // ----> timeouts, library caching, profiling diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java index 9f39de1c26aac..d0fb376623a54 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java @@ -24,7 +24,6 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; -import org.apache.flink.runtime.io.network.netty.NettyConfig; import org.apache.flink.runtime.io.network.partition.ResultPartition; import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; @@ -41,7 +40,6 @@ import org.junit.Test; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import scala.Some; import scala.concurrent.duration.FiniteDuration; import scala.concurrent.impl.Promise; @@ -78,7 +76,7 @@ public void testEagerlyDeployConsumers() throws Exception { 0, 0, 0, - Some.empty(), + null, 0, 0); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java index 5b8e6e65cd061..2a004c56f701e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java @@ -30,7 +30,6 @@ import java.lang.reflect.InvocationHandler; import java.lang.reflect.Method; import java.lang.reflect.Proxy; -import java.net.InetAddress; import java.util.BitSet; import java.util.UUID; import java.util.concurrent.Callable; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java index 26218dd22387c..9c1f28832c14a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java @@ -26,8 +26,9 @@ import org.apache.flink.runtime.io.network.NetworkEnvironment; import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService; import org.apache.flink.runtime.memory.MemoryManager; -import org.apache.flink.runtime.rpc.TestingRpcService; +import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.TestingSerialRpcService; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.util.TestLogger; @@ -51,8 +52,8 @@ public void testImmediatelyRegistersIfLeaderIsKnown() throws Exception { try { // register a mock resource manager gateway ResourceManagerGateway rmGateway = mock(ResourceManagerGateway.class); - TaskExecutorConfiguration taskExecutorConfiguration = mock(TaskExecutorConfiguration.class); - PowerMockito.when(taskExecutorConfiguration.getNumberOfSlots()).thenReturn(1); + TaskManagerConfiguration taskManagerServicesConfiguration = mock(TaskManagerConfiguration.class); + PowerMockito.when(taskManagerServicesConfiguration.getNumberSlots()).thenReturn(1); rpc.registerGateway(resourceManagerAddress, rmGateway); TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class); @@ -61,12 +62,15 @@ public void testImmediatelyRegistersIfLeaderIsKnown() throws Exception { NonHaServices haServices = new NonHaServices(resourceManagerAddress); TaskExecutor taskManager = new TaskExecutor( - taskExecutorConfiguration, + taskManagerServicesConfiguration, taskManagerLocation, - rpc, mock(MemoryManager.class), + rpc, + mock(MemoryManager.class), mock(IOManager.class), mock(NetworkEnvironment.class), - haServices); + haServices, + mock(MetricRegistry.class), + mock(FatalErrorHandler.class)); taskManager.start(); String taskManagerAddress = taskManager.getAddress(); @@ -101,19 +105,22 @@ public void testTriggerRegistrationOnLeaderChange() throws Exception { TestingHighAvailabilityServices haServices = new TestingHighAvailabilityServices(); haServices.setResourceManagerLeaderRetriever(testLeaderService); - TaskExecutorConfiguration taskExecutorConfiguration = mock(TaskExecutorConfiguration.class); - PowerMockito.when(taskExecutorConfiguration.getNumberOfSlots()).thenReturn(1); + TaskManagerConfiguration taskManagerServicesConfiguration = mock(TaskManagerConfiguration.class); + PowerMockito.when(taskManagerServicesConfiguration.getNumberSlots()).thenReturn(1); TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class); when(taskManagerLocation.getResourceID()).thenReturn(resourceID); TaskExecutor taskManager = new TaskExecutor( - taskExecutorConfiguration, + taskManagerServicesConfiguration, taskManagerLocation, - rpc, mock(MemoryManager.class), + rpc, + mock(MemoryManager.class), mock(IOManager.class), mock(NetworkEnvironment.class), - haServices); + haServices, + mock(MetricRegistry.class), + mock(FatalErrorHandler.class)); taskManager.start(); String taskManagerAddress = taskManager.getAddress(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java index 1f93e9bd24312..627a25a653ada 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java @@ -39,7 +39,6 @@ import org.apache.flink.runtime.io.network.NetworkEnvironment; import org.apache.flink.runtime.io.network.TaskEventDispatcher; import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; -import org.apache.flink.runtime.io.network.netty.NettyConfig; import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; import org.apache.flink.runtime.jobmanager.JobManager; import org.apache.flink.runtime.jobmanager.MemoryArchivist; @@ -105,7 +104,7 @@ public void testComponentsStartupShutdown() { final NetworkEnvironmentConfiguration netConf = new NetworkEnvironmentConfiguration( 32, BUFFER_SIZE, MemoryType.HEAP, IOManager.IOMode.SYNC, 0, 0, 0, - Option.empty(), 0, 0); + null, 0, 0); ResourceID taskManagerId = ResourceID.generate(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerConfigurationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerConfigurationTest.java index acfbbfdd3490a..c0d04551728b1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerConfigurationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerConfigurationTest.java @@ -29,7 +29,6 @@ import scala.Tuple2; import java.io.File; -import java.io.FileNotFoundException; import java.io.IOException; import java.io.PrintWriter; import java.lang.reflect.Field; From 69d08ca4ed877f55279c8177a6b00b8e406eda97 Mon Sep 17 00:00:00 2001 From: beyond1920 Date: Fri, 9 Sep 2016 09:11:24 +0800 Subject: [PATCH 44/50] [FLINK-4606] Integrate the new ResourceManager with the existed FlinkResourceManager --- .../InfoMessageListenerRpcGateway.java | 35 +++ .../resourcemanager/ResourceManager.java | 214 ++++++++++++++++-- .../ResourceManagerGateway.java | 23 ++ .../StandaloneResourceManager.java | 64 ++++++ .../ResourceManagerHATest.java | 2 +- .../ResourceManagerJobMasterTest.java | 2 +- .../ResourceManagerTaskExecutorTest.java | 2 +- .../slotmanager/SlotProtocolTest.java | 5 +- 8 files changed, 318 insertions(+), 29 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/InfoMessageListenerRpcGateway.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/InfoMessageListenerRpcGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/InfoMessageListenerRpcGateway.java new file mode 100644 index 0000000000000..c1eeefa2315f6 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/InfoMessageListenerRpcGateway.java @@ -0,0 +1,35 @@ +/* + * 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.resourcemanager; + +import org.apache.flink.runtime.clusterframework.messages.InfoMessage; +import org.apache.flink.runtime.jobmaster.JobMaster; +import org.apache.flink.runtime.rpc.RpcGateway; + +/** + * A gateway to listen for info messages from {@link ResourceManager} + */ +public interface InfoMessageListenerRpcGateway extends RpcGateway { + + /** + * Notifies when resource manager need to notify listener about InfoMessage + * @param infoMessage + */ + void notifyInfoMessage(InfoMessage infoMessage); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java index 88b8a115eb215..83dc4db11899a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java @@ -20,19 +20,22 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.clusterframework.ApplicationStatus; +import org.apache.flink.runtime.clusterframework.messages.InfoMessage; import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.concurrent.AcceptFunction; +import org.apache.flink.runtime.concurrent.ApplyFunction; import org.apache.flink.runtime.concurrent.BiFunction; import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.jobmaster.JobMasterRegistrationSuccess; import org.apache.flink.runtime.leaderelection.LeaderContender; import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; -import org.apache.flink.runtime.rpc.RpcMethod; import org.apache.flink.runtime.rpc.RpcEndpoint; +import org.apache.flink.runtime.rpc.RpcMethod; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.jobmaster.JobMaster; import org.apache.flink.runtime.jobmaster.JobMasterGateway; @@ -42,8 +45,6 @@ import org.apache.flink.runtime.taskexecutor.TaskExecutorRegistrationSuccess; import org.apache.flink.runtime.util.LeaderConnectionInfo; import org.apache.flink.runtime.util.LeaderRetrievalUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import scala.concurrent.duration.FiniteDuration; import java.util.HashMap; @@ -66,15 +67,16 @@ *

  • {@link #requestSlot(SlotRequest)} requests a slot from the resource manager
  • * */ -public class ResourceManager extends RpcEndpoint implements LeaderContender { +public abstract class ResourceManager extends RpcEndpoint implements LeaderContender { - private final Logger LOG = LoggerFactory.getLogger(getClass()); + /** The exit code with which the process is stopped in case of a fatal error */ + protected static final int EXIT_CODE_FATAL_ERROR = -13; private final Map jobMasterGateways; private final Set jobMasterLeaderRetrievalListeners; - private final Map taskExecutorGateways; + private final Map taskExecutorGateways; private final HighAvailabilityServices highAvailabilityServices; @@ -84,16 +86,16 @@ public class ResourceManager extends RpcEndpoint impleme private UUID leaderSessionID; - public ResourceManager( - RpcService rpcService, - HighAvailabilityServices highAvailabilityServices, - SlotManager slotManager) { + private Map infoMessageListeners; + + public ResourceManager(RpcService rpcService, HighAvailabilityServices highAvailabilityServices, SlotManager slotManager) { super(rpcService); this.highAvailabilityServices = checkNotNull(highAvailabilityServices); this.jobMasterGateways = new HashMap<>(); - this.slotManager = slotManager; + this.slotManager = checkNotNull(slotManager); this.jobMasterLeaderRetrievalListeners = new HashSet<>(); this.taskExecutorGateways = new HashMap<>(); + infoMessageListeners = new HashMap<>(); } @Override @@ -103,6 +105,8 @@ public void start() { super.start(); leaderElectionService = highAvailabilityServices.getResourceManagerLeaderElectionService(); leaderElectionService.start(this); + // framework specific initialization + initialize(); } catch (Throwable e) { log.error("A fatal error happened when starting the ResourceManager", e); throw new RuntimeException("A fatal error happened when starting the ResourceManager", e); @@ -166,12 +170,12 @@ public JobMasterGateway call() throws Exception { jobMasterLeaderInfo = LeaderRetrievalUtils.retrieveLeaderConnectionInfo( highAvailabilityServices.getJobMasterLeaderRetriever(jobID), new FiniteDuration(5, TimeUnit.SECONDS)); } catch (Exception e) { - LOG.warn("Failed to start JobMasterLeaderRetriever for JobID {}", jobID); + log.warn("Failed to start JobMasterLeaderRetriever for JobID {}", jobID); throw new Exception("Failed to retrieve JobMasterLeaderRetriever"); } if (!jobMasterLeaderId.equals(jobMasterLeaderInfo.getLeaderSessionID())) { - LOG.info("Declining registration request from non-leading JobManager {}", jobMasterAddress); + log.info("Declining registration request from non-leading JobManager {}", jobMasterAddress); throw new Exception("JobManager is not leading"); } @@ -190,7 +194,7 @@ public RegistrationResponse apply(JobMasterGateway jobMasterGateway, Throwable t LeaderRetrievalService jobMasterLeaderRetriever = highAvailabilityServices.getJobMasterLeaderRetriever(jobID); jobMasterLeaderRetriever.start(jobMasterLeaderListener); } catch (Exception e) { - LOG.warn("Failed to start JobMasterLeaderRetriever for JobID {}", jobID); + log.warn("Failed to start JobMasterLeaderRetriever for JobID {}", jobID); return new RegistrationResponse.Decline("Failed to retrieve JobMasterLeaderRetriever"); } jobMasterLeaderRetrievalListeners.add(jobMasterLeaderListener); @@ -237,13 +241,24 @@ public RegistrationResponse apply(TaskExecutorGateway taskExecutorGateway, Throw if (throwable != null) { return new RegistrationResponse.Decline(throwable.getMessage()); } else { - InstanceID id = new InstanceID(); - TaskExecutorRegistration oldTaskExecutor = - taskExecutorGateways.put(resourceID, new TaskExecutorRegistration(taskExecutorGateway, id)); - if (oldTaskExecutor != null) { - log.warn("Receive a duplicate registration from TaskExecutor {} at ({})", resourceID, taskExecutorAddress); + WorkerType startedWorker = taskExecutorGateways.get(resourceID); + if(startedWorker != null) { + String oldWorkerAddress = startedWorker.getTaskExecutorGateway().getAddress(); + if (taskExecutorAddress.equals(oldWorkerAddress)) { + log.warn("Receive a duplicate registration from TaskExecutor {} at ({})", resourceID, taskExecutorAddress); + } else { + log.warn("Receive a duplicate registration from TaskExecutor {} at different address, previous ({}), new ({})", + resourceID, oldWorkerAddress, taskExecutorAddress); + // TODO :: suggest old taskExecutor to stop itself + slotManager.notifyTaskManagerFailure(resourceID); + startedWorker = workerStarted(resourceID, taskExecutorGateway); + taskExecutorGateways.put(resourceID, startedWorker); + } + } else { + startedWorker = workerStarted(resourceID, taskExecutorGateway); + taskExecutorGateways.put(resourceID, startedWorker); } - return new TaskExecutorRegistrationSuccess(id, 5000); + return new TaskExecutorRegistrationSuccess(startedWorker.getInstanceID(), 5000); } } }, getMainThreadExecutor()); @@ -263,14 +278,12 @@ public SlotRequestReply requestSlot(SlotRequest slotRequest) { if (jobMasterGateway != null) { return slotManager.requestSlot(slotRequest); } else { - LOG.info("Ignoring slot request for unknown JobMaster with JobID {}", jobId); + log.info("Ignoring slot request for unknown JobMaster with JobID {}", jobId); return new SlotRequestRejected(slotRequest.getAllocationId()); } } - - // ------------------------------------------------------------------------ // Leader Contender // ------------------------------------------------------------------------ @@ -324,6 +337,158 @@ public void handleError(final Exception exception) { shutDown(); } + /** + * Registers an infoMessage listener + * + * @param infoMessageListenerAddress address of infoMessage listener to register to this resource manager + */ + @RpcMethod + public void registerInfoMessageListener(final String infoMessageListenerAddress) { + if(infoMessageListeners.containsKey(infoMessageListenerAddress)) { + log.warn("Receive a duplicate registration from info message listener on ({})", infoMessageListenerAddress); + } else { + Future infoMessageListenerRpcGatewayFuture = getRpcService().connect(infoMessageListenerAddress, InfoMessageListenerRpcGateway.class); + + infoMessageListenerRpcGatewayFuture.thenAcceptAsync(new AcceptFunction() { + @Override + public void accept(InfoMessageListenerRpcGateway gateway) { + log.info("Receive a registration from info message listener on ({})", infoMessageListenerAddress); + infoMessageListeners.put(infoMessageListenerAddress, gateway); + } + }, getMainThreadExecutor()); + + infoMessageListenerRpcGatewayFuture.exceptionallyAsync(new ApplyFunction() { + @Override + public Void apply(Throwable failure) { + log.warn("Receive a registration from unreachable info message listener on ({})", infoMessageListenerAddress); + return null; + } + }, getMainThreadExecutor()); + } + } + + /** + * Unregisters an infoMessage listener + * + * @param infoMessageListenerAddress address of infoMessage listener to unregister from this resource manager + * + */ + @RpcMethod + public void unRegisterInfoMessageListener(final String infoMessageListenerAddress) { + infoMessageListeners.remove(infoMessageListenerAddress); + } + + /** + * Shutdowns cluster + * + * @param finalStatus + * @param optionalDiagnostics + */ + @RpcMethod + public void shutDownCluster(final ApplicationStatus finalStatus, final String optionalDiagnostics) { + log.info("shut down cluster because application is in {}, diagnostics {}", finalStatus, optionalDiagnostics); + shutDownApplication(finalStatus, optionalDiagnostics); + } + + /** + * This method should be called by the framework once it detects that a currently registered task executor has failed. + * + * @param resourceID Id of the worker that has failed. + * @param message An informational message that explains why the worker failed. + */ + public void notifyWorkerFailed(final ResourceID resourceID, String message) { + runAsync(new Runnable() { + @Override + public void run() { + WorkerType worker = taskExecutorGateways.remove(resourceID); + if (worker != null) { + // TODO :: suggest failed task executor to stop itself + slotManager.notifyTaskManagerFailure(resourceID); + } + } + }); + } + + /** + * Gets the number of currently started TaskManagers. + * + * @return The number of currently started TaskManagers. + */ + public int getNumberOfStartedTaskManagers() { + return taskExecutorGateways.size(); + } + + /** + * Notifies the resource manager of a fatal error. + * + *

    IMPORTANT: This should not cleanly shut down this master, but exit it in + * such a way that a high-availability setting would restart this or fail over + * to another master. + */ + public void onFatalError(final String message, final Throwable error) { + runAsync(new Runnable() { + @Override + public void run() { + fatalError(message, error); + } + }); + } + + // ------------------------------------------------------------------------ + // Framework specific behavior + // ------------------------------------------------------------------------ + + /** + * Initializes the framework specific components. + * + * @throws Exception Exceptions during initialization cause the resource manager to fail. + */ + protected abstract void initialize() throws Exception; + + /** + * Callback when a task executor register. + * + * @param resourceID The worker resource id + * @param taskExecutorGateway the task executor gateway + */ + protected abstract WorkerType workerStarted(ResourceID resourceID, TaskExecutorGateway taskExecutorGateway); + + /** + * Callback when a resource manager faced a fatal error + * @param message + * @param error + */ + protected abstract void fatalError(String message, Throwable error); + + /** + * The framework specific code for shutting down the application. This should report the + * application's final status and shut down the resource manager cleanly. + * + * This method also needs to make sure all pending containers that are not registered + * yet are returned. + * + * @param finalStatus The application status to report. + * @param optionalDiagnostics An optional diagnostics message. + */ + protected abstract void shutDownApplication(ApplicationStatus finalStatus, String optionalDiagnostics); + + // ------------------------------------------------------------------------ + // Info messaging + // ------------------------------------------------------------------------ + + public void sendInfoMessage(final String message) { + runAsync(new Runnable() { + @Override + public void run() { + InfoMessage infoMessage = new InfoMessage(message); + for (InfoMessageListenerRpcGateway listenerRpcGateway : infoMessageListeners.values()) { + listenerRpcGateway + .notifyInfoMessage(infoMessage); + } + } + }); + } + private static class JobMasterLeaderListener implements LeaderRetrievalListener { private final JobID jobID; @@ -343,5 +508,6 @@ public void handleError(final Exception exception) { // TODO } } + } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java index 484cea7459f56..7c4400628f989 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.rpc.RpcGateway; @@ -75,4 +76,26 @@ Future registerTaskExecutor( String taskExecutorAddress, ResourceID resourceID, @RpcTimeout Time timeout); + + /** + * Registers an infoMessage listener + * + * @param infoMessageListenerAddress address of infoMessage listener to register to this resource manager + */ + void registerInfoMessageListener(String infoMessageListenerAddress); + + /** + * Unregisters an infoMessage listener + * + * @param infoMessageListenerAddress address of infoMessage listener to unregister from this resource manager + * + */ + void unRegisterInfoMessageListener(String infoMessageListenerAddress); + + /** + * shutdown cluster + * @param finalStatus + * @param optionalDiagnostics + */ + void shutDownCluster(final ApplicationStatus finalStatus, final String optionalDiagnostics); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java new file mode 100644 index 0000000000000..84db1eeeb71d1 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java @@ -0,0 +1,64 @@ +/* + * 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.resourcemanager; + +import org.apache.flink.runtime.clusterframework.ApplicationStatus; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.instance.InstanceID; +import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; +import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; + +/** + * A standalone implementation of the resource manager. Used when the system is started in + * standalone mode (via scripts), rather than via a resource framework like YARN or Mesos. + */ +public class StandaloneResourceManager extends ResourceManager { + + public StandaloneResourceManager(RpcService rpcService, + HighAvailabilityServices highAvailabilityServices, + SlotManager slotManager) { + super(rpcService, highAvailabilityServices, slotManager); + } + + @Override + protected void initialize() throws Exception { + // nothing to initialize + } + + @Override + protected void fatalError(final String message, final Throwable error) { + log.error("FATAL ERROR IN RESOURCE MANAGER: " + message, error); + // kill this process + System.exit(EXIT_CODE_FATAL_ERROR); + } + + @Override + protected TaskExecutorRegistration workerStarted(ResourceID resourceID, TaskExecutorGateway taskExecutorGateway) { + InstanceID instanceID = new InstanceID(); + TaskExecutorRegistration taskExecutorRegistration = new TaskExecutorRegistration(taskExecutorGateway, instanceID); + return taskExecutorRegistration; + } + + @Override + protected void shutDownApplication(ApplicationStatus finalStatus, String optionalDiagnostics) { + + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java index 64a1191211dea..fdb83f52a7b9c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java @@ -55,7 +55,7 @@ public void testGrantAndRevokeLeadership() throws Exception { highAvailabilityServices.setResourceManagerLeaderElectionService(leaderElectionService); SlotManager slotManager = mock(SlotManager.class); - final ResourceManager resourceManager = new ResourceManager(rpcService, highAvailabilityServices, slotManager); + final ResourceManager resourceManager = new StandaloneResourceManager(rpcService, highAvailabilityServices, slotManager); resourceManager.start(); // before grant leadership, resourceManager's leaderId is null Assert.assertNull(resourceManager.getLeaderSessionID()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java index 332c093590f7b..8f09152926a97 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java @@ -160,7 +160,7 @@ private ResourceManager createAndStartResourceManager(TestingLeaderElectionServi TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices(); highAvailabilityServices.setResourceManagerLeaderElectionService(resourceManagerLeaderElectionService); highAvailabilityServices.setJobMasterLeaderRetriever(jobID, jobMasterLeaderRetrievalService); - ResourceManager resourceManager = new ResourceManager(rpcService, highAvailabilityServices, new SimpleSlotManager()); + ResourceManager resourceManager = new StandaloneResourceManager(rpcService, highAvailabilityServices, new SimpleSlotManager()); resourceManager.start(); return resourceManager; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java index ed7c7d7a2cbc0..e6d1ed577e000 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java @@ -121,7 +121,7 @@ private ResourceID mockTaskExecutor(String taskExecutorAddress) { private ResourceManager createAndStartResourceManager(TestingLeaderElectionService rmLeaderElectionService) { TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices(); highAvailabilityServices.setResourceManagerLeaderElectionService(rmLeaderElectionService); - ResourceManager resourceManager = new ResourceManager(rpcService, highAvailabilityServices, new SimpleSlotManager()); + ResourceManager resourceManager = new StandaloneResourceManager(rpcService, highAvailabilityServices, new SimpleSlotManager()); resourceManager.start(); return resourceManager; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java index 0232fab062acf..ff2589725d2aa 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java @@ -35,6 +35,7 @@ import org.apache.flink.runtime.resourcemanager.ResourceManager; import org.apache.flink.runtime.resourcemanager.SlotRequest; import org.apache.flink.runtime.resourcemanager.SlotRequestReply; +import org.apache.flink.runtime.resourcemanager.StandaloneResourceManager; import org.apache.flink.runtime.rpc.TestingSerialRpcService; import org.apache.flink.runtime.taskexecutor.SlotReport; import org.apache.flink.runtime.taskexecutor.SlotStatus; @@ -100,7 +101,7 @@ public void testSlotsUnavailableRequest() throws Exception { TestingSlotManager slotManager = Mockito.spy(new TestingSlotManager()); ResourceManager resourceManager = - new ResourceManager(testRpcService, testingHaServices, slotManager); + new StandaloneResourceManager(testRpcService, testingHaServices, slotManager); resourceManager.start(); rmLeaderElectionService.isLeader(rmLeaderID); @@ -179,7 +180,7 @@ public void testSlotAvailableRequest() throws Exception { TestingSlotManager slotManager = Mockito.spy(new TestingSlotManager()); ResourceManager resourceManager = - new ResourceManager(testRpcService, testingHaServices, slotManager); + new StandaloneResourceManager(testRpcService, testingHaServices, slotManager); resourceManager.start(); rmLeaderElectionService.isLeader(rmLeaderID); From 87767ab4daf06300aac0e5d969e6e44808657187 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Wed, 28 Sep 2016 12:39:30 +0200 Subject: [PATCH 45/50] [FLINK-4703] RpcCompletenessTest: Add support for type arguments and subclasses This closes #2561 --- .../apache/flink/runtime/rpc/RpcEndpoint.java | 23 +++++- .../runtime/rpc/RpcCompletenessTest.java | 80 +++++++++++++++++-- 2 files changed, 94 insertions(+), 9 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java index 4e5e49a527708..79961f7305db9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java @@ -85,9 +85,9 @@ protected RpcEndpoint(final RpcService rpcService) { // IMPORTANT: Don't change order of selfGatewayType and self because rpcService.startServer // requires that selfGatewayType has been initialized - this.selfGatewayType = ReflectionUtil.getTemplateType1(getClass()); + this.selfGatewayType = determineSelfGatewayType(); this.self = rpcService.startServer(this); - + this.mainThreadExecutor = new MainThreadExecutor((MainThreadExecutable) self); } @@ -255,4 +255,23 @@ public void execute(Runnable runnable) { gateway.runAsync(runnable); } } + + /** + * Determines the self gateway type specified in one of the subclasses which extend this class. + * May traverse multiple class hierarchies until a Gateway type is found as a first type argument. + * @return Class The determined self gateway type + */ + private Class determineSelfGatewayType() { + + // determine self gateway type + Class c = getClass(); + Class determinedSelfGatewayType; + do { + determinedSelfGatewayType = ReflectionUtil.getTemplateType1(c); + // check if super class contains self gateway type in next loop + c = c.getSuperclass(); + } while (!RpcGateway.class.isAssignableFrom(determinedSelfGatewayType)); + + return determinedSelfGatewayType; + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java index 53355e805e0af..e7143aea3e708 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java @@ -26,9 +26,14 @@ import org.apache.flink.util.TestLogger; import org.junit.Test; import org.reflections.Reflections; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.lang.annotation.Annotation; import java.lang.reflect.Method; +import java.lang.reflect.Modifier; +import java.lang.reflect.Type; +import java.lang.reflect.TypeVariable; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -41,8 +46,33 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +/** + * Test which ensures that all classes of subtype {@link RpcEndpoint} implement + * the methods specified in the generic gateway type argument. + * + * {@code + * RpcEndpoint + * } + * + * Note, that the class hierarchy can also be nested. In this case the type argument + * always has to be the first argument, e.g. {@code + * + * // RpcClass needs to implement RpcGatewayClass' methods + * RpcClass extends RpcEndpoint + * + * // RpcClass2 or its subclass needs to implement RpcGatewayClass' methods + * RpcClass extends RpcEndpoint + * RpcClass2 extends RpcClass + * + * // needless to say, this can even be nested further + * ... + * } + * + */ public class RpcCompletenessTest extends TestLogger { + private static Logger LOG = LoggerFactory.getLogger(RpcCompletenessTest.class); + private static final Class futureClass = Future.class; private static final Class timeoutClass = Time.class; @@ -55,16 +85,52 @@ public void testRpcCompleteness() { Class c; - for (Class rpcEndpoint :classes){ + mainloop: + for (Class rpcEndpoint : classes) { c = rpcEndpoint; - Class rpcGatewayType = ReflectionUtil.getTemplateType1(c); + LOG.debug("-------------"); + LOG.debug("c: {}", c); - if (rpcGatewayType != null) { - checkCompleteness(rpcEndpoint, (Class) rpcGatewayType); - } else { - fail("Could not retrieve the rpc gateway class for the given rpc endpoint class " + rpcEndpoint.getName()); + // skip abstract classes + if (Modifier.isAbstract(c.getModifiers())) { + LOG.debug("Skipping abstract class"); + continue; } + + // check for type parameter bound to RpcGateway + // skip if one is found because a subclass will provide the concrete argument + TypeVariable>[] typeParameters = c.getTypeParameters(); + LOG.debug("Checking {} parameters.", typeParameters.length); + for (int i = 0; i < typeParameters.length; i++) { + for (Type bound : typeParameters[i].getBounds()) { + LOG.debug("checking bound {} of type parameter {}", bound, typeParameters[i]); + if (bound.toString().equals("interface " + RpcGateway.class.getName())) { + if (i > 0) { + fail("Type parameter for RpcGateway should come first in " + c); + } + LOG.debug("Skipping class with type parameter bound to RpcGateway."); + // Type parameter is bound to RpcGateway which a subclass will provide + continue mainloop; + } + } + } + + // check if this class or any super class contains the RpcGateway argument + Class rpcGatewayType; + do { + LOG.debug("checking type argument of class: {}", c); + rpcGatewayType = ReflectionUtil.getTemplateType1(c); + LOG.debug("type argument is: {}", rpcGatewayType); + + c = (Class) c.getSuperclass(); + + } while (!RpcGateway.class.isAssignableFrom(rpcGatewayType)); + + LOG.debug("Checking RRC completeness of endpoint '{}' with gateway '{}'", + rpcEndpoint.getSimpleName(), rpcGatewayType.getSimpleName()); + + checkCompleteness(rpcEndpoint, (Class) rpcGatewayType); } } @@ -352,7 +418,7 @@ private static Class resolvePrimitiveType(Class primitveType) { */ private List getRpcMethodsFromGateway(Class interfaceClass) { if(!interfaceClass.isInterface()) { - fail(interfaceClass.getName() + "is not a interface"); + fail(interfaceClass.getName() + " is not a interface"); } ArrayList allMethods = new ArrayList<>(); From 387663094adbffb898d3c3cc33eb33c65e18ba64 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Tue, 27 Sep 2016 10:38:02 +0200 Subject: [PATCH 46/50] [FLINK-4606] integrate features of old ResourceManager This closes #2540 --- .../InfoMessageListenerRpcGateway.java | 1 - .../resourcemanager/ResourceManager.java | 146 ++++++++++++------ .../ResourceManagerGateway.java | 6 +- .../ResourceManagerServices.java | 44 ++++++ .../StandaloneResourceManager.java | 19 ++- .../TaskExecutorRegistration.java | 2 +- .../slotmanager/SimpleSlotManager.java | 6 - .../slotmanager/SlotManager.java | 63 ++++++-- .../slotmanager/SlotManagerTest.java | 25 ++- .../slotmanager/SlotProtocolTest.java | 42 ++--- 10 files changed, 245 insertions(+), 109 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerServices.java rename flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/{ => registration}/TaskExecutorRegistration.java (96%) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/InfoMessageListenerRpcGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/InfoMessageListenerRpcGateway.java index c1eeefa2315f6..d1373ec22db07 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/InfoMessageListenerRpcGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/InfoMessageListenerRpcGateway.java @@ -19,7 +19,6 @@ package org.apache.flink.runtime.resourcemanager; import org.apache.flink.runtime.clusterframework.messages.InfoMessage; -import org.apache.flink.runtime.jobmaster.JobMaster; import org.apache.flink.runtime.rpc.RpcGateway; /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java index 83dc4db11899a..190a4de786c08 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java @@ -20,14 +20,18 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.clusterframework.messages.InfoMessage; import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.concurrent.AcceptFunction; import org.apache.flink.runtime.concurrent.ApplyFunction; import org.apache.flink.runtime.concurrent.BiFunction; import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.jobmaster.JobMasterRegistrationSuccess; import org.apache.flink.runtime.leaderelection.LeaderContender; import org.apache.flink.runtime.leaderelection.LeaderElectionService; @@ -48,11 +52,10 @@ import scala.concurrent.duration.FiniteDuration; import java.util.HashMap; -import java.util.HashSet; import java.util.Map; -import java.util.Set; import java.util.UUID; import java.util.concurrent.Callable; +import java.util.concurrent.Executor; import java.util.concurrent.TimeUnit; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -64,36 +67,43 @@ * It offers the following methods as part of its rpc interface to interact with the him remotely: *

      *
    • {@link #registerJobMaster(UUID, UUID, String, JobID)} registers a {@link JobMaster} at the resource manager
    • - *
    • {@link #requestSlot(SlotRequest)} requests a slot from the resource manager
    • + *
    • {@link #requestSlot(UUID, UUID, SlotRequest)} requests a slot from the resource manager
    • *
    */ -public abstract class ResourceManager extends RpcEndpoint implements LeaderContender { +public abstract class ResourceManager + extends RpcEndpoint + implements LeaderContender { /** The exit code with which the process is stopped in case of a fatal error */ protected static final int EXIT_CODE_FATAL_ERROR = -13; private final Map jobMasterGateways; - private final Set jobMasterLeaderRetrievalListeners; + private final Map jobMasterLeaderRetrievalListeners; private final Map taskExecutorGateways; private final HighAvailabilityServices highAvailabilityServices; - private LeaderElectionService leaderElectionService; - private final SlotManager slotManager; + private LeaderElectionService leaderElectionService; + private UUID leaderSessionID; private Map infoMessageListeners; - public ResourceManager(RpcService rpcService, HighAvailabilityServices highAvailabilityServices, SlotManager slotManager) { + private final Time timeout = Time.seconds(5); + + public ResourceManager( + RpcService rpcService, + HighAvailabilityServices highAvailabilityServices, + SlotManager slotManager) { super(rpcService); this.highAvailabilityServices = checkNotNull(highAvailabilityServices); this.jobMasterGateways = new HashMap<>(); this.slotManager = checkNotNull(slotManager); - this.jobMasterLeaderRetrievalListeners = new HashSet<>(); + this.jobMasterLeaderRetrievalListeners = new HashMap<>(); this.taskExecutorGateways = new HashMap<>(); infoMessageListeners = new HashMap<>(); } @@ -105,6 +115,7 @@ public void start() { super.start(); leaderElectionService = highAvailabilityServices.getResourceManagerLeaderElectionService(); leaderElectionService.start(this); + slotManager.setupResourceManagerServices(new DefaultResourceManagerServices()); // framework specific initialization initialize(); } catch (Throwable e) { @@ -117,7 +128,7 @@ public void start() { public void shutDown() { try { leaderElectionService.stop(); - for(JobID jobID : jobMasterGateways.keySet()) { + for (JobID jobID : jobMasterGateways.keySet()) { highAvailabilityServices.getJobMasterLeaderRetriever(jobID).stop(); } super.shutDown(); @@ -189,15 +200,17 @@ public RegistrationResponse apply(JobMasterGateway jobMasterGateway, Throwable t if (throwable != null) { return new RegistrationResponse.Decline(throwable.getMessage()); } else { - JobMasterLeaderListener jobMasterLeaderListener = new JobMasterLeaderListener(jobID); - try { - LeaderRetrievalService jobMasterLeaderRetriever = highAvailabilityServices.getJobMasterLeaderRetriever(jobID); - jobMasterLeaderRetriever.start(jobMasterLeaderListener); - } catch (Exception e) { - log.warn("Failed to start JobMasterLeaderRetriever for JobID {}", jobID); - return new RegistrationResponse.Decline("Failed to retrieve JobMasterLeaderRetriever"); + if (!jobMasterLeaderRetrievalListeners.containsKey(jobID)) { + JobMasterLeaderListener jobMasterLeaderListener = new JobMasterLeaderListener(jobID); + try { + LeaderRetrievalService jobMasterLeaderRetriever = highAvailabilityServices.getJobMasterLeaderRetriever(jobID); + jobMasterLeaderRetriever.start(jobMasterLeaderListener); + } catch (Exception e) { + log.warn("Failed to start JobMasterLeaderRetriever for JobID {}", jobID); + return new RegistrationResponse.Decline("Failed to retrieve JobMasterLeaderRetriever"); + } + jobMasterLeaderRetrievalListeners.put(jobID, jobMasterLeaderListener); } - jobMasterLeaderRetrievalListeners.add(jobMasterLeaderListener); final JobMasterGateway existingGateway = jobMasterGateways.put(jobID, jobMasterGateway); if (existingGateway != null) { log.info("Replacing gateway for registered JobID {}.", jobID); @@ -232,7 +245,6 @@ public TaskExecutorGateway call() throws Exception { resourceID, taskExecutorAddress, leaderSessionID, resourceManagerLeaderId); throw new Exception("Invalid leader session id"); } - return getRpcService().connect(taskExecutorAddress, TaskExecutorGateway.class).get(5, TimeUnit.SECONDS); } }).handleAsync(new BiFunction() { @@ -241,24 +253,14 @@ public RegistrationResponse apply(TaskExecutorGateway taskExecutorGateway, Throw if (throwable != null) { return new RegistrationResponse.Decline(throwable.getMessage()); } else { - WorkerType startedWorker = taskExecutorGateways.get(resourceID); - if(startedWorker != null) { - String oldWorkerAddress = startedWorker.getTaskExecutorGateway().getAddress(); - if (taskExecutorAddress.equals(oldWorkerAddress)) { - log.warn("Receive a duplicate registration from TaskExecutor {} at ({})", resourceID, taskExecutorAddress); - } else { - log.warn("Receive a duplicate registration from TaskExecutor {} at different address, previous ({}), new ({})", - resourceID, oldWorkerAddress, taskExecutorAddress); - // TODO :: suggest old taskExecutor to stop itself - slotManager.notifyTaskManagerFailure(resourceID); - startedWorker = workerStarted(resourceID, taskExecutorGateway); - taskExecutorGateways.put(resourceID, startedWorker); - } - } else { - startedWorker = workerStarted(resourceID, taskExecutorGateway); - taskExecutorGateways.put(resourceID, startedWorker); + WorkerType oldWorker = taskExecutorGateways.remove(resourceID); + if (oldWorker != null) { + // TODO :: suggest old taskExecutor to stop itself + slotManager.notifyTaskManagerFailure(resourceID); } - return new TaskExecutorRegistrationSuccess(startedWorker.getInstanceID(), 5000); + WorkerType newWorker = workerStarted(resourceID); + taskExecutorGateways.put(resourceID, newWorker); + return new TaskExecutorRegistrationSuccess(new InstanceID(), 5000); } } }, getMainThreadExecutor()); @@ -271,11 +273,20 @@ public RegistrationResponse apply(TaskExecutorGateway taskExecutorGateway, Throw * @return Slot assignment */ @RpcMethod - public SlotRequestReply requestSlot(SlotRequest slotRequest) { - final JobID jobId = slotRequest.getJobId(); - final JobMasterGateway jobMasterGateway = jobMasterGateways.get(jobId); + public SlotRequestReply requestSlot( + UUID jobMasterLeaderID, + UUID resourceManagerLeaderID, + SlotRequest slotRequest) { + + JobID jobId = slotRequest.getJobId(); + JobMasterGateway jobMasterGateway = jobMasterGateways.get(jobId); + JobMasterLeaderListener jobMasterLeaderListener = jobMasterLeaderRetrievalListeners.get(jobId); + + UUID leaderID = jobMasterLeaderListener.getLeaderID(); - if (jobMasterGateway != null) { + if (jobMasterGateway != null + && jobMasterLeaderID.equals(leaderID) + && resourceManagerLeaderID.equals(leaderSessionID)) { return slotManager.requestSlot(slotRequest); } else { log.info("Ignoring slot request for unknown JobMaster with JobID {}", jobId); @@ -379,7 +390,7 @@ public void unRegisterInfoMessageListener(final String infoMessageListenerAddres } /** - * Shutdowns cluster + * Cleanup application and shut down cluster * * @param finalStatus * @param optionalDiagnostics @@ -446,17 +457,11 @@ public void run() { protected abstract void initialize() throws Exception; /** - * Callback when a task executor register. + * Notifies the resource master of a fatal error. * - * @param resourceID The worker resource id - * @param taskExecutorGateway the task executor gateway - */ - protected abstract WorkerType workerStarted(ResourceID resourceID, TaskExecutorGateway taskExecutorGateway); - - /** - * Callback when a resource manager faced a fatal error - * @param message - * @param error + *

    IMPORTANT: This should not cleanly shut down this master, but exit it in + * such a way that a high-availability setting would restart this or fail over + * to another master. */ protected abstract void fatalError(String message, Throwable error); @@ -472,6 +477,19 @@ public void run() { */ protected abstract void shutDownApplication(ApplicationStatus finalStatus, String optionalDiagnostics); + /** + * Allocates a resource using the resource profile. + * @param resourceProfile The resource description + */ + @VisibleForTesting + public abstract void startNewWorker(ResourceProfile resourceProfile); + + /** + * Callback when a worker was started. + * @param resourceID The worker resource id + */ + protected abstract WorkerType workerStarted(ResourceID resourceID); + // ------------------------------------------------------------------------ // Info messaging // ------------------------------------------------------------------------ @@ -489,6 +507,24 @@ public void run() { }); } + private class DefaultResourceManagerServices implements ResourceManagerServices { + + @Override + public void allocateResource(ResourceProfile resourceProfile) { + ResourceManager.this.startNewWorker(resourceProfile); + } + + @Override + public Executor getAsyncExecutor() { + return ResourceManager.this.getRpcService().getExecutor(); + } + + @Override + public Executor getExecutor() { + return ResourceManager.this.getMainThreadExecutor(); + } + } + private static class JobMasterLeaderListener implements LeaderRetrievalListener { private final JobID jobID; @@ -498,6 +534,14 @@ private JobMasterLeaderListener(JobID jobID) { this.jobID = jobID; } + public JobID getJobID() { + return jobID; + } + + public UUID getLeaderID() { + return leaderID; + } + @Override public void notifyLeaderAddress(final String leaderAddress, final UUID leaderSessionID) { this.leaderID = leaderSessionID; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java index 7c4400628f989..87303a17581e4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java @@ -59,7 +59,11 @@ Future registerJobMaster( * @param slotRequest Slot request * @return Future slot assignment */ - Future requestSlot(SlotRequest slotRequest); + Future requestSlot( + UUID jobMasterLeaderID, + UUID resourceManagerLeaderID, + SlotRequest slotRequest, + @RpcTimeout Time timeout); /** * Register a {@link org.apache.flink.runtime.taskexecutor.TaskExecutor} at the resource manager. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerServices.java new file mode 100644 index 0000000000000..30994dc1a5154 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerServices.java @@ -0,0 +1,44 @@ +/* + * 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.resourcemanager; + +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; + +import java.util.concurrent.Executor; + +/** + * Interface which provides access to services of the ResourceManager. + */ +public interface ResourceManagerServices { + + /** + * Allocates a resource according to the resource profile. + */ + void allocateResource(ResourceProfile resourceProfile); + + /** + * Gets the async excutor which executes outside of the main thread of the ResourceManager + */ + Executor getAsyncExecutor(); + + /** + * Gets the executor which executes in the main thread of the ResourceManager + */ + Executor getExecutor(); + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java index 84db1eeeb71d1..deca8d3702984 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java @@ -20,17 +20,18 @@ import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; import org.apache.flink.runtime.rpc.RpcService; -import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; /** * A standalone implementation of the resource manager. Used when the system is started in * standalone mode (via scripts), rather than via a resource framework like YARN or Mesos. + * + * This ResourceManager doesn't acquire new resources. */ -public class StandaloneResourceManager extends ResourceManager { +public class StandaloneResourceManager extends ResourceManager { public StandaloneResourceManager(RpcService rpcService, HighAvailabilityServices highAvailabilityServices, @@ -51,14 +52,16 @@ protected void fatalError(final String message, final Throwable error) { } @Override - protected TaskExecutorRegistration workerStarted(ResourceID resourceID, TaskExecutorGateway taskExecutorGateway) { - InstanceID instanceID = new InstanceID(); - TaskExecutorRegistration taskExecutorRegistration = new TaskExecutorRegistration(taskExecutorGateway, instanceID); - return taskExecutorRegistration; + protected void shutDownApplication(ApplicationStatus finalStatus, String optionalDiagnostics) { } @Override - protected void shutDownApplication(ApplicationStatus finalStatus, String optionalDiagnostics) { + public void startNewWorker(ResourceProfile resourceProfile) { + } + @Override + protected ResourceID workerStarted(ResourceID resourceID) { + return resourceID; } + } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/TaskExecutorRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/registration/TaskExecutorRegistration.java similarity index 96% rename from flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/TaskExecutorRegistration.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/registration/TaskExecutorRegistration.java index f8dfdc7f9f809..6b21f5cd7d626 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/TaskExecutorRegistration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/registration/TaskExecutorRegistration.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.resourcemanager; +package org.apache.flink.runtime.resourcemanager.registration; import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SimpleSlotManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SimpleSlotManager.java index ef5ce3128d85a..ae1de5a00f644 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SimpleSlotManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SimpleSlotManager.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.resourcemanager.slotmanager; import org.apache.flink.runtime.clusterframework.types.AllocationID; -import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.clusterframework.types.ResourceSlot; import org.apache.flink.runtime.clusterframework.types.SlotID; import org.apache.flink.runtime.resourcemanager.SlotRequest; @@ -51,9 +50,4 @@ protected SlotRequest chooseRequestToFulfill(ResourceSlot offeredSlot, Map(16); this.pendingSlotRequests = new LinkedHashMap<>(16); @@ -91,6 +96,16 @@ public SlotManager() { this.timeout = Time.seconds(10); } + /** + * Initializes the resource supplier which is needed to request new resources. + */ + public void setupResourceManagerServices(ResourceManagerServices resourceManagerServices) { + if (this.resourceManagerServices != null) { + throw new IllegalStateException("ResourceManagerServices may only be set once."); + } + this.resourceManagerServices = resourceManagerServices; + } + // ------------------------------------------------------------------------ // slot managements @@ -120,17 +135,32 @@ public SlotRequestRegistered requestSlot(final SlotRequest request) { // record this allocation in bookkeeping allocationMap.addAllocation(slot.getSlotId(), allocationId); - // remove selected slot from free pool - freeSlots.remove(slot.getSlotId()); + final ResourceSlot removedSlot = freeSlots.remove(slot.getSlotId()); final Future slotRequestReplyFuture = slot.getTaskExecutorGateway().requestSlot(allocationId, leaderID, timeout); - // TODO handle timeouts and response + + slotRequestReplyFuture.handleAsync(new BiFunction() { + @Override + public Object apply(SlotRequestReply slotRequestReply, Throwable throwable) { + if (throwable != null) { + // we failed, put the slot and the request back again + if (allocationMap.isAllocated(slot.getSlotId())) { + // only re-add if the slot hasn't been removed in the meantime + freeSlots.put(slot.getSlotId(), removedSlot); + } + pendingSlotRequests.put(allocationId, request); + } + return null; + } + }, resourceManagerServices.getExecutor()); } else { LOG.info("Cannot fulfil slot request, try to allocate a new container for it, " + "AllocationID:{}, JobID:{}", allocationId, request.getJobId()); - allocateContainer(request.getResourceProfile()); + Preconditions.checkState(resourceManagerServices != null, + "Attempted to allocate resources but no ResourceManagerServices set."); + resourceManagerServices.allocateResource(request.getResourceProfile()); pendingSlotRequests.put(allocationId, request); } @@ -343,7 +373,7 @@ private void handleFreeSlot(final ResourceSlot freeSlot) { if (chosenRequest != null) { final AllocationID allocationId = chosenRequest.getAllocationId(); - pendingSlotRequests.remove(allocationId); + final SlotRequest removedSlotRequest = pendingSlotRequests.remove(allocationId); LOG.info("Assigning SlotID({}) to AllocationID({}), JobID:{}", freeSlot.getSlotId(), allocationId, chosenRequest.getJobId()); @@ -351,7 +381,19 @@ private void handleFreeSlot(final ResourceSlot freeSlot) { final Future slotRequestReplyFuture = freeSlot.getTaskExecutorGateway().requestSlot(allocationId, leaderID, timeout); - // TODO handle timeouts and response + + slotRequestReplyFuture.handleAsync(new BiFunction() { + @Override + public Object apply(SlotRequestReply slotRequestReply, Throwable throwable) { + if (throwable != null) { + // we failed, add the request back again + if (allocationMap.isAllocated(freeSlot.getSlotId())) { + pendingSlotRequests.put(allocationId, removedSlotRequest); + } + } + return null; + } + }, resourceManagerServices.getExecutor()); } else { freeSlots.put(freeSlot.getSlotId(), freeSlot); } @@ -417,13 +459,6 @@ protected abstract ResourceSlot chooseSlotToUse(final SlotRequest request, protected abstract SlotRequest chooseRequestToFulfill(final ResourceSlot offeredSlot, final Map pendingRequests); - /** - * The framework specific code for allocating a container for specified resource profile. - * - * @param resourceProfile The resource profile - */ - protected abstract void allocateContainer(final ResourceProfile resourceProfile); - // ------------------------------------------------------------------------ // Helper classes // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerTest.java index 9ee96903ad028..0fed79e057edf 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerTest.java @@ -19,12 +19,16 @@ package org.apache.flink.runtime.resourcemanager.slotmanager; import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.clusterframework.types.ResourceSlot; import org.apache.flink.runtime.clusterframework.types.SlotID; +import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture; +import org.apache.flink.runtime.resourcemanager.ResourceManagerServices; import org.apache.flink.runtime.resourcemanager.SlotRequest; +import org.apache.flink.runtime.resourcemanager.SlotRequestReply; import org.apache.flink.runtime.taskexecutor.SlotStatus; import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; import org.junit.BeforeClass; @@ -34,10 +38,13 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.UUID; +import java.util.concurrent.Executor; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.any; import static org.mockito.Mockito.mock; public class SlotManagerTest { @@ -57,6 +64,8 @@ public class SlotManagerTest { @BeforeClass public static void setUp() { taskExecutorGateway = Mockito.mock(TaskExecutorGateway.class); + Mockito.when(taskExecutorGateway.requestSlot(any(AllocationID.class), any(UUID.class), any(Time.class))) + .thenReturn(new FlinkCompletableFuture()); } /** @@ -498,12 +507,13 @@ private void directlyProvideFreeSlots( // testing classes // ------------------------------------------------------------------------ - private static class TestingSlotManager extends SlotManager { + private static class TestingSlotManager extends SlotManager implements ResourceManagerServices { private final List allocatedContainers; TestingSlotManager() { this.allocatedContainers = new LinkedList<>(); + setupResourceManagerServices(this); } /** @@ -543,12 +553,23 @@ protected SlotRequest chooseRequestToFulfill(ResourceSlot offeredSlot, } @Override - protected void allocateContainer(ResourceProfile resourceProfile) { + public void allocateResource(ResourceProfile resourceProfile) { allocatedContainers.add(resourceProfile); } + @Override + public Executor getAsyncExecutor() { + return Mockito.mock(Executor.class); + } + + @Override + public Executor getExecutor() { + return Mockito.mock(Executor.class); + } + List getAllocatedContainers() { return allocatedContainers; } + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java index ff2589725d2aa..e3018c9265f90 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java @@ -24,18 +24,14 @@ import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.clusterframework.types.SlotID; import org.apache.flink.runtime.concurrent.Future; -import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.highavailability.NonHaServices; +import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture; +import org.apache.flink.runtime.concurrent.impl.FlinkFuture; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.jobmaster.JobMasterGateway; -import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService; import org.apache.flink.runtime.registration.RegistrationResponse; -import org.apache.flink.runtime.resourcemanager.ResourceManager; -import org.apache.flink.runtime.resourcemanager.SlotRequest; -import org.apache.flink.runtime.resourcemanager.SlotRequestReply; -import org.apache.flink.runtime.resourcemanager.StandaloneResourceManager; +import org.apache.flink.runtime.resourcemanager.*; import org.apache.flink.runtime.rpc.TestingSerialRpcService; import org.apache.flink.runtime.taskexecutor.SlotReport; import org.apache.flink.runtime.taskexecutor.SlotStatus; @@ -47,9 +43,12 @@ import org.junit.BeforeClass; import org.junit.Test; import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; import java.util.Collections; import java.util.UUID; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import static org.mockito.Matchers.any; @@ -99,9 +98,9 @@ public void testSlotsUnavailableRequest() throws Exception { TestingLeaderElectionService rmLeaderElectionService = configureHA(testingHaServices, jobID, rmAddress, rmLeaderID, jmAddress, jmLeaderID); - TestingSlotManager slotManager = Mockito.spy(new TestingSlotManager()); + SlotManager slotManager = Mockito.spy(new SimpleSlotManager()); ResourceManager resourceManager = - new StandaloneResourceManager(testRpcService, testingHaServices, slotManager); + Mockito.spy(new StandaloneResourceManager(testRpcService, testingHaServices, slotManager)); resourceManager.start(); rmLeaderElectionService.isLeader(rmLeaderID); @@ -118,7 +117,7 @@ public void testSlotsUnavailableRequest() throws Exception { SlotRequest slotRequest = new SlotRequest(jobID, allocationID, resourceProfile); SlotRequestReply slotRequestReply = - resourceManager.requestSlot(slotRequest); + resourceManager.requestSlot(jmLeaderID, rmLeaderID, slotRequest); // 1) SlotRequest is routed to the SlotManager verify(slotManager).requestSlot(slotRequest); @@ -129,13 +128,15 @@ public void testSlotsUnavailableRequest() throws Exception { allocationID); // 3) SlotRequest leads to a container allocation - verify(slotManager, timeout(5000)).allocateContainer(resourceProfile); + verify(resourceManager, timeout(5000)).startNewWorker(resourceProfile); Assert.assertFalse(slotManager.isAllocated(allocationID)); // slot becomes available final String tmAddress = "/tm1"; TaskExecutorGateway taskExecutorGateway = mock(TaskExecutorGateway.class); + Mockito.when(taskExecutorGateway.requestSlot(any(AllocationID.class), any(UUID.class), any(Time.class))) + .thenReturn(new FlinkCompletableFuture()); testRpcService.registerGateway(tmAddress, taskExecutorGateway); final ResourceID resourceID = ResourceID.generate(); @@ -176,11 +177,13 @@ public void testSlotAvailableRequest() throws Exception { configureHA(testingHaServices, jobID, rmAddress, rmLeaderID, jmAddress, jmLeaderID); TaskExecutorGateway taskExecutorGateway = mock(TaskExecutorGateway.class); + Mockito.when(taskExecutorGateway.requestSlot(any(AllocationID.class), any(UUID.class), any(Time.class))) + .thenReturn(new FlinkCompletableFuture()); testRpcService.registerGateway(tmAddress, taskExecutorGateway); - TestingSlotManager slotManager = Mockito.spy(new TestingSlotManager()); + SlotManager slotManager = Mockito.spy(new SimpleSlotManager()); ResourceManager resourceManager = - new StandaloneResourceManager(testRpcService, testingHaServices, slotManager); + Mockito.spy(new StandaloneResourceManager(testRpcService, testingHaServices, slotManager)); resourceManager.start(); rmLeaderElectionService.isLeader(rmLeaderID); @@ -207,7 +210,7 @@ public void testSlotAvailableRequest() throws Exception { SlotRequest slotRequest = new SlotRequest(jobID, allocationID, resourceProfile); SlotRequestReply slotRequestReply = - resourceManager.requestSlot(slotRequest); + resourceManager.requestSlot(jmLeaderID, rmLeaderID, slotRequest); // 1) a SlotRequest is routed to the SlotManager verify(slotManager).requestSlot(slotRequest); @@ -241,15 +244,4 @@ private static TestingLeaderElectionService configureHA( return rmLeaderElectionService; } - private static class TestingSlotManager extends SimpleSlotManager { - - // change visibility of function to public for testing - @Override - public void allocateContainer(ResourceProfile resourceProfile) { - super.allocateContainer(resourceProfile); - } - - - } - } From cf1e875d7ec994fc34271cb50b380c274d0895b0 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 28 Sep 2016 14:04:54 +0200 Subject: [PATCH 47/50] [hotfix] Replace TaskManager.createTaskManagerComponents by TaskManagerServices --- .../clusterframework/MesosTaskManager.scala | 3 +- .../TaskManagerConfiguration.java | 25 +- .../TaskManagerServicesConfiguration.java | 2 +- .../minicluster/LocalFlinkMiniCluster.scala | 47 +- .../runtime/taskmanager/TaskManager.scala | 609 ++---------------- .../TaskManagerConfiguration.scala | 56 -- ...kManagerComponentsStartupShutdownTest.java | 24 +- .../testingUtils/TestingTaskManager.scala | 3 +- .../runtime/testingUtils/TestingUtils.scala | 1 - .../flink/yarn/TestingYarnTaskManager.scala | 3 +- .../apache/flink/yarn/YarnTaskManager.scala | 3 +- 11 files changed, 129 insertions(+), 647 deletions(-) delete mode 100644 flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManagerConfiguration.scala diff --git a/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala b/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala index 3972a579f4efc..e8d6a58601772 100644 --- a/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala +++ b/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala @@ -24,7 +24,8 @@ import org.apache.flink.runtime.io.network.NetworkEnvironment import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService import org.apache.flink.runtime.memory.MemoryManager import org.apache.flink.runtime.metrics.MetricRegistry -import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerConfiguration, TaskManagerLocation} +import org.apache.flink.runtime.taskexecutor.TaskManagerConfiguration +import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerLocation} /** An extension of the TaskManager that listens for additional Mesos-related * messages. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java index 32eb8c17325f3..f58af77980ce5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java @@ -41,6 +41,7 @@ public class TaskManagerConfiguration { private final String[] tmpDirPaths; private final Time timeout; + // null indicates an infinite duration private final Time maxRegistrationDuration; private final Time initialRegistrationPause; private final Time maxRegistrationPause; @@ -48,6 +49,9 @@ public class TaskManagerConfiguration { private final long cleanupInterval; + // TODO: remove necessity for complete configuration object + private final Configuration configuration; + public TaskManagerConfiguration( int numberSlots, String[] tmpDirPaths, @@ -56,16 +60,18 @@ public TaskManagerConfiguration( Time initialRegistrationPause, Time maxRegistrationPause, Time refusedRegistrationPause, - long cleanupInterval) { + long cleanupInterval, + Configuration configuration) { this.numberSlots = numberSlots; this.tmpDirPaths = Preconditions.checkNotNull(tmpDirPaths); this.timeout = Preconditions.checkNotNull(timeout); - this.maxRegistrationDuration = Preconditions.checkNotNull(maxRegistrationDuration); + this.maxRegistrationDuration = maxRegistrationDuration; this.initialRegistrationPause = Preconditions.checkNotNull(initialRegistrationPause); this.maxRegistrationPause = Preconditions.checkNotNull(maxRegistrationPause); this.refusedRegistrationPause = Preconditions.checkNotNull(refusedRegistrationPause); this.cleanupInterval = Preconditions.checkNotNull(cleanupInterval); + this.configuration = Preconditions.checkNotNull(configuration); } public int getNumberSlots() { @@ -100,6 +106,10 @@ public long getCleanupInterval() { return cleanupInterval; } + public Configuration getConfiguration() { + return configuration; + } + // -------------------------------------------------------------------------------------------- // Static factory methods // -------------------------------------------------------------------------------------------- @@ -138,7 +148,7 @@ public static TaskManagerConfiguration fromConfiguration(Configuration configura ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION, ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_DURATION)); if (maxRegistrationDuration.isFinite()) { - finiteRegistrationDuration = Time.seconds(maxRegistrationDuration.toSeconds()); + finiteRegistrationDuration = Time.milliseconds(maxRegistrationDuration.toMillis()); } else { finiteRegistrationDuration = null; } @@ -153,7 +163,7 @@ public static TaskManagerConfiguration fromConfiguration(Configuration configura ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, ConfigConstants.DEFAULT_TASK_MANAGER_INITIAL_REGISTRATION_PAUSE)); if (pause.isFinite()) { - initialRegistrationPause = Time.seconds(pause.toSeconds()); + initialRegistrationPause = Time.milliseconds(pause.toMillis()); } else { throw new IllegalArgumentException("The initial registration pause must be finite: " + pause); } @@ -168,7 +178,7 @@ public static TaskManagerConfiguration fromConfiguration(Configuration configura ConfigConstants.TASK_MANAGER_MAX_REGISTARTION_PAUSE, ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_PAUSE)); if (pause.isFinite()) { - maxRegistrationPause = Time.seconds(pause.toSeconds()); + maxRegistrationPause = Time.milliseconds(pause.toMillis()); } else { throw new IllegalArgumentException("The maximum registration pause must be finite: " + pause); } @@ -183,7 +193,7 @@ public static TaskManagerConfiguration fromConfiguration(Configuration configura ConfigConstants.TASK_MANAGER_REFUSED_REGISTRATION_PAUSE, ConfigConstants.DEFAULT_TASK_MANAGER_REFUSED_REGISTRATION_PAUSE)); if (pause.isFinite()) { - refusedRegistrationPause = Time.seconds(pause.toSeconds()); + refusedRegistrationPause = Time.milliseconds(pause.toMillis()); } else { throw new IllegalArgumentException("The refused registration pause must be finite: " + pause); } @@ -200,6 +210,7 @@ public static TaskManagerConfiguration fromConfiguration(Configuration configura initialRegistrationPause, maxRegistrationPause, refusedRegistrationPause, - cleanupInterval); + cleanupInterval, + configuration); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java index 66d969ad005a8..80dfc09f30fde 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java @@ -208,7 +208,7 @@ private static NetworkEnvironmentConfiguration parseNetworkEnvironmentConfigurat int dataport = configuration.getInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT); - checkConfigParameter(dataport > 0, dataport, ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, + checkConfigParameter(dataport >= 0, dataport, ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, "Leave config parameter empty or use 0 to let the system choose a port automatically."); checkConfigParameter(slots >= 1, slots, ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala index 27c9dd91faba0..d29f73b58eb83 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala @@ -18,6 +18,7 @@ package org.apache.flink.runtime.minicluster +import java.net.InetAddress import java.util.concurrent.ExecutorService import akka.actor.{ActorRef, ActorSystem, Props} @@ -43,8 +44,9 @@ import org.apache.flink.runtime.memory.MemoryManager import org.apache.flink.runtime.messages.JobManagerMessages import org.apache.flink.runtime.messages.JobManagerMessages.{RunningJobsStatus, StoppingFailure, StoppingResponse} import org.apache.flink.runtime.metrics.MetricRegistry -import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerConfiguration, TaskManagerLocation} -import org.apache.flink.runtime.util.EnvironmentInformation +import org.apache.flink.runtime.taskexecutor.{TaskManagerConfiguration, TaskManagerServices, TaskManagerServicesConfiguration} +import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerLocation} +import org.apache.flink.runtime.util.{EnvironmentInformation, LeaderRetrievalUtils} import scala.concurrent.Await import scala.concurrent.duration.FiniteDuration @@ -198,31 +200,32 @@ class LocalFlinkMiniCluster( val resourceID = ResourceID.generate() // generate random resource id - val (taskManagerConfig, - taskManagerLocation, - memoryManager, - ioManager, - network, - leaderRetrievalService, - metricsRegistry) = TaskManager.createTaskManagerComponents( + val taskManagerAddress = InetAddress.getByName(hostname) + + val taskManagerConfiguration = TaskManagerConfiguration.fromConfiguration(config) + val taskManagerServicesConfiguration = TaskManagerServicesConfiguration.fromConfiguration( config, - resourceID, - hostname, // network interface to bind to - localExecution, // start network stack? - Some(createLeaderRetrievalService())) + taskManagerAddress, + localExecution) + + val taskManagerServices = TaskManagerServices.fromConfiguration( + taskManagerServicesConfiguration, + resourceID) + + val metricRegistry = taskManagerServices.getMetricRegistry() val props = getTaskManagerProps( taskManagerClass, - taskManagerConfig, + taskManagerConfiguration, resourceID, - taskManagerLocation, - memoryManager, - ioManager, - network, - leaderRetrievalService, - metricsRegistry) - - metricsRegistry.startQueryService(system) + taskManagerServices.getTaskManagerLocation(), + taskManagerServices.getMemoryManager(), + taskManagerServices.getIOManager(), + taskManagerServices.getNetworkEnvironment, + createLeaderRetrievalService(), + metricRegistry) + + metricRegistry.startQueryService(system) system.actorOf(props, taskManagerActorName) } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala index 21964b53a2e00..12391b8541ffb 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala @@ -37,7 +37,6 @@ import com.fasterxml.jackson.databind.ObjectMapper import grizzled.slf4j.Logger import org.apache.flink.configuration._ import org.apache.flink.core.fs.FileSystem -import org.apache.flink.core.memory.{HeapMemorySegment, HybridMemorySegment, MemorySegmentFactory, MemoryType} import org.apache.flink.metrics.{MetricGroup, Gauge => FlinkGauge} import org.apache.flink.runtime.accumulators.AccumulatorSnapshot import org.apache.flink.runtime.clusterframework.messages.StopCluster @@ -51,12 +50,8 @@ import org.apache.flink.runtime.execution.librarycache.{BlobLibraryCacheManager, import org.apache.flink.runtime.executiongraph.ExecutionAttemptID import org.apache.flink.runtime.filecache.FileCache import org.apache.flink.runtime.instance.{AkkaActorGateway, HardwareDescription, InstanceID} -import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode -import org.apache.flink.runtime.io.disk.iomanager.{IOManager, IOManagerAsync} -import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool -import org.apache.flink.runtime.io.network.{LocalConnectionManager, NetworkEnvironment, TaskEventDispatcher} -import org.apache.flink.runtime.io.network.netty.{NettyConfig, NettyConnectionManager} -import org.apache.flink.runtime.io.network.partition.ResultPartitionManager +import org.apache.flink.runtime.io.disk.iomanager.IOManager +import org.apache.flink.runtime.io.network.NetworkEnvironment import org.apache.flink.runtime.jobgraph.IntermediateDataSetID import org.apache.flink.runtime.leaderretrieval.{LeaderRetrievalListener, LeaderRetrievalService} import org.apache.flink.runtime.memory.MemoryManager @@ -66,16 +61,15 @@ import org.apache.flink.runtime.messages.StackTraceSampleMessages.{ResponseStack import org.apache.flink.runtime.messages.TaskManagerMessages._ import org.apache.flink.runtime.messages.TaskMessages._ import org.apache.flink.runtime.messages.checkpoint.{AbstractCheckpointMessage, NotifyCheckpointComplete, TriggerCheckpoint} -import org.apache.flink.runtime.metrics.{MetricRegistryConfiguration, MetricRegistry => FlinkMetricRegistry} +import org.apache.flink.runtime.metrics.{MetricRegistry => FlinkMetricRegistry} import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup import org.apache.flink.runtime.process.ProcessReaper -import org.apache.flink.runtime.query.KvStateRegistry -import org.apache.flink.runtime.query.netty.{DisabledKvStateRequestStats, KvStateServer} import org.apache.flink.runtime.security.SecurityContext.{FlinkSecuredRunner, SecurityConfiguration} import org.apache.flink.runtime.security.SecurityContext +import org.apache.flink.runtime.taskexecutor.{TaskManagerConfiguration, TaskManagerServices, TaskManagerServicesConfiguration} import org.apache.flink.runtime.util._ import org.apache.flink.runtime.{FlinkActor, LeaderSessionMessageFilter, LogMessages} -import org.apache.flink.util.{MathUtils, NetUtils} +import org.apache.flink.util.NetUtils import scala.collection.JavaConverters._ import scala.concurrent._ @@ -142,7 +136,7 @@ class TaskManager( override val log = Logger(getClass) /** The timeout for all actor ask futures */ - protected val askTimeout = new Timeout(config.timeout) + protected val askTimeout = new Timeout(config.getTimeout().getSize, config.getTimeout().getUnit()) /** The TaskManager's physical execution resources */ protected val resources = HardwareDescription.extractFromSystem(memoryManager.getMemorySize()) @@ -154,7 +148,7 @@ class TaskManager( protected val bcVarManager = new BroadcastVariableManager() /** Handler for distributed files cached by this TaskManager */ - protected val fileCache = new FileCache(config.configuration) + protected val fileCache = new FileCache(config.getConfiguration()) /** Registry of metrics periodically transmitted to the JobManager */ private val metricRegistry = TaskManager.createMetricsRegistry() @@ -190,8 +184,8 @@ class TaskManager( private val runtimeInfo = new TaskManagerRuntimeInfo( location.getHostname(), - new UnmodifiableConfiguration(config.configuration), - config.tmpDirPaths) + new UnmodifiableConfiguration(config.getConfiguration()), + config.getTmpDirPaths()) private var scheduledTaskManagerRegistration: Option[Cancellable] = None private var currentRegistrationRun: UUID = UUID.randomUUID() @@ -618,7 +612,9 @@ class TaskManager( ) // the next timeout computes via exponential backoff with cap - val nextTimeout = (timeout * 2).min(config.maxRegistrationPause) + val nextTimeout = (timeout * 2).min(new FiniteDuration( + config.getMaxRegistrationPause().toMilliseconds, + TimeUnit.MILLISECONDS)) // schedule (with our timeout s delay) a check triggers a new registration // attempt, if we are not registered by then @@ -692,10 +688,14 @@ class TaskManager( if(jobManagerAkkaURL.isDefined) { // try the registration again after some time - val delay: FiniteDuration = config.refusedRegistrationPause - val deadline: Option[Deadline] = config.maxRegistrationDuration.map { - timeout => timeout + delay fromNow - } + val delay: FiniteDuration = new FiniteDuration( + config.getRefusedRegistrationPause().getSize(), + config.getRefusedRegistrationPause().getUnit()) + val deadline: Option[Deadline] = Option(config.getMaxRegistrationDuration()) + .map { + duration => new FiniteDuration(duration.getSize(), duration.getUnit()) + + delay fromNow + } // start a new registration run currentRegistrationRun = UUID.randomUUID() @@ -707,7 +707,9 @@ class TaskManager( self ! decorateMessage( TriggerTaskManagerRegistration( jobManagerAkkaURL.get, - config.initialRegistrationPause, + new FiniteDuration( + config.getInitialRegistrationPause().getSize(), + config.getInitialRegistrationPause().getUnit()), deadline, 1, currentRegistrationRun) @@ -846,7 +848,7 @@ class TaskManager( requestType: LogTypeRequest, jobManager: ActorRef) : Unit = { - val logFilePathOption = Option(config.configuration.getString( + val logFilePathOption = Option(config.getConfiguration().getString( ConfigConstants.TASK_MANAGER_LOG_PATH_KEY, System.getProperty("log.file"))); logFilePathOption match { case None => throw new IOException("TaskManager log files are unavailable. " + @@ -945,7 +947,9 @@ class TaskManager( context.dispatcher, jobManagerGateway, taskmanagerGateway, - config.timeout)) + new FiniteDuration( + config.getTimeout().getSize(), + config.getTimeout.getUnit()))) val kvStateServer = network.getKvStateServer() @@ -967,9 +971,10 @@ class TaskManager( log.info(s"Determined BLOB server address to be $address. Starting BLOB cache.") try { - val blobcache = new BlobCache(address, config.configuration) + val blobcache = new BlobCache(address, config.getConfiguration()) blobService = Option(blobcache) - libraryCacheManager = Some(new BlobLibraryCacheManager(blobcache, config.cleanupInterval)) + libraryCacheManager = Some( + new BlobLibraryCacheManager(blobcache, config.getCleanupInterval())) } catch { case e: Exception => @@ -1151,7 +1156,9 @@ class TaskManager( tdd.getJobID, tdd.getVertexID, tdd.getExecutionId, - config.timeout) + new FiniteDuration( + config.getTimeout().getSize(), + config.getTimeout().getUnit())) val checkpointResponder = new ActorGatewayCheckpointResponder(jobManagerGateway); @@ -1420,7 +1427,8 @@ class TaskManager( def triggerTaskManagerRegistration(): Unit = { if(jobManagerAkkaURL.isDefined) { // begin attempts to reconnect - val deadline: Option[Deadline] = config.maxRegistrationDuration.map(_.fromNow) + val deadline: Option[Deadline] = Option(config.getMaxRegistrationDuration()) + .map{ duration => new FiniteDuration(duration.getSize(), duration.getUnit()).fromNow } // start a new registration run currentRegistrationRun = UUID.randomUUID() @@ -1430,7 +1438,9 @@ class TaskManager( self ! decorateMessage( TriggerTaskManagerRegistration( jobManagerAkkaURL.get, - config.initialRegistrationPause, + new FiniteDuration( + config.getInitialRegistrationPause().getSize(), + config.getInitialRegistrationPause().getUnit()), deadline, 1, currentRegistrationRun) @@ -1837,32 +1847,37 @@ object TaskManager { taskManagerClass: Class[_ <: TaskManager]) : ActorRef = { - val (taskManagerConfig, - connectionInfo, - memoryManager, - ioManager, - network, - leaderRetrievalService, - metricsRegistry) = createTaskManagerComponents( - configuration, - resourceID, - taskManagerHostname, - localTaskManagerCommunication, - leaderRetrievalServiceOption) + val taskManagerAddress = InetAddress.getByName(taskManagerHostname) + + val taskManagerServicesConfiguration = TaskManagerServicesConfiguration + .fromConfiguration(configuration, taskManagerAddress, false) + + val taskManagerConfiguration = TaskManagerConfiguration.fromConfiguration(configuration) + + val taskManagerServices = TaskManagerServices.fromConfiguration( + taskManagerServicesConfiguration, + resourceID) + + val metricRegistry = taskManagerServices.getMetricRegistry() + + val leaderRetrievalService = leaderRetrievalServiceOption match { + case Some(lrs) => lrs + case None => LeaderRetrievalUtils.createLeaderRetrievalService(configuration) + } // create the actor properties (which define the actor constructor parameters) val tmProps = getTaskManagerProps( taskManagerClass, - taskManagerConfig, + taskManagerConfiguration, resourceID, - connectionInfo, - memoryManager, - ioManager, - network, + taskManagerServices.getTaskManagerLocation(), + taskManagerServices.getMemoryManager(), + taskManagerServices.getIOManager(), + taskManagerServices.getNetworkEnvironment(), leaderRetrievalService, - metricsRegistry) + metricRegistry) - metricsRegistry.startQueryService(actorSystem) + metricRegistry.startQueryService(actorSystem) taskManagerActorName match { case Some(actorName) => actorSystem.actorOf(tmProps, actorName) @@ -1889,211 +1904,11 @@ object TaskManager { memoryManager, ioManager, networkEnvironment, - taskManagerConfig.numberOfSlots, + taskManagerConfig.getNumberSlots(), leaderRetrievalService, metricsRegistry) } - def createTaskManagerComponents( - configuration: Configuration, - resourceID: ResourceID, - taskManagerHostname: String, - localTaskManagerCommunication: Boolean, - leaderRetrievalServiceOption: Option[LeaderRetrievalService]): - (TaskManagerConfiguration, - TaskManagerLocation, - MemoryManager, - IOManager, - NetworkEnvironment, - LeaderRetrievalService, - FlinkMetricRegistry) = { - - val (taskManagerConfig : TaskManagerConfiguration, - netConfig: NetworkEnvironmentConfiguration, - taskManagerAddress: InetSocketAddress, - memType: MemoryType - ) = parseTaskManagerConfiguration( - configuration, - taskManagerHostname, - localTaskManagerCommunication) - - // pre-start checks - checkTempDirs(taskManagerConfig.tmpDirPaths) - - val networkBufferPool = new NetworkBufferPool( - netConfig.numNetworkBuffers, - netConfig.networkBufferSize, - netConfig.memoryType) - - val connectionManager = Option(netConfig.nettyConfig) match { - case Some(nettyConfig) => new NettyConnectionManager(nettyConfig) - case None => new LocalConnectionManager() - } - - val resultPartitionManager = new ResultPartitionManager() - val taskEventDispatcher = new TaskEventDispatcher() - - val kvStateRegistry = new KvStateRegistry() - - val kvStateServer = Option(netConfig.nettyConfig) match { - case Some(nettyConfig) => - - val numNetworkThreads = if (netConfig.queryServerNetworkThreads == 0) { - nettyConfig.getNumberOfSlots - } else { - netConfig.queryServerNetworkThreads - } - - val numQueryThreads = if (netConfig.queryServerQueryThreads == 0) { - nettyConfig.getNumberOfSlots - } else { - netConfig.queryServerQueryThreads - } - - new KvStateServer( - taskManagerAddress.getAddress(), - netConfig.queryServerPort, - numNetworkThreads, - numQueryThreads, - kvStateRegistry, - new DisabledKvStateRequestStats()) - - case None => null - } - - // we start the network first, to make sure it can allocate its buffers first - val network = new NetworkEnvironment( - networkBufferPool, - connectionManager, - resultPartitionManager, - taskEventDispatcher, - kvStateRegistry, - kvStateServer, - netConfig.ioMode, - netConfig.partitionRequestInitialBackoff, - netConfig.partitinRequestMaxBackoff) - - network.start() - - val taskManagerLocation = new TaskManagerLocation( - resourceID, - taskManagerAddress.getAddress(), - network.getConnectionManager().getDataPort()) - - // computing the amount of memory to use depends on how much memory is available - // it strictly needs to happen AFTER the network stack has been initialized - - // check if a value has been configured - val configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L) - checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory, - ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, - "MemoryManager needs at least one MB of memory. " + - "If you leave this config parameter empty, the system automatically " + - "pick a fraction of the available memory.") - - - val preAllocateMemory = configuration.getBoolean( - ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY, - ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE) - - val memorySize = if (configuredMemory > 0) { - if (preAllocateMemory) { - LOG.info(s"Using $configuredMemory MB for managed memory.") - } else { - LOG.info(s"Limiting managed memory to $configuredMemory MB, " + - s"memory will be allocated lazily.") - } - configuredMemory << 20 // megabytes to bytes - } - else { - val fraction = configuration.getFloat( - ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY, - ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION) - checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction, - ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY, - "MemoryManager fraction of the free memory must be between 0.0 and 1.0") - - if (memType == MemoryType.HEAP) { - val relativeMemSize = (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * - fraction).toLong - - if (preAllocateMemory) { - LOG.info(s"Using $fraction of the currently free heap space for managed " + - s"heap memory (${relativeMemSize >> 20} MB).") - } else { - LOG.info(s"Limiting managed memory to $fraction of the currently free heap space " + - s"(${relativeMemSize >> 20} MB), memory will be allocated lazily.") - } - - relativeMemSize - } - else if (memType == MemoryType.OFF_HEAP) { - - // The maximum heap memory has been adjusted according to the fraction - val maxMemory = EnvironmentInformation.getMaxJvmHeapMemory() - val directMemorySize = (maxMemory / (1.0 - fraction) * fraction).toLong - - if (preAllocateMemory) { - LOG.info(s"Using $fraction of the maximum memory size for " + - s"managed off-heap memory (${directMemorySize >> 20} MB).") - } else { - LOG.info(s"Limiting managed memory to $fraction of the maximum memory size " + - s"(${directMemorySize >> 20} MB), memory will be allocated lazily.") - } - - directMemorySize - } - else { - throw new RuntimeException("No supported memory type detected.") - } - } - - // now start the memory manager - val memoryManager = try { - new MemoryManager( - memorySize, - taskManagerConfig.numberOfSlots, - netConfig.networkBufferSize, - memType, - preAllocateMemory) - } - catch { - case e: OutOfMemoryError => - memType match { - case MemoryType.HEAP => - throw new Exception(s"OutOfMemory error (${e.getMessage()})" + - s" while allocating the TaskManager heap memory ($memorySize bytes).", e) - - case MemoryType.OFF_HEAP => - throw new Exception(s"OutOfMemory error (${e.getMessage()})" + - s" while allocating the TaskManager off-heap memory ($memorySize bytes). " + - s"Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e) - - case _ => throw e - } - } - - // start the I/O manager last, it will create some temp directories. - val ioManager: IOManager = new IOManagerAsync(taskManagerConfig.tmpDirPaths) - - val leaderRetrievalService = leaderRetrievalServiceOption match { - case Some(lrs) => lrs - case None => LeaderRetrievalUtils.createLeaderRetrievalService(configuration) - } - - val metricsRegistry = new FlinkMetricRegistry( - MetricRegistryConfiguration.fromConfiguration(configuration)) - - (taskManagerConfig, - taskManagerLocation, - memoryManager, - ioManager, - network, - leaderRetrievalService, - metricsRegistry) - } - - // -------------------------------------------------------------------------- // Resolving the TaskManager actor // -------------------------------------------------------------------------- @@ -2132,239 +1947,6 @@ object TaskManager { // Parsing and checking the TaskManager Configuration // -------------------------------------------------------------------------- - /** - * Utility method to extract TaskManager config parameters from the configuration and to - * sanity check them. - * - * @param configuration The configuration. - * @param taskManagerHostname The host name under which the TaskManager communicates. - * @param localTaskManagerCommunication True, to skip initializing the network stack. - * Use only in cases where only one task manager runs. - * @return A tuple (TaskManagerConfiguration, network configuration, inet socket address, - * memory tyep). - */ - @throws(classOf[IllegalArgumentException]) - def parseTaskManagerConfiguration( - configuration: Configuration, - taskManagerHostname: String, - localTaskManagerCommunication: Boolean) - : (TaskManagerConfiguration, - NetworkEnvironmentConfiguration, - InetSocketAddress, - MemoryType) = { - - // ------- read values from the config and check them --------- - // (a lot of them) - - // ----> hosts / ports for communication and data exchange - - val dataport = configuration.getInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, - ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT) - - checkConfigParameter(dataport >= 0, dataport, ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, - "Leave config parameter empty or use 0 to let the system choose a port automatically.") - - val taskManagerAddress = InetAddress.getByName(taskManagerHostname) - val taskManagerInetSocketAddress = new InetSocketAddress(taskManagerAddress, dataport) - - // ----> memory / network stack (shuffles/broadcasts), task slots, temp directories - - // we need this because many configs have been written with a "-1" entry - val slots = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1) match { - case -1 => 1 - case x => x - } - - checkConfigParameter(slots >= 1, slots, ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, - "Number of task slots must be at least one.") - - val numNetworkBuffers = configuration.getInteger( - ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, - ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS) - - checkConfigParameter(numNetworkBuffers > 0, numNetworkBuffers, - ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY) - - val pageSize: Int = configuration.getInteger( - ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY, - ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE) - - // check page size of for minimum size - checkConfigParameter(pageSize >= MemoryManager.MIN_PAGE_SIZE, pageSize, - ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY, - "Minimum memory segment size is " + MemoryManager.MIN_PAGE_SIZE) - - // check page size for power of two - checkConfigParameter(MathUtils.isPowerOf2(pageSize), pageSize, - ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY, - "Memory segment size must be a power of 2.") - - // check whether we use heap or off-heap memory - val memType: MemoryType = - if (configuration.getBoolean(ConfigConstants.TASK_MANAGER_MEMORY_OFF_HEAP_KEY, false)) { - MemoryType.OFF_HEAP - } else { - MemoryType.HEAP - } - - // initialize the memory segment factory accordingly - memType match { - case MemoryType.HEAP => - if (!MemorySegmentFactory.initializeIfNotInitialized(HeapMemorySegment.FACTORY)) { - throw new Exception("Memory type is set to heap memory, but memory segment " + - "factory has been initialized for off-heap memory segments") - } - - case MemoryType.OFF_HEAP => - if (!MemorySegmentFactory.initializeIfNotInitialized(HybridMemorySegment.FACTORY)) { - throw new Exception("Memory type is set to off-heap memory, but memory segment " + - "factory has been initialized for heap memory segments") - } - } - - val tmpDirs = configuration.getString( - ConfigConstants.TASK_MANAGER_TMP_DIR_KEY, - ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH) - .split(",|" + File.pathSeparator) - - val nettyConfig = if (localTaskManagerCommunication) { - None - } else { - Some( - new NettyConfig( - taskManagerInetSocketAddress.getAddress(), - taskManagerInetSocketAddress.getPort(), - pageSize, - slots, - configuration) - ) - } - - // Default spill I/O mode for intermediate results - val syncOrAsync = configuration.getString( - ConfigConstants.TASK_MANAGER_NETWORK_DEFAULT_IO_MODE, - ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_DEFAULT_IO_MODE) - - val ioMode : IOMode = if (syncOrAsync == "async") IOMode.ASYNC else IOMode.SYNC - - val queryServerPort = configuration.getInteger( - ConfigConstants.QUERYABLE_STATE_SERVER_PORT, - ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_PORT) - - val queryServerNetworkThreads = configuration.getInteger( - ConfigConstants.QUERYABLE_STATE_SERVER_NETWORK_THREADS, - ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_NETWORK_THREADS) - - val queryServerQueryThreads = configuration.getInteger( - ConfigConstants.QUERYABLE_STATE_SERVER_QUERY_THREADS, - ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_QUERY_THREADS) - - val networkConfig = NetworkEnvironmentConfiguration( - numNetworkBuffers, - pageSize, - memType, - ioMode, - queryServerPort, - queryServerNetworkThreads, - queryServerQueryThreads, - nettyConfig.getOrElse(null)) - - // ----> timeouts, library caching, profiling - - val timeout = try { - AkkaUtils.getTimeout(configuration) - } catch { - case e: Exception => throw new IllegalArgumentException( - s"Invalid format for '${ConfigConstants.AKKA_ASK_TIMEOUT}'. " + - s"Use formats like '50 s' or '1 min' to specify the timeout.") - } - LOG.info("Messages between TaskManager and JobManager have a max timeout of " + timeout) - - val cleanupInterval = configuration.getLong( - ConfigConstants.LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL, - ConfigConstants.DEFAULT_LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL) * 1000 - - val finiteRegistrationDuration = try { - val maxRegistrationDuration = Duration(configuration.getString( - ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION, - ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_DURATION)) - - if (maxRegistrationDuration.isFinite()) { - Some(maxRegistrationDuration.asInstanceOf[FiniteDuration]) - } else { - None - } - } catch { - case e: NumberFormatException => throw new IllegalArgumentException( - "Invalid format for parameter " + ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION, - e) - } - - val initialRegistrationPause = try { - val pause = Duration(configuration.getString( - ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, - ConfigConstants.DEFAULT_TASK_MANAGER_INITIAL_REGISTRATION_PAUSE - )) - - if (pause.isFinite()) { - pause.asInstanceOf[FiniteDuration] - } else { - throw new IllegalArgumentException(s"The initial registration pause must be finite: $pause") - } - } catch { - case e: NumberFormatException => throw new IllegalArgumentException( - "Invalid format for parameter " + ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, - e) - } - - val maxRegistrationPause = try { - val pause = Duration(configuration.getString( - ConfigConstants.TASK_MANAGER_MAX_REGISTARTION_PAUSE, - ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_PAUSE - )) - - if (pause.isFinite()) { - pause.asInstanceOf[FiniteDuration] - } else { - throw new IllegalArgumentException(s"The maximum registration pause must be finite: $pause") - } - } catch { - case e: NumberFormatException => throw new IllegalArgumentException( - "Invalid format for parameter " + ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, - e) - } - - val refusedRegistrationPause = try { - val pause = Duration(configuration.getString( - ConfigConstants.TASK_MANAGER_REFUSED_REGISTRATION_PAUSE, - ConfigConstants.DEFAULT_TASK_MANAGER_REFUSED_REGISTRATION_PAUSE - )) - - if (pause.isFinite()) { - pause.asInstanceOf[FiniteDuration] - } else { - throw new IllegalArgumentException(s"The refused registration pause must be finite: $pause") - } - } catch { - case e: NumberFormatException => throw new IllegalArgumentException( - "Invalid format for parameter " + ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, - e) - } - - val taskManagerConfig = TaskManagerConfiguration( - tmpDirs, - cleanupInterval, - timeout, - finiteRegistrationDuration, - slots, - configuration, - initialRegistrationPause, - maxRegistrationPause, - refusedRegistrationPause) - - (taskManagerConfig, networkConfig, taskManagerInetSocketAddress, memType) - } - /** * Gets the hostname and port of the JobManager from the configuration. Also checks that * the hostname is not null and the port non-negative. @@ -2398,71 +1980,6 @@ object TaskManager { // Miscellaneous Utilities // -------------------------------------------------------------------------- - /** - * Validates a condition for a config parameter and displays a standard exception, if the - * the condition does not hold. - * - * @param condition The condition that must hold. If the condition is false, an - * exception is thrown. - * @param parameter The parameter value. Will be shown in the exception message. - * @param name The name of the config parameter. Will be shown in the exception message. - * @param errorMessage The optional custom error message to append to the exception message. - * @throws IllegalConfigurationException Thrown if the condition is violated. - */ - @throws(classOf[IllegalConfigurationException]) - private def checkConfigParameter( - condition: Boolean, - parameter: Any, - name: String, - errorMessage: String = "") - : Unit = { - if (!condition) { - throw new IllegalConfigurationException( - s"Invalid configuration value for '$name' : $parameter - $errorMessage") - } - } - - /** - * Validates that all the directories denoted by the strings do actually exist, are proper - * directories (not files), and are writable. - * - * @param tmpDirs The array of directory paths to check. - * @throws Exception Thrown if any of the directories does not exist or is not writable - * or is a file, rather than a directory. - */ - @throws(classOf[IOException]) - private def checkTempDirs(tmpDirs: Array[String]): Unit = { - tmpDirs.zipWithIndex.foreach { - case (dir: String, _) => - val file = new File(dir) - - if (!file.exists) { - throw new IOException( - s"Temporary file directory ${file.getAbsolutePath} does not exist.") - } - if (!file.isDirectory) { - throw new IOException( - s"Temporary file directory ${file.getAbsolutePath} is not a directory.") - } - if (!file.canWrite) { - throw new IOException( - s"Temporary file directory ${file.getAbsolutePath} is not writable.") - } - - if (LOG.isInfoEnabled) { - val totalSpaceGb = file.getTotalSpace >> 30 - val usableSpaceGb = file.getUsableSpace >> 30 - val usablePercentage = usableSpaceGb.asInstanceOf[Double] / totalSpaceGb * 100 - - val path = file.getAbsolutePath - - LOG.info(f"Temporary file directory '$path': total $totalSpaceGb GB, " + - f"usable $usableSpaceGb GB ($usablePercentage%.2f%% usable)") - } - case (_, id) => throw new IllegalArgumentException(s"Temporary file directory #$id is null.") - } - } - /** * Creates the registry of default metrics, including stats about garbage collection, memory * usage, and system CPU load. diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManagerConfiguration.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManagerConfiguration.scala deleted file mode 100644 index aab3c5f3eba2f..0000000000000 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManagerConfiguration.scala +++ /dev/null @@ -1,56 +0,0 @@ -/* - * 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.taskmanager - -import java.util.concurrent.TimeUnit - -import org.apache.flink.configuration.Configuration - -import scala.concurrent.duration.FiniteDuration - -case class TaskManagerConfiguration( - tmpDirPaths: Array[String], - cleanupInterval: Long, - timeout: FiniteDuration, - maxRegistrationDuration: Option[FiniteDuration], - numberOfSlots: Int, - configuration: Configuration, - initialRegistrationPause: FiniteDuration, - maxRegistrationPause: FiniteDuration, - refusedRegistrationPause: FiniteDuration) { - - def this( - tmpDirPaths: Array[String], - cleanupInterval: Long, - timeout: FiniteDuration, - maxRegistrationDuration: Option[FiniteDuration], - numberOfSlots: Int, - configuration: Configuration) { - this ( - tmpDirPaths, - cleanupInterval, - timeout, - maxRegistrationDuration, - numberOfSlots, - configuration, - FiniteDuration(500, TimeUnit.MILLISECONDS), - FiniteDuration(30, TimeUnit.SECONDS), - FiniteDuration(10, TimeUnit.SECONDS)) - } -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java index 627a25a653ada..500d1bdc6670b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java @@ -26,6 +26,7 @@ import akka.actor.Props; import akka.testkit.JavaTestKit; +import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.memory.MemoryType; @@ -49,11 +50,11 @@ import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; import org.apache.flink.runtime.query.KvStateRegistry; +import org.apache.flink.runtime.taskexecutor.TaskManagerConfiguration; import org.apache.flink.runtime.util.LeaderRetrievalUtils; import org.junit.Test; -import scala.Option; import scala.concurrent.duration.FiniteDuration; import java.net.InetAddress; @@ -69,7 +70,7 @@ public class TaskManagerComponentsStartupShutdownTest { public void testComponentsStartupShutdown() { final String[] TMP_DIR = new String[] { ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH }; - final FiniteDuration timeout = new FiniteDuration(100, TimeUnit.SECONDS); + final Time timeout = Time.seconds(100); final int BUFFER_SIZE = 32 * 1024; Configuration config = new Configuration(); @@ -93,14 +94,19 @@ public void testComponentsStartupShutdown() { LeaderRetrievalUtils.createLeaderRetrievalService(config, jobManager), StandaloneResourceManager.class); + final int numberOfSlots = 1; + // create the components for the TaskManager manually final TaskManagerConfiguration tmConfig = new TaskManagerConfiguration( - TMP_DIR, - 1000000, - timeout, - Option.empty(), - 1, - config); + numberOfSlots, + TMP_DIR, + timeout, + null, + Time.milliseconds(500), + Time.seconds(30), + Time.seconds(10), + 1000000, // cleanup interval + config); final NetworkEnvironmentConfiguration netConf = new NetworkEnvironmentConfiguration( 32, BUFFER_SIZE, MemoryType.HEAP, IOManager.IOMode.SYNC, 0, 0, 0, @@ -125,8 +131,6 @@ public void testComponentsStartupShutdown() { network.start(); - final int numberOfSlots = 1; - LeaderRetrievalService leaderRetrievalService = new StandaloneLeaderRetrievalService(jobManager.path().toString()); MetricRegistryConfiguration metricRegistryConfiguration = MetricRegistryConfiguration.fromConfiguration(config); diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala index 707401b14855b..09dc5ed24f271 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala @@ -24,7 +24,8 @@ import org.apache.flink.runtime.io.network.NetworkEnvironment import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService import org.apache.flink.runtime.memory.MemoryManager import org.apache.flink.runtime.metrics.MetricRegistry -import org.apache.flink.runtime.taskmanager.{TaskManagerLocation, TaskManager, TaskManagerConfiguration} +import org.apache.flink.runtime.taskexecutor.TaskManagerConfiguration +import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerLocation} import scala.language.postfixOps diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala index 5628f3c4fd0b6..e878097c08390 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala @@ -243,7 +243,6 @@ object TestingUtils { ) } - def createTaskManager( actorSystem: ActorSystem, jobManagerURL: String, diff --git a/flink-yarn-tests/src/test/scala/org/apache/flink/yarn/TestingYarnTaskManager.scala b/flink-yarn-tests/src/test/scala/org/apache/flink/yarn/TestingYarnTaskManager.scala index 1010432a7856e..0f82faa40dc99 100644 --- a/flink-yarn-tests/src/test/scala/org/apache/flink/yarn/TestingYarnTaskManager.scala +++ b/flink-yarn-tests/src/test/scala/org/apache/flink/yarn/TestingYarnTaskManager.scala @@ -24,7 +24,8 @@ import org.apache.flink.runtime.io.network.NetworkEnvironment import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService import org.apache.flink.runtime.memory.MemoryManager import org.apache.flink.runtime.metrics.MetricRegistry -import org.apache.flink.runtime.taskmanager.{TaskManagerConfiguration, TaskManagerLocation} +import org.apache.flink.runtime.taskexecutor.TaskManagerConfiguration +import org.apache.flink.runtime.taskmanager.TaskManagerLocation import org.apache.flink.runtime.testingUtils.TestingTaskManagerLike /** [[YarnTaskManager]] implementation which mixes in the [[TestingTaskManagerLike]] mixin. diff --git a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala index 2ab9b20f089d8..be310854855ad 100644 --- a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala +++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala @@ -23,8 +23,9 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManager import org.apache.flink.runtime.io.network.NetworkEnvironment import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService import org.apache.flink.runtime.memory.MemoryManager -import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerConfiguration, TaskManagerLocation} +import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerLocation} import org.apache.flink.runtime.metrics.MetricRegistry +import org.apache.flink.runtime.taskexecutor.TaskManagerConfiguration /** An extension of the TaskManager that listens for additional YARN related * messages. From 7d82de93297675a7f1cad9901fcd019598e1282b Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Wed, 21 Sep 2016 20:20:25 +0200 Subject: [PATCH 48/50] Refactor JobClientActor to adapt to new Rpc framework and new cluster managerment --- .../runtime/jobClient/JobClientUtils.java | 278 ++++++++++++++++++ .../runtime/jobClient/JobInfoTracker.java | 257 ++++++++++++++++ .../jobClient/JobInfoTrackerGateway.java | 58 ++++ .../flink/runtime/jobmaster/JobMaster.java | 27 +- .../runtime/jobmaster/JobMasterGateway.java | 20 ++ .../apache/flink/runtime/rpc/RpcService.java | 8 + .../runtime/rpc/akka/AkkaRpcService.java | 20 ++ .../flink/runtime/rpc/TestingRpcService.java | 21 ++ .../runtime/rpc/TestingSerialRpcService.java | 7 +- 9 files changed, 694 insertions(+), 2 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobClient/JobClientUtils.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobClient/JobInfoTracker.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobClient/JobInfoTrackerGateway.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobClient/JobClientUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobClient/JobClientUtils.java new file mode 100644 index 0000000000000..9a0ff4ccf1252 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobClient/JobClientUtils.java @@ -0,0 +1,278 @@ +/* + * 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.jobClient; + +import akka.actor.ActorSystem; +import akka.actor.Address; +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.akka.ListeningBehaviour; +import org.apache.flink.runtime.blob.BlobCache; +import org.apache.flink.runtime.blob.BlobKey; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.runtime.client.JobRetrievalException; +import org.apache.flink.runtime.concurrent.Future; +import org.apache.flink.runtime.jobmaster.JobMasterGateway; +import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; +import org.apache.flink.runtime.messages.JobManagerMessages; +import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.rpc.akka.AkkaRpcService; +import org.apache.flink.util.NetUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.Option; +import scala.Some; +import scala.Tuple2; + +import java.io.IOException; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.net.URL; +import java.net.URLClassLoader; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + +/** + * JobClientUtils is a utility for client. + * It offers the following methods: + *

      + *
    • {@link #startJobClientRpcService(Configuration)} Starts a rpc service for client
    • + *
    • {@link #retrieveRunningJobResult(JobID, JobMasterGateway, RpcService, LeaderRetrievalService, boolean, Time, Configuration)} + * Attaches to a running Job using the JobID, and wait for its job result
    • + *
    • {@link #awaitJobResult(JobInfoTracker, ClassLoader, Time)} Awaits the result of the job execution which jobInfoTracker listen for
    • + *
    • {@link #retrieveClassLoader(JobID, JobMasterGateway, Configuration)} Reconstructs the class loader by first requesting information about it at the JobMaster + * and then downloading missing jar files
    • + *
    + */ +public class JobClientUtils { + + private static final Logger LOG = LoggerFactory.getLogger(JobClientUtils.class); + + + /** + * Starts a rpc service for client + * + * @param config the flink configuration + * @return + * @throws IOException + */ + public static RpcService startJobClientRpcService(Configuration config) + throws IOException + { + LOG.info("Starting JobClientUtils rpc service"); + Option> remoting = new Some<>(new Tuple2("", 0)); + + // start a remote actor system to listen on an arbitrary port + ActorSystem system = AkkaUtils.createActorSystem(config, remoting); + Address address = system.provider().getDefaultAddress(); + + String hostAddress = address.host().isDefined() ? + NetUtils.ipAddressToUrlString(InetAddress.getByName(address.host().get())) : + "(unknown)"; + int port = address.port().isDefined() ? ((Integer) address.port().get()) : -1; + LOG.info("Started JobClientUtils actor system at " + hostAddress + ':' + port); + return new AkkaRpcService(system, Time.milliseconds(AkkaUtils.getClientTimeout(config).toMillis())); + } + + /** + * Attaches to a running Job using the JobID, and wait for its job result + * + * @param jobID id of job + * @param jobMasterGateway gateway to the JobMaster + * @param rpcService + * @param leaderRetrievalService leader retriever service of jobMaster + * @param sysoutLogUpdates whether status messages shall be printed to sysout + * @param timeout register timeout + * @param configuration the flink configuration + * @return + * @throws JobExecutionException + */ + public static JobExecutionResult retrieveRunningJobResult( + JobID jobID, + JobMasterGateway jobMasterGateway, + RpcService rpcService, + LeaderRetrievalService leaderRetrievalService, + boolean sysoutLogUpdates, + Time timeout, + Configuration configuration) throws JobExecutionException + { + + checkNotNull(jobID, "The jobID must not be null."); + checkNotNull(jobMasterGateway, "The jobMasterGateway must not be null."); + checkNotNull(rpcService, "The rpcService must not be null."); + checkNotNull(leaderRetrievalService, "The leaderRetrievalService must not be null."); + checkNotNull(timeout, "The timeout must not be null"); + checkNotNull(configuration, "The configuration must not be null"); + + JobInfoTracker jobInfoTracker = null; + try { + jobInfoTracker = new JobInfoTracker(rpcService, leaderRetrievalService, jobID, sysoutLogUpdates); + jobInfoTracker.start(); + registerClientAtJobMaster(jobID, jobInfoTracker.getAddress(), jobMasterGateway, timeout); + ClassLoader classLoader = retrieveClassLoader(jobID, jobMasterGateway, configuration); + return awaitJobResult(jobInfoTracker, classLoader, timeout); + } finally { + if (jobInfoTracker != null) { + jobInfoTracker.shutDown(); + } + } + } + + /** + * Awaits the result of the job execution which jobInfoTracker listen for + * + * @param jobInfoTracker job info tracker + * @param classLoader classloader to parse the job result + * @return + * @throws JobExecutionException + */ + public static JobExecutionResult awaitJobResult(JobInfoTracker jobInfoTracker, + ClassLoader classLoader, Time timeout) throws JobExecutionException + { + try { + long timeoutInMillis = timeout.toMilliseconds(); + JobID jobID = jobInfoTracker.getJobID(); + Future jobExecutionResultFuture = jobInfoTracker.getJobExecutionResult(classLoader); + while (true) { + try { + JobExecutionResult jobExecutionResult = jobExecutionResultFuture.get(timeoutInMillis, TimeUnit.MILLISECONDS); + return jobExecutionResult; + } catch (InterruptedException e) { + throw new JobExecutionException( + jobID, + "Interrupted while waiting for job completion."); + } catch (TimeoutException e) { + try { + // retry when timeout exception happened util jobInfoTracker is dead + Future isAliveFuture = jobInfoTracker.getRpcService().isReachable(jobInfoTracker.getAddress()); + boolean isAlive = isAliveFuture.get(timeoutInMillis, TimeUnit.MILLISECONDS); + checkState(isAlive, "JobInfoTracker has been dead!"); + } catch (Exception eInner) { + if (!jobExecutionResultFuture.isDone()) { + throw new JobExecutionException( + jobID, + "JobInfoTracker seems to have died before the JobExecutionResult could be retrieved.", + eInner); + } + } + } catch (ExecutionException e) { + if (e.getCause() instanceof JobExecutionException) { + throw (JobExecutionException) e.getCause(); + } else { + throw new JobExecutionException(jobID, + "Couldn't retrieve the JobExecutionResult from the JobMaster.", e.getCause()); + } + + } + } + } finally { + jobInfoTracker.shutDown(); + } + } + + /** + * Reconstructs the class loader by first requesting information about it at the JobMaster + * and then downloading missing jar files. + * + * @param jobID id of job + * @param jobMasterGateway gateway to the JobMaster + * @param config the flink configuration + * @return A classloader that should behave like the original classloader + * @throws JobRetrievalException if anything goes wrong + */ + public static ClassLoader retrieveClassLoader( + JobID jobID, + JobMasterGateway jobMasterGateway, + Configuration config) + throws JobRetrievalException + { + + final JobManagerMessages.ClassloadingProps props; + try { + long defaultTimeoutInMillis = AkkaUtils.getDefaultTimeout().toMillis(); + props = jobMasterGateway.requestClassloadingProps(Time.milliseconds(defaultTimeoutInMillis)).get(defaultTimeoutInMillis, TimeUnit.MILLISECONDS); + } catch (Exception e) { + throw new JobRetrievalException(jobID, "Couldn't retrieve class loading properties from JobMaster.", e); + } + String jmHostname = null; + try { + InetSocketAddress jobMasterInetSocketAddress = AkkaUtils.getInetSockeAddressFromAkkaURL(jobMasterGateway.getAddress()); + jmHostname = jobMasterInetSocketAddress.getHostName(); + } catch (Exception e) { + throw new RuntimeException("Failed to retrieve JobMaster address", e); + } + InetSocketAddress serverAddress = new InetSocketAddress(jmHostname, props.blobManagerPort()); + final BlobCache blobClient = new BlobCache(serverAddress, config); + + final List requiredJarFiles = props.requiredJarFiles(); + final List requiredClasspaths = props.requiredClasspaths(); + + final URL[] allURLs = new URL[requiredJarFiles.size() + requiredClasspaths.size()]; + + int pos = 0; + for (BlobKey blobKey : props.requiredJarFiles()) { + try { + allURLs[pos++] = blobClient.getURL(blobKey); + } catch (Exception e) { + blobClient.shutdown(); + throw new JobRetrievalException(jobID, "Failed to download BlobKey " + blobKey); + } + } + + for (URL url : requiredClasspaths) { + allURLs[pos++] = url; + } + + return new URLClassLoader(allURLs, org.apache.flink.runtime.client.JobClient.class.getClassLoader()); + } + + /** + * Registers client at job master + * + * @param jobID id of job + * @param clientAddress client address + * @param jobMasterGateway gateway to the JobMaster + * @param timeout register timeout + * @throws JobExecutionException + */ + private static JobManagerMessages.RegisterJobClientSuccess registerClientAtJobMaster(JobID jobID, + String clientAddress, JobMasterGateway jobMasterGateway, + Time timeout) throws JobExecutionException + { + try { + Future registerJobClientSuccessFuture = + jobMasterGateway.registerJobInfoTracker(clientAddress, ListeningBehaviour.EXECUTION_RESULT_AND_STATE_CHANGES, timeout); + JobManagerMessages.RegisterJobClientSuccess registerJobClientSuccess = registerJobClientSuccessFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + return registerJobClientSuccess; + } catch (Throwable e) { + throw new JobExecutionException(jobID, "Registration for Job at the JobMaster " + + "timed out. " + "You may increase '" + ConfigConstants.AKKA_CLIENT_TIMEOUT + + "' in case the JobMaster needs more time to confirm the job client registration.", e); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobClient/JobInfoTracker.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobClient/JobInfoTracker.java new file mode 100644 index 0000000000000..a29cd696934a8 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobClient/JobInfoTracker.java @@ -0,0 +1,257 @@ +/* + * 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.jobClient; + +import akka.dispatch.ExecutionContexts$; +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.client.SerializedJobExecutionResult; +import org.apache.flink.runtime.concurrent.Future; +import org.apache.flink.runtime.concurrent.impl.FlinkFuture; +import org.apache.flink.runtime.jobgraph.JobStatus; +import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener; +import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; +import org.apache.flink.runtime.messages.ExecutionGraphMessages; +import org.apache.flink.runtime.messages.JobManagerMessages; +import org.apache.flink.runtime.rpc.RpcEndpoint; +import org.apache.flink.runtime.rpc.RpcMethod; +import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.util.SerializedThrowable; +import org.apache.flink.runtime.client.JobExecutionException; +import scala.concurrent.Promise; +import akka.dispatch.Mapper; + +import java.util.UUID; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * This base receives and prints job updates until job completion, receive job result when job completes. + */ +public class JobInfoTracker extends RpcEndpoint { + + private final LeaderRetrievalService leaderRetrievalService; + + /** identifier of a job */ + private final JobID jobID; + + /** true if status messages shall be printed to sysout */ + private final boolean sysoutUpdates; + + /** Promise about job result which will be completed when receive job result from job master */ + private final Promise jobResultMessagePromise; + + /** the leader session id of job master which is responsible for the given job */ + private UUID jobMasterLeaderSessionID = null; + + /** + * Constructs a new jobInfoTracker + * + * @param rpcService rpc service + * @param leaderRetrievalService leader retrieval service of jobMaster + * @param jobID identifier of job + * @param sysoutUpdates whether status messages shall be printed to sysout + */ + public JobInfoTracker(RpcService rpcService, LeaderRetrievalService leaderRetrievalService, JobID jobID, + boolean sysoutUpdates) + { + super(rpcService); + this.leaderRetrievalService = checkNotNull(leaderRetrievalService); + this.jobID = checkNotNull(jobID); + this.sysoutUpdates = sysoutUpdates; + this.jobResultMessagePromise = new scala.concurrent.impl.Promise.DefaultPromise<>(); + } + + @Override + public void start() { + super.start(); + try { + leaderRetrievalService.start(new JobMasterLeaderListener()); + } catch (Throwable e) { + log.error("A fatal error happened when start a new jobMasterListener at leaderRetriever of job {}", jobID); + throw new RuntimeException("A fatal error happened when get jobMaster leaderRetriever ", e); + } + } + + @Override + public void shutDown() { + super.shutDown(); + if (!jobResultMessagePromise.isCompleted()) { + jobResultMessagePromise.tryFailure(new RuntimeException("JobInfoTracker service stopped before receive the job result")); + } + try { + leaderRetrievalService.stop(); + } catch (Throwable e) { + log.error("A fatal error happened when stop the jobMaster leaderRetriever service", e); + throw new RuntimeException("A fatal error happened when stop the jobMaster leaderRetriever service", e); + } + } + + /** + * Receives notification about execution state changed event from {@link org.apache.flink.runtime.jobmaster.JobMaster} + * + * @param jobMasterLeaderSessionID leaderSessionID of jobMaster which send this notification + * @param executionStateChanged the execution state change message + */ + @RpcMethod + public void notifyJobExecutionStateChanged(UUID jobMasterLeaderSessionID, + ExecutionGraphMessages.ExecutionStateChanged executionStateChanged) + { + if (jobMasterLeaderSessionID.equals(this.jobMasterLeaderSessionID)) { + logAndPrintMessage(executionStateChanged.toString()); + } + } + + /** + * Receives notification about job status changed event from {@link org.apache.flink.runtime.jobmaster.JobMaster} + * + * @param jobMasterLeaderSessionID leaderSessionID of jobMaster which send this notifcation + * @param jobStatusChanged the job state change message + */ + @RpcMethod + public void notifyJobStatusChanged(UUID jobMasterLeaderSessionID, + ExecutionGraphMessages.JobStatusChanged jobStatusChanged) + { + if (jobMasterLeaderSessionID.equals(this.jobMasterLeaderSessionID)) { + logAndPrintMessage(jobStatusChanged); + } + } + + /** + * Receives notification about job result from {@link org.apache.flink.runtime.jobmaster.JobMaster} + * + * @param jobMasterLeaderSessionID leaderSessionID of jobMaster which send this notifcation + * @param jobResultMessage job result + */ + @RpcMethod + public void notifyJobResult(UUID jobMasterLeaderSessionID, JobManagerMessages.JobResultMessage jobResultMessage) { + if (jobMasterLeaderSessionID.equals(this.jobMasterLeaderSessionID)) { + jobResultMessagePromise.success(jobResultMessage); + shutDown(); + } + } + + /** + * Gets jobID + * + * @return jobID + */ + public JobID getJobID() { + return this.jobID; + } + + /** + * Gets the final job execution result wrapped in Future + * + * @param classLoader the classloader to parse JobResultMessage + * @return job execution result wrapped in Future + */ + public Future getJobExecutionResult(final ClassLoader classLoader) { + scala.concurrent.Future jobResultMessageFuture = jobResultMessagePromise.future().map(new Mapper() { + + @Override + public JobExecutionResult checkedApply( + final JobManagerMessages.JobResultMessage jobResultMessage) throws JobExecutionException + { + if (jobResultMessage instanceof JobManagerMessages.JobResultSuccess) { + log.info("Job execution complete"); + SerializedJobExecutionResult result = ((JobManagerMessages.JobResultSuccess) jobResultMessage).result(); + if (result != null) { + try { + return result.toJobExecutionResult(classLoader); + } catch (Throwable t) { + throw new JobExecutionException(jobID, + "Job was successfully executed but JobExecutionResult could not be deserialized."); + } + } else { + throw new JobExecutionException(jobID, + "Job was successfully executed but result contained a null JobExecutionResult."); + } + } else if (jobResultMessage instanceof JobManagerMessages.JobResultFailure) { + log.info("Job execution failed"); + SerializedThrowable serThrowable = ((JobManagerMessages.JobResultFailure) jobResultMessage).cause(); + if (serThrowable != null) { + Throwable cause = serThrowable.deserializeError(classLoader); + if (cause instanceof JobExecutionException) { + throw (JobExecutionException) cause; + } else { + throw new JobExecutionException(jobID, "Job execution failed", cause); + } + } else { + throw new JobExecutionException(jobID, + "Job execution failed with null as failure cause."); + } + } else { + throw new JobExecutionException(jobID, + "Unknown answer from JobManager after submitting the job: " + jobResultMessage); + } + } + }, ExecutionContexts$.MODULE$.fromExecutor(getRpcService().getExecutor())); + + return new FlinkFuture<>(jobResultMessageFuture); + } + + private void handleFatalError(final Throwable e) { + runAsync(new Runnable() { + @Override + public void run() { + log.error("Error occurred.", e); + if (!jobResultMessagePromise.isCompleted()) { + jobResultMessagePromise.tryFailure(e); + } + shutDown(); + } + }); + } + + private void logAndPrintMessage(String message) { + log.info(message); + if (sysoutUpdates) { + System.out.println(message); + } + } + + private void logAndPrintMessage(ExecutionGraphMessages.JobStatusChanged message) { + // by default, this only prints the status, and not any exception. + // in state FAILING, we report the exception in addition + if (message.newJobStatus() != JobStatus.FAILING || message.error() == null) { + logAndPrintMessage(message.toString()); + } else { + log.info(message.toString(), message.error()); + if (sysoutUpdates) { + System.out.println(message.toString()); + message.error().printStackTrace(System.out); + } + } + } + + private class JobMasterLeaderListener implements LeaderRetrievalListener { + + @Override + public void notifyLeaderAddress(final String leaderAddress, final UUID leaderSessionID) { + jobMasterLeaderSessionID = leaderSessionID; + } + + @Override + public void handleError(final Exception exception) { + log.error("Error occurred in the LeaderRetrievalService.", exception); + handleFatalError(exception); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobClient/JobInfoTrackerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobClient/JobInfoTrackerGateway.java new file mode 100644 index 0000000000000..5e2430dc9c1fd --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobClient/JobInfoTrackerGateway.java @@ -0,0 +1,58 @@ +/* + * 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.jobClient; + +import org.apache.flink.runtime.messages.ExecutionGraphMessages; +import org.apache.flink.runtime.messages.JobManagerMessages; +import org.apache.flink.runtime.rpc.RpcGateway; + +import java.util.UUID; + +/** + * The {@link JobInfoTrackerGateway}'s RPC gateway interface, receive notification from {@link org.apache.flink.runtime.jobmaster.JobMaster}. + */ +public interface JobInfoTrackerGateway extends RpcGateway { + + /** + * Receives notification about execution state changed event from {@link org.apache.flink.runtime.jobmaster.JobMaster} + * + * @param jobMasterLeaderSessionID leaderSessionID of jobMaster which send this notifcation + * @param executionStateChanged the execution state change message + */ + void notifyJobExecutionStateChanged(UUID jobMasterLeaderSessionID, + ExecutionGraphMessages.ExecutionStateChanged executionStateChanged); + + /** + * Receives notification about job status changed event from {@link org.apache.flink.runtime.jobmaster.JobMaster} + * + * @param jobMasterLeaderSessionID leaderSessionID of jobMaster which send this notifcation + * @param jobStatusChanged the job state change message + */ + void notifyJobStatusChanged(UUID jobMasterLeaderSessionID, + ExecutionGraphMessages.JobStatusChanged jobStatusChanged); + + /** + * Receives notification about job result from {@link org.apache.flink.runtime.jobmaster.JobMaster} + * + * @param jobMasterLeaderSessionID leaderSessionID of jobMaster which send this notifcation + * @param jobResultMessage job result + */ + void notifyJobResult(UUID jobMasterLeaderSessionID, JobManagerMessages.JobResultMessage jobResultMessage); + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java index b52a23c925100..9c7260beb5969 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java @@ -39,6 +39,7 @@ import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; import org.apache.flink.runtime.executiongraph.restart.RestartStrategy; import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory; +import org.apache.flink.runtime.akka.ListeningBehaviour; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobVertex; @@ -49,9 +50,10 @@ import org.apache.flink.runtime.jobmanager.scheduler.Scheduler; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; +import org.apache.flink.runtime.messages.JobManagerMessages; +import org.apache.flink.runtime.rpc.RpcMethod; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.rpc.RpcEndpoint; -import org.apache.flink.runtime.rpc.RpcMethod; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.taskmanager.TaskExecutionState; import scala.concurrent.ExecutionContext$; @@ -467,6 +469,29 @@ public void registerAtResourceManager(final String address) { //TODO:: register at the RM } + /** + * Register jobInfo tracker + * + * @param clientAddress address of jobInfo tracker + * @param listeningBehaviour listening behaviour of jobInfo tracker + */ + @RpcMethod + public JobManagerMessages.RegisterJobClientSuccess registerJobInfoTracker(String clientAddress, ListeningBehaviour listeningBehaviour) { + // TODO:: add client to listeners + return new JobManagerMessages.RegisterJobClientSuccess(jobGraph.getJobID()); + } + + /** + * Request class load property of job + * + * @return + */ + @RpcMethod + public JobManagerMessages.ClassloadingProps requestClassloadingProps() { + // TODO + return null; + } + //---------------------------------------------------------------------------------------------- // Helper methods //---------------------------------------------------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java index b281ea8ce7132..463e9a9a54ea5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java @@ -18,9 +18,13 @@ package org.apache.flink.runtime.jobmaster; +import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.concurrent.Future; +import org.apache.flink.runtime.akka.ListeningBehaviour; import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.messages.JobManagerMessages; import org.apache.flink.runtime.rpc.RpcGateway; +import org.apache.flink.runtime.rpc.RpcTimeout; import org.apache.flink.runtime.taskmanager.TaskExecutionState; /** @@ -55,4 +59,20 @@ public interface JobMasterGateway extends RpcGateway { * @param address Address of the resource manager */ void registerAtResourceManager(final String address); + + /** + * Register jobInfo tracker + * + * @param clientAddress address of jobInfo tracker + * @param listeningBehaviour listening behaviour of jobInfo tracker + * @param timeout + */ + Future registerJobInfoTracker(String clientAddress, ListeningBehaviour listeningBehaviour, @RpcTimeout Time timeout); + + /** + * Request class load property of job + * @param timeout + * @return + */ + Future requestClassloadingProps(@RpcTimeout Time timeout); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java index 96844ed53fb79..98fa5bf2848f3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java @@ -53,6 +53,14 @@ public interface RpcService { */ Future connect(String address, Class clazz); + /** + * Try to connect to a remote rpc server under the provided address. Return false if the address is unreachable. + * + * @param address Address of the remote rpc server + * @return + */ + Future isReachable(String address); + /** * Start a rpc server which forwards the remote procedure calls to the provided rpc endpoint. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java index fb7896aa15842..10d15613baa03 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java @@ -152,6 +152,26 @@ public C checkedApply(Object obj) throws Exception { return new FlinkFuture<>(resultFuture); } + @Override + public Future isReachable(String address) { + checkState(!stopped, "RpcService is stopped"); + + LOG.debug("Try to connect to remote RPC endpoint with address {}.", address); + + final scala.concurrent.Future identify = Patterns.ask(actorSystem.actorSelection(address), new Identify(42), timeout.toMilliseconds()); + scala.concurrent.Future resultFuture = identify.map(new Mapper(){ + @Override + public Boolean apply(Object obj) { + + ActorIdentity actorIdentity = (ActorIdentity) obj; + return actorIdentity.getRef() != null; + } + }, actorSystem.dispatcher()); + + return new FlinkFuture<>(resultFuture); + } + + @Override public > C startServer(S rpcEndpoint) { checkNotNull(rpcEndpoint, "rpc endpoint"); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java index 47c9e24cdd855..be16735ca4ccd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java @@ -92,6 +92,22 @@ public void registerGateway(String address, RpcGateway gateway) { } } + @Override + public > C startServer(S rpcEndpoint) { + C gateway = super.startServer(rpcEndpoint); + // register started gateway of rpcEndpoint + registerGateway(gateway.getAddress(), gateway); + return gateway; + } + + @Override + public void stopServer(RpcGateway selfGateway) { + super.stopServer(selfGateway); + // unregister gateway + registeredConnections.remove(selfGateway.getAddress()); + } + + @Override public Future connect(String address, Class clazz) { RpcGateway gateway = registeredConnections.get(address); @@ -110,6 +126,11 @@ public Future connect(String address, Class clazz) } } + @Override + public Future isReachable(String address) { + return FlinkCompletableFuture.completed(registeredConnections.containsKey(address)); + } + public void clearGateways() { registeredConnections.clear(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java index 2a004c56f701e..0d0ac8402114b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java @@ -115,7 +115,7 @@ public > C startServer(S rpcEndpo akkaInvocationHandler); // register self - registeredConnections.putIfAbsent(self.getAddress(), self); + registerGateway(self.getAddress(), self); return self; } @@ -143,6 +143,11 @@ public Future connect(String address, Class clazz) } } + @Override + public Future isReachable(String address) { + return FlinkCompletableFuture.completed(registeredConnections.containsKey(address)); + } + // ------------------------------------------------------------------------ // connections // ------------------------------------------------------------------------ From c82d204a092e9aedbb4e36a7c84aae435d4d1133 Mon Sep 17 00:00:00 2001 From: beyond1920 Date: Tue, 27 Sep 2016 10:33:32 +0800 Subject: [PATCH 49/50] modify testcases --- .../org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java index 5d76024c364ec..2f65255245aa5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java @@ -33,7 +33,9 @@ import org.junit.Test; import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import static org.apache.flink.util.Preconditions.checkArgument; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; From 29fb66d5057e63fd21f5f897469db356203e2f38 Mon Sep 17 00:00:00 2001 From: beyond1920 Date: Wed, 28 Sep 2016 09:48:46 +0800 Subject: [PATCH 50/50] override AkkaRpcActorTest --- .../org/apache/flink/runtime/jobClient/JobClientUtils.java | 6 +++--- .../org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java | 2 -- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobClient/JobClientUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobClient/JobClientUtils.java index 9a0ff4ccf1252..80062544e0fd2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobClient/JobClientUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobClient/JobClientUtils.java @@ -36,7 +36,7 @@ import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.messages.JobManagerMessages; import org.apache.flink.runtime.rpc.RpcService; -import org.apache.flink.runtime.rpc.akka.AkkaRpcService; +import org.apache.flink.runtime.rpc.RpcServiceUtils; import org.apache.flink.util.NetUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -82,7 +82,7 @@ public class JobClientUtils { * @throws IOException */ public static RpcService startJobClientRpcService(Configuration config) - throws IOException + throws Exception { LOG.info("Starting JobClientUtils rpc service"); Option> remoting = new Some<>(new Tuple2("", 0)); @@ -96,7 +96,7 @@ public static RpcService startJobClientRpcService(Configuration config) "(unknown)"; int port = address.port().isDefined() ? ((Integer) address.port().get()) : -1; LOG.info("Started JobClientUtils actor system at " + hostAddress + ':' + port); - return new AkkaRpcService(system, Time.milliseconds(AkkaUtils.getClientTimeout(config).toMillis())); + return RpcServiceUtils.createRpcService(hostAddress, port, config); } /** diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java index 2f65255245aa5..5d76024c364ec 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java @@ -33,9 +33,7 @@ import org.junit.Test; import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import static org.apache.flink.util.Preconditions.checkArgument; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue;