dmvk commented on a change in pull request #18169: URL: https://github.com/apache/flink/pull/18169#discussion_r788841207
########## File path: flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerDisconnectOnShutdownITCase.java ########## @@ -0,0 +1,304 @@ +/* + * 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.test.recovery; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.AkkaOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ConfigurationUtils; +import org.apache.flink.configuration.HeartbeatManagerOptions; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.core.plugin.PluginManager; +import org.apache.flink.core.plugin.PluginUtils; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.entrypoint.ClusterInformation; +import org.apache.flink.runtime.entrypoint.StandaloneSessionClusterEntrypoint; +import org.apache.flink.runtime.entrypoint.component.DefaultDispatcherResourceManagerComponentFactory; +import org.apache.flink.runtime.heartbeat.HeartbeatServices; +import org.apache.flink.runtime.instance.InstanceID; +import org.apache.flink.runtime.io.network.partition.ResourceManagerPartitionTrackerImpl; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup; +import org.apache.flink.runtime.resourcemanager.ArbitraryWorkerResourceSpecFactory; +import org.apache.flink.runtime.resourcemanager.ResourceManager; +import org.apache.flink.runtime.resourcemanager.ResourceManagerFactory; +import org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServices; +import org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServicesConfiguration; +import org.apache.flink.runtime.resourcemanager.StandaloneResourceManager; +import org.apache.flink.runtime.resourcemanager.StandaloneResourceManagerFactory; +import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.taskexecutor.SlotReport; +import org.apache.flink.runtime.taskexecutor.TaskExecutorResourceUtils; +import org.apache.flink.runtime.taskexecutor.TaskManagerRunner; +import org.apache.flink.test.util.TestProcessBuilder; +import org.apache.flink.util.ConfigurationException; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.time.Duration; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +/** This test ensures the TaskManager disconnects from the ResourceManager on shutdown. */ +public class TaskManagerDisconnectOnShutdownITCase extends TestLogger { + + @Test + public void testTaskManagerProcessFailure() throws Exception { + Configuration config = new Configuration(); + config.setString(JobManagerOptions.ADDRESS, "localhost"); + + // disable heartbeats + config.set(HeartbeatManagerOptions.HEARTBEAT_RPC_FAILURE_THRESHOLD, -1); + config.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, 2); + + config.set(TaskManagerOptions.MANAGED_MEMORY_SIZE, MemorySize.parse("4m")); + config.set(TaskManagerOptions.NETWORK_MEMORY_MIN, MemorySize.parse("3200k")); + config.set(TaskManagerOptions.NETWORK_MEMORY_MAX, MemorySize.parse("3200k")); + config.set(TaskManagerOptions.TASK_HEAP_MEMORY, MemorySize.parse("128m")); + config.set(TaskManagerOptions.CPU_CORES, 1.0); + config.setString(JobManagerOptions.EXECUTION_FAILOVER_STRATEGY, "full"); + config.set(JobManagerOptions.RESOURCE_WAIT_TIMEOUT, Duration.ofSeconds(30L)); + + // check that we run this test only if the java command + // is available on this machine + String javaCommand = getJavaCommandPath(); + if (javaCommand == null) { + fail("cannot find java executable"); + } + + TaskManagerConnectionTracker tracker = new TaskManagerConnectionTracker(); + + TestProcessBuilder.TestProcess taskManagerProcess = null; + try (final StandaloneSessionClusterEntrypoint clusterEntrypoint = + new StandaloneSessionClusterEntrypoint(config) { + @Override + protected DefaultDispatcherResourceManagerComponentFactory + createDispatcherResourceManagerComponentFactory( + Configuration configuration) { + return DefaultDispatcherResourceManagerComponentFactory + .createSessionComponentFactory( + new TestingStandaloneResourceManagerFactory(tracker)); + } + }) { + clusterEntrypoint.startCluster(); + + TestProcessBuilder taskManagerProcessBuilder = + new TestProcessBuilder(TaskExecutorProcessEntryPoint.class.getName()); + taskManagerProcessBuilder.addConfigAsMainClassArgs(config); + + // start the TaskManager processes + taskManagerProcess = taskManagerProcessBuilder.start(); + + tracker.waitForTaskManagerConnected(); + + // shutdown TaskManager + taskManagerProcess.destroy(); + + tracker.waitForTaskManagerDisconnected(); + + assertEquals( + "only one TaskManager should connect", + 1, + tracker.getNumberOfConnectedTaskManager()); + } catch (Exception e) { + e.printStackTrace(); + printProcessLog(taskManagerProcess); + fail(e.getMessage()); + } catch (Error e) { + e.printStackTrace(); + printProcessLog(taskManagerProcess); + fail(e.getMessage()); + throw e; + } finally { + if (taskManagerProcess != null) { + taskManagerProcess.destroy(); + if (!taskManagerProcess.getProcess().waitFor(30, TimeUnit.SECONDS)) { + log.error("TaskManager did not shutdown in time."); + printProcessLog(taskManagerProcess); + taskManagerProcess.getProcess().destroyForcibly(); + } + } + } + } + + protected static void printProcessLog(TestProcessBuilder.TestProcess process) { Review comment: can we just reuse the `AbstractTaskManagerProcessFailureRecoveryTest#printProcessLog` method? ########## File path: flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerDisconnectOnShutdownITCase.java ########## @@ -0,0 +1,304 @@ +/* + * 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.test.recovery; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.AkkaOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ConfigurationUtils; +import org.apache.flink.configuration.HeartbeatManagerOptions; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.core.plugin.PluginManager; +import org.apache.flink.core.plugin.PluginUtils; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.entrypoint.ClusterInformation; +import org.apache.flink.runtime.entrypoint.StandaloneSessionClusterEntrypoint; +import org.apache.flink.runtime.entrypoint.component.DefaultDispatcherResourceManagerComponentFactory; +import org.apache.flink.runtime.heartbeat.HeartbeatServices; +import org.apache.flink.runtime.instance.InstanceID; +import org.apache.flink.runtime.io.network.partition.ResourceManagerPartitionTrackerImpl; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup; +import org.apache.flink.runtime.resourcemanager.ArbitraryWorkerResourceSpecFactory; +import org.apache.flink.runtime.resourcemanager.ResourceManager; +import org.apache.flink.runtime.resourcemanager.ResourceManagerFactory; +import org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServices; +import org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServicesConfiguration; +import org.apache.flink.runtime.resourcemanager.StandaloneResourceManager; +import org.apache.flink.runtime.resourcemanager.StandaloneResourceManagerFactory; +import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.taskexecutor.SlotReport; +import org.apache.flink.runtime.taskexecutor.TaskExecutorResourceUtils; +import org.apache.flink.runtime.taskexecutor.TaskManagerRunner; +import org.apache.flink.test.util.TestProcessBuilder; +import org.apache.flink.util.ConfigurationException; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.time.Duration; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath; +import static org.junit.Assert.assertEquals; Review comment: also the community has agreed upon unifying the assertion framework, so we should stick with using assertj for the new tests ########## File path: flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerDisconnectOnShutdownITCase.java ########## @@ -0,0 +1,304 @@ +/* + * 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.test.recovery; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.AkkaOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ConfigurationUtils; +import org.apache.flink.configuration.HeartbeatManagerOptions; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.core.plugin.PluginManager; +import org.apache.flink.core.plugin.PluginUtils; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.entrypoint.ClusterInformation; +import org.apache.flink.runtime.entrypoint.StandaloneSessionClusterEntrypoint; +import org.apache.flink.runtime.entrypoint.component.DefaultDispatcherResourceManagerComponentFactory; +import org.apache.flink.runtime.heartbeat.HeartbeatServices; +import org.apache.flink.runtime.instance.InstanceID; +import org.apache.flink.runtime.io.network.partition.ResourceManagerPartitionTrackerImpl; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup; +import org.apache.flink.runtime.resourcemanager.ArbitraryWorkerResourceSpecFactory; +import org.apache.flink.runtime.resourcemanager.ResourceManager; +import org.apache.flink.runtime.resourcemanager.ResourceManagerFactory; +import org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServices; +import org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServicesConfiguration; +import org.apache.flink.runtime.resourcemanager.StandaloneResourceManager; +import org.apache.flink.runtime.resourcemanager.StandaloneResourceManagerFactory; +import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.taskexecutor.SlotReport; +import org.apache.flink.runtime.taskexecutor.TaskExecutorResourceUtils; +import org.apache.flink.runtime.taskexecutor.TaskManagerRunner; +import org.apache.flink.test.util.TestProcessBuilder; +import org.apache.flink.util.ConfigurationException; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.time.Duration; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +/** This test ensures the TaskManager disconnects from the ResourceManager on shutdown. */ +public class TaskManagerDisconnectOnShutdownITCase extends TestLogger { Review comment: with junit5 we have an junit extension for the testlogger ########## File path: flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerDisconnectOnShutdownITCase.java ########## @@ -0,0 +1,304 @@ +/* + * 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.test.recovery; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.AkkaOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ConfigurationUtils; +import org.apache.flink.configuration.HeartbeatManagerOptions; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.core.plugin.PluginManager; +import org.apache.flink.core.plugin.PluginUtils; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.entrypoint.ClusterInformation; +import org.apache.flink.runtime.entrypoint.StandaloneSessionClusterEntrypoint; +import org.apache.flink.runtime.entrypoint.component.DefaultDispatcherResourceManagerComponentFactory; +import org.apache.flink.runtime.heartbeat.HeartbeatServices; +import org.apache.flink.runtime.instance.InstanceID; +import org.apache.flink.runtime.io.network.partition.ResourceManagerPartitionTrackerImpl; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup; +import org.apache.flink.runtime.resourcemanager.ArbitraryWorkerResourceSpecFactory; +import org.apache.flink.runtime.resourcemanager.ResourceManager; +import org.apache.flink.runtime.resourcemanager.ResourceManagerFactory; +import org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServices; +import org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServicesConfiguration; +import org.apache.flink.runtime.resourcemanager.StandaloneResourceManager; +import org.apache.flink.runtime.resourcemanager.StandaloneResourceManagerFactory; +import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.taskexecutor.SlotReport; +import org.apache.flink.runtime.taskexecutor.TaskExecutorResourceUtils; +import org.apache.flink.runtime.taskexecutor.TaskManagerRunner; +import org.apache.flink.test.util.TestProcessBuilder; +import org.apache.flink.util.ConfigurationException; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.time.Duration; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +/** This test ensures the TaskManager disconnects from the ResourceManager on shutdown. */ +public class TaskManagerDisconnectOnShutdownITCase extends TestLogger { + + @Test + public void testTaskManagerProcessFailure() throws Exception { + Configuration config = new Configuration(); + config.setString(JobManagerOptions.ADDRESS, "localhost"); + + // disable heartbeats + config.set(HeartbeatManagerOptions.HEARTBEAT_RPC_FAILURE_THRESHOLD, -1); + config.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, 2); + + config.set(TaskManagerOptions.MANAGED_MEMORY_SIZE, MemorySize.parse("4m")); + config.set(TaskManagerOptions.NETWORK_MEMORY_MIN, MemorySize.parse("3200k")); + config.set(TaskManagerOptions.NETWORK_MEMORY_MAX, MemorySize.parse("3200k")); + config.set(TaskManagerOptions.TASK_HEAP_MEMORY, MemorySize.parse("128m")); + config.set(TaskManagerOptions.CPU_CORES, 1.0); + config.setString(JobManagerOptions.EXECUTION_FAILOVER_STRATEGY, "full"); + config.set(JobManagerOptions.RESOURCE_WAIT_TIMEOUT, Duration.ofSeconds(30L)); + + // check that we run this test only if the java command + // is available on this machine + String javaCommand = getJavaCommandPath(); + if (javaCommand == null) { + fail("cannot find java executable"); + } + + TaskManagerConnectionTracker tracker = new TaskManagerConnectionTracker(); + + TestProcessBuilder.TestProcess taskManagerProcess = null; + try (final StandaloneSessionClusterEntrypoint clusterEntrypoint = + new StandaloneSessionClusterEntrypoint(config) { + @Override + protected DefaultDispatcherResourceManagerComponentFactory + createDispatcherResourceManagerComponentFactory( + Configuration configuration) { + return DefaultDispatcherResourceManagerComponentFactory + .createSessionComponentFactory( + new TestingStandaloneResourceManagerFactory(tracker)); + } + }) { + clusterEntrypoint.startCluster(); + + TestProcessBuilder taskManagerProcessBuilder = + new TestProcessBuilder(TaskExecutorProcessEntryPoint.class.getName()); + taskManagerProcessBuilder.addConfigAsMainClassArgs(config); + + // start the TaskManager processes + taskManagerProcess = taskManagerProcessBuilder.start(); + + tracker.waitForTaskManagerConnected(); + + // shutdown TaskManager + taskManagerProcess.destroy(); + + tracker.waitForTaskManagerDisconnected(); + + assertEquals( + "only one TaskManager should connect", + 1, + tracker.getNumberOfConnectedTaskManager()); + } catch (Exception e) { + e.printStackTrace(); + printProcessLog(taskManagerProcess); + fail(e.getMessage()); + } catch (Error e) { + e.printStackTrace(); + printProcessLog(taskManagerProcess); + fail(e.getMessage()); + throw e; Review comment: ```suggestion } catch (Throwable t) { printProcessLog(taskManagerProcess); throw t; ``` We shouldn't really print the stack traces directly (I know we're doing it in the original code, but that seems to be a leftover). ########## File path: flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerDisconnectOnShutdownITCase.java ########## @@ -0,0 +1,304 @@ +/* + * 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.test.recovery; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.AkkaOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ConfigurationUtils; +import org.apache.flink.configuration.HeartbeatManagerOptions; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.core.plugin.PluginManager; +import org.apache.flink.core.plugin.PluginUtils; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.entrypoint.ClusterInformation; +import org.apache.flink.runtime.entrypoint.StandaloneSessionClusterEntrypoint; +import org.apache.flink.runtime.entrypoint.component.DefaultDispatcherResourceManagerComponentFactory; +import org.apache.flink.runtime.heartbeat.HeartbeatServices; +import org.apache.flink.runtime.instance.InstanceID; +import org.apache.flink.runtime.io.network.partition.ResourceManagerPartitionTrackerImpl; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup; +import org.apache.flink.runtime.resourcemanager.ArbitraryWorkerResourceSpecFactory; +import org.apache.flink.runtime.resourcemanager.ResourceManager; +import org.apache.flink.runtime.resourcemanager.ResourceManagerFactory; +import org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServices; +import org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServicesConfiguration; +import org.apache.flink.runtime.resourcemanager.StandaloneResourceManager; +import org.apache.flink.runtime.resourcemanager.StandaloneResourceManagerFactory; +import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.taskexecutor.SlotReport; +import org.apache.flink.runtime.taskexecutor.TaskExecutorResourceUtils; +import org.apache.flink.runtime.taskexecutor.TaskManagerRunner; +import org.apache.flink.test.util.TestProcessBuilder; +import org.apache.flink.util.ConfigurationException; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; Review comment: we should use junit5 for new unit tests ########## File path: flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerDisconnectOnShutdownITCase.java ########## @@ -0,0 +1,304 @@ +/* + * 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.test.recovery; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.AkkaOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ConfigurationUtils; +import org.apache.flink.configuration.HeartbeatManagerOptions; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.core.plugin.PluginManager; +import org.apache.flink.core.plugin.PluginUtils; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.entrypoint.ClusterInformation; +import org.apache.flink.runtime.entrypoint.StandaloneSessionClusterEntrypoint; +import org.apache.flink.runtime.entrypoint.component.DefaultDispatcherResourceManagerComponentFactory; +import org.apache.flink.runtime.heartbeat.HeartbeatServices; +import org.apache.flink.runtime.instance.InstanceID; +import org.apache.flink.runtime.io.network.partition.ResourceManagerPartitionTrackerImpl; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup; +import org.apache.flink.runtime.resourcemanager.ArbitraryWorkerResourceSpecFactory; +import org.apache.flink.runtime.resourcemanager.ResourceManager; +import org.apache.flink.runtime.resourcemanager.ResourceManagerFactory; +import org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServices; +import org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServicesConfiguration; +import org.apache.flink.runtime.resourcemanager.StandaloneResourceManager; +import org.apache.flink.runtime.resourcemanager.StandaloneResourceManagerFactory; +import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.taskexecutor.SlotReport; +import org.apache.flink.runtime.taskexecutor.TaskExecutorResourceUtils; +import org.apache.flink.runtime.taskexecutor.TaskManagerRunner; +import org.apache.flink.test.util.TestProcessBuilder; +import org.apache.flink.util.ConfigurationException; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.time.Duration; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +/** This test ensures the TaskManager disconnects from the ResourceManager on shutdown. */ +public class TaskManagerDisconnectOnShutdownITCase extends TestLogger { + + @Test + public void testTaskManagerProcessFailure() throws Exception { + Configuration config = new Configuration(); + config.setString(JobManagerOptions.ADDRESS, "localhost"); + + // disable heartbeats + config.set(HeartbeatManagerOptions.HEARTBEAT_RPC_FAILURE_THRESHOLD, -1); + config.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, 2); + + config.set(TaskManagerOptions.MANAGED_MEMORY_SIZE, MemorySize.parse("4m")); + config.set(TaskManagerOptions.NETWORK_MEMORY_MIN, MemorySize.parse("3200k")); + config.set(TaskManagerOptions.NETWORK_MEMORY_MAX, MemorySize.parse("3200k")); + config.set(TaskManagerOptions.TASK_HEAP_MEMORY, MemorySize.parse("128m")); + config.set(TaskManagerOptions.CPU_CORES, 1.0); + config.setString(JobManagerOptions.EXECUTION_FAILOVER_STRATEGY, "full"); + config.set(JobManagerOptions.RESOURCE_WAIT_TIMEOUT, Duration.ofSeconds(30L)); + + // check that we run this test only if the java command + // is available on this machine + String javaCommand = getJavaCommandPath(); + if (javaCommand == null) { + fail("cannot find java executable"); + } + + TaskManagerConnectionTracker tracker = new TaskManagerConnectionTracker(); + + TestProcessBuilder.TestProcess taskManagerProcess = null; + try (final StandaloneSessionClusterEntrypoint clusterEntrypoint = + new StandaloneSessionClusterEntrypoint(config) { + @Override + protected DefaultDispatcherResourceManagerComponentFactory + createDispatcherResourceManagerComponentFactory( + Configuration configuration) { + return DefaultDispatcherResourceManagerComponentFactory + .createSessionComponentFactory( + new TestingStandaloneResourceManagerFactory(tracker)); + } + }) { + clusterEntrypoint.startCluster(); + + TestProcessBuilder taskManagerProcessBuilder = + new TestProcessBuilder(TaskExecutorProcessEntryPoint.class.getName()); + taskManagerProcessBuilder.addConfigAsMainClassArgs(config); + + // start the TaskManager processes + taskManagerProcess = taskManagerProcessBuilder.start(); + + tracker.waitForTaskManagerConnected(); + + // shutdown TaskManager + taskManagerProcess.destroy(); + + tracker.waitForTaskManagerDisconnected(); + + assertEquals( + "only one TaskManager should connect", + 1, + tracker.getNumberOfConnectedTaskManager()); + } catch (Exception e) { + e.printStackTrace(); + printProcessLog(taskManagerProcess); + fail(e.getMessage()); + } catch (Error e) { + e.printStackTrace(); + printProcessLog(taskManagerProcess); + fail(e.getMessage()); + throw e; + } finally { + if (taskManagerProcess != null) { + taskManagerProcess.destroy(); + if (!taskManagerProcess.getProcess().waitFor(30, TimeUnit.SECONDS)) { + log.error("TaskManager did not shutdown in time."); + printProcessLog(taskManagerProcess); + taskManagerProcess.getProcess().destroyForcibly(); + } + } + } + } + + protected static void printProcessLog(TestProcessBuilder.TestProcess process) { + if (process == null) { + System.out.println("-----------------------------------------"); + System.out.println(" TaskManager WAS NOT STARTED."); + System.out.println("-----------------------------------------"); + } else { + System.out.println("-----------------------------------------"); + System.out.println(" BEGIN SPAWNED PROCESS LOG FOR TaskManager"); + System.out.println("-----------------------------------------"); + System.out.println(process.getErrorOutput().toString()); + System.out.println("-----------------------------------------"); + System.out.println(" END SPAWNED PROCESS LOG"); + System.out.println("-----------------------------------------"); + } + } + + // -------------------------------------------------------------------------------------------- + + /** + * The entry point for the TaskExecutor JVM. Simply configures and runs a TaskExecutor. Copy + * from AbstractTaskManagerProcessFailureRecoveryTest. + */ + public static class TaskExecutorProcessEntryPoint { Review comment: again, can we reuse the one from `AbstractTaskManagerProcessFailureRecoveryTest`? ########## File path: flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerDisconnectOnShutdownITCase.java ########## @@ -0,0 +1,304 @@ +/* + * 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.test.recovery; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.AkkaOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ConfigurationUtils; +import org.apache.flink.configuration.HeartbeatManagerOptions; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.core.plugin.PluginManager; +import org.apache.flink.core.plugin.PluginUtils; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.entrypoint.ClusterInformation; +import org.apache.flink.runtime.entrypoint.StandaloneSessionClusterEntrypoint; +import org.apache.flink.runtime.entrypoint.component.DefaultDispatcherResourceManagerComponentFactory; +import org.apache.flink.runtime.heartbeat.HeartbeatServices; +import org.apache.flink.runtime.instance.InstanceID; +import org.apache.flink.runtime.io.network.partition.ResourceManagerPartitionTrackerImpl; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup; +import org.apache.flink.runtime.resourcemanager.ArbitraryWorkerResourceSpecFactory; +import org.apache.flink.runtime.resourcemanager.ResourceManager; +import org.apache.flink.runtime.resourcemanager.ResourceManagerFactory; +import org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServices; +import org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServicesConfiguration; +import org.apache.flink.runtime.resourcemanager.StandaloneResourceManager; +import org.apache.flink.runtime.resourcemanager.StandaloneResourceManagerFactory; +import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.taskexecutor.SlotReport; +import org.apache.flink.runtime.taskexecutor.TaskExecutorResourceUtils; +import org.apache.flink.runtime.taskexecutor.TaskManagerRunner; +import org.apache.flink.test.util.TestProcessBuilder; +import org.apache.flink.util.ConfigurationException; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.time.Duration; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +/** This test ensures the TaskManager disconnects from the ResourceManager on shutdown. */ +public class TaskManagerDisconnectOnShutdownITCase extends TestLogger { + + @Test + public void testTaskManagerProcessFailure() throws Exception { + Configuration config = new Configuration(); + config.setString(JobManagerOptions.ADDRESS, "localhost"); + + // disable heartbeats + config.set(HeartbeatManagerOptions.HEARTBEAT_RPC_FAILURE_THRESHOLD, -1); + config.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, 2); + + config.set(TaskManagerOptions.MANAGED_MEMORY_SIZE, MemorySize.parse("4m")); + config.set(TaskManagerOptions.NETWORK_MEMORY_MIN, MemorySize.parse("3200k")); + config.set(TaskManagerOptions.NETWORK_MEMORY_MAX, MemorySize.parse("3200k")); + config.set(TaskManagerOptions.TASK_HEAP_MEMORY, MemorySize.parse("128m")); + config.set(TaskManagerOptions.CPU_CORES, 1.0); + config.setString(JobManagerOptions.EXECUTION_FAILOVER_STRATEGY, "full"); + config.set(JobManagerOptions.RESOURCE_WAIT_TIMEOUT, Duration.ofSeconds(30L)); + + // check that we run this test only if the java command + // is available on this machine + String javaCommand = getJavaCommandPath(); + if (javaCommand == null) { + fail("cannot find java executable"); + } + + TaskManagerConnectionTracker tracker = new TaskManagerConnectionTracker(); + + TestProcessBuilder.TestProcess taskManagerProcess = null; + try (final StandaloneSessionClusterEntrypoint clusterEntrypoint = + new StandaloneSessionClusterEntrypoint(config) { + @Override + protected DefaultDispatcherResourceManagerComponentFactory + createDispatcherResourceManagerComponentFactory( + Configuration configuration) { + return DefaultDispatcherResourceManagerComponentFactory + .createSessionComponentFactory( + new TestingStandaloneResourceManagerFactory(tracker)); + } + }) { + clusterEntrypoint.startCluster(); + + TestProcessBuilder taskManagerProcessBuilder = + new TestProcessBuilder(TaskExecutorProcessEntryPoint.class.getName()); + taskManagerProcessBuilder.addConfigAsMainClassArgs(config); + + // start the TaskManager processes + taskManagerProcess = taskManagerProcessBuilder.start(); + + tracker.waitForTaskManagerConnected(); + + // shutdown TaskManager + taskManagerProcess.destroy(); + + tracker.waitForTaskManagerDisconnected(); + + assertEquals( + "only one TaskManager should connect", + 1, + tracker.getNumberOfConnectedTaskManager()); + } catch (Exception e) { + e.printStackTrace(); + printProcessLog(taskManagerProcess); + fail(e.getMessage()); + } catch (Error e) { + e.printStackTrace(); + printProcessLog(taskManagerProcess); + fail(e.getMessage()); + throw e; + } finally { + if (taskManagerProcess != null) { + taskManagerProcess.destroy(); + if (!taskManagerProcess.getProcess().waitFor(30, TimeUnit.SECONDS)) { + log.error("TaskManager did not shutdown in time."); + printProcessLog(taskManagerProcess); + taskManagerProcess.getProcess().destroyForcibly(); + } + } + } + } + + protected static void printProcessLog(TestProcessBuilder.TestProcess process) { + if (process == null) { + System.out.println("-----------------------------------------"); + System.out.println(" TaskManager WAS NOT STARTED."); + System.out.println("-----------------------------------------"); + } else { + System.out.println("-----------------------------------------"); + System.out.println(" BEGIN SPAWNED PROCESS LOG FOR TaskManager"); + System.out.println("-----------------------------------------"); + System.out.println(process.getErrorOutput().toString()); + System.out.println("-----------------------------------------"); + System.out.println(" END SPAWNED PROCESS LOG"); + System.out.println("-----------------------------------------"); + } + } + + // -------------------------------------------------------------------------------------------- + + /** + * The entry point for the TaskExecutor JVM. Simply configures and runs a TaskExecutor. Copy + * from AbstractTaskManagerProcessFailureRecoveryTest. + */ + public static class TaskExecutorProcessEntryPoint { + + private static final Logger LOG = + LoggerFactory.getLogger( + AbstractTaskManagerProcessFailureRecoveryTest.TaskExecutorProcessEntryPoint + .class); + + public static void main(String[] args) { + try { + final ParameterTool parameterTool = ParameterTool.fromArgs(args); + Configuration cfg = parameterTool.getConfiguration(); + final PluginManager pluginManager = + PluginUtils.createPluginManagerFromRootFolder(cfg); + TaskExecutorResourceUtils.adjustForLocalExecution(cfg); + + TaskManagerRunner.runTaskManager(cfg, pluginManager); + } catch (Throwable t) { + LOG.error("Failed to run the TaskManager process", t); + System.exit(1); + } + } + } + + private static class TestingStandaloneResourceManagerFactory + extends ResourceManagerFactory<ResourceID> { + + TaskManagerConnectionTracker tracker; + + public TestingStandaloneResourceManagerFactory(TaskManagerConnectionTracker tracker) { + this.tracker = tracker; + } + + @Override + protected ResourceManager<ResourceID> createResourceManager( + Configuration configuration, + ResourceID resourceId, + RpcService rpcService, + UUID leaderSessionId, + HeartbeatServices heartbeatServices, + FatalErrorHandler fatalErrorHandler, + ClusterInformation clusterInformation, + @Nullable String webInterfaceUrl, + ResourceManagerMetricGroup resourceManagerMetricGroup, + ResourceManagerRuntimeServices resourceManagerRuntimeServices, + Executor ioExecutor) { + + final Time standaloneClusterStartupPeriodTime = + ConfigurationUtils.getStandaloneClusterStartupPeriodTime(configuration); + + return new StandaloneResourceManager( + rpcService, + leaderSessionId, + resourceId, + heartbeatServices, + resourceManagerRuntimeServices.getSlotManager(), + ResourceManagerPartitionTrackerImpl::new, + resourceManagerRuntimeServices.getJobLeaderIdService(), + clusterInformation, + fatalErrorHandler, + resourceManagerMetricGroup, + standaloneClusterStartupPeriodTime, + Time.fromDuration(configuration.get(AkkaOptions.ASK_TIMEOUT_DURATION)), + ioExecutor) { + + @Override + public void disconnectTaskManager(ResourceID resourceId, Exception cause) { + tracker.disconnectTaskRanager(); + super.disconnectTaskManager(resourceId, cause); + } + + @Override + public CompletableFuture<Acknowledge> sendSlotReport( + ResourceID taskManagerResourceId, + InstanceID taskManagerRegistrationId, + SlotReport slotReport, + Time timeout) { + tracker.connectTaskRanager(); + return super.sendSlotReport( + taskManagerResourceId, taskManagerRegistrationId, slotReport, timeout); + } + }; + } + + @Override + protected ResourceManagerRuntimeServicesConfiguration + createResourceManagerRuntimeServicesConfiguration(Configuration configuration) + throws ConfigurationException { + return ResourceManagerRuntimeServicesConfiguration.fromConfiguration( + StandaloneResourceManagerFactory.getConfigurationWithoutMaxSlotNumberIfSet( + configuration), + ArbitraryWorkerResourceSpecFactory.INSTANCE); + } + } + + private static class TaskManagerConnectionTracker { + private final CompletableFuture<Void> taskManagerConnectedFuture = + new CompletableFuture<>(); + private final CompletableFuture<Void> taskManagerDisconnectedFuture = + new CompletableFuture<>(); + private final AtomicInteger numberOfConnectedTaskManager = new AtomicInteger(); + + public void connectTaskRanager() { + if (!taskManagerConnectedFuture.isDone()) { + taskManagerConnectedFuture.complete(null); + } + numberOfConnectedTaskManager.incrementAndGet(); + } + + public void disconnectTaskRanager() { Review comment: typo, same with the connect method ########## File path: flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerDisconnectOnShutdownITCase.java ########## @@ -0,0 +1,304 @@ +/* + * 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.test.recovery; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.AkkaOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ConfigurationUtils; +import org.apache.flink.configuration.HeartbeatManagerOptions; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.core.plugin.PluginManager; +import org.apache.flink.core.plugin.PluginUtils; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.entrypoint.ClusterInformation; +import org.apache.flink.runtime.entrypoint.StandaloneSessionClusterEntrypoint; +import org.apache.flink.runtime.entrypoint.component.DefaultDispatcherResourceManagerComponentFactory; +import org.apache.flink.runtime.heartbeat.HeartbeatServices; +import org.apache.flink.runtime.instance.InstanceID; +import org.apache.flink.runtime.io.network.partition.ResourceManagerPartitionTrackerImpl; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup; +import org.apache.flink.runtime.resourcemanager.ArbitraryWorkerResourceSpecFactory; +import org.apache.flink.runtime.resourcemanager.ResourceManager; +import org.apache.flink.runtime.resourcemanager.ResourceManagerFactory; +import org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServices; +import org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServicesConfiguration; +import org.apache.flink.runtime.resourcemanager.StandaloneResourceManager; +import org.apache.flink.runtime.resourcemanager.StandaloneResourceManagerFactory; +import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.taskexecutor.SlotReport; +import org.apache.flink.runtime.taskexecutor.TaskExecutorResourceUtils; +import org.apache.flink.runtime.taskexecutor.TaskManagerRunner; +import org.apache.flink.test.util.TestProcessBuilder; +import org.apache.flink.util.ConfigurationException; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.time.Duration; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +/** This test ensures the TaskManager disconnects from the ResourceManager on shutdown. */ +public class TaskManagerDisconnectOnShutdownITCase extends TestLogger { + + @Test + public void testTaskManagerProcessFailure() throws Exception { + Configuration config = new Configuration(); + config.setString(JobManagerOptions.ADDRESS, "localhost"); + + // disable heartbeats + config.set(HeartbeatManagerOptions.HEARTBEAT_RPC_FAILURE_THRESHOLD, -1); + config.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, 2); + + config.set(TaskManagerOptions.MANAGED_MEMORY_SIZE, MemorySize.parse("4m")); + config.set(TaskManagerOptions.NETWORK_MEMORY_MIN, MemorySize.parse("3200k")); + config.set(TaskManagerOptions.NETWORK_MEMORY_MAX, MemorySize.parse("3200k")); + config.set(TaskManagerOptions.TASK_HEAP_MEMORY, MemorySize.parse("128m")); + config.set(TaskManagerOptions.CPU_CORES, 1.0); + config.setString(JobManagerOptions.EXECUTION_FAILOVER_STRATEGY, "full"); + config.set(JobManagerOptions.RESOURCE_WAIT_TIMEOUT, Duration.ofSeconds(30L)); + + // check that we run this test only if the java command + // is available on this machine + String javaCommand = getJavaCommandPath(); + if (javaCommand == null) { + fail("cannot find java executable"); + } + + TaskManagerConnectionTracker tracker = new TaskManagerConnectionTracker(); + + TestProcessBuilder.TestProcess taskManagerProcess = null; + try (final StandaloneSessionClusterEntrypoint clusterEntrypoint = + new StandaloneSessionClusterEntrypoint(config) { + @Override + protected DefaultDispatcherResourceManagerComponentFactory + createDispatcherResourceManagerComponentFactory( + Configuration configuration) { + return DefaultDispatcherResourceManagerComponentFactory + .createSessionComponentFactory( + new TestingStandaloneResourceManagerFactory(tracker)); + } + }) { + clusterEntrypoint.startCluster(); + + TestProcessBuilder taskManagerProcessBuilder = + new TestProcessBuilder(TaskExecutorProcessEntryPoint.class.getName()); + taskManagerProcessBuilder.addConfigAsMainClassArgs(config); + + // start the TaskManager processes + taskManagerProcess = taskManagerProcessBuilder.start(); + + tracker.waitForTaskManagerConnected(); + + // shutdown TaskManager + taskManagerProcess.destroy(); + + tracker.waitForTaskManagerDisconnected(); + + assertEquals( + "only one TaskManager should connect", + 1, + tracker.getNumberOfConnectedTaskManager()); + } catch (Exception e) { + e.printStackTrace(); + printProcessLog(taskManagerProcess); + fail(e.getMessage()); + } catch (Error e) { + e.printStackTrace(); + printProcessLog(taskManagerProcess); + fail(e.getMessage()); + throw e; + } finally { + if (taskManagerProcess != null) { + taskManagerProcess.destroy(); Review comment: Should we send a sigkill here right away in case the process is still running? We've already sent a sigterm during the test. ########## File path: flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerDisconnectOnShutdownITCase.java ########## @@ -0,0 +1,304 @@ +/* + * 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.test.recovery; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.AkkaOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ConfigurationUtils; +import org.apache.flink.configuration.HeartbeatManagerOptions; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.core.plugin.PluginManager; +import org.apache.flink.core.plugin.PluginUtils; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.entrypoint.ClusterInformation; +import org.apache.flink.runtime.entrypoint.StandaloneSessionClusterEntrypoint; +import org.apache.flink.runtime.entrypoint.component.DefaultDispatcherResourceManagerComponentFactory; +import org.apache.flink.runtime.heartbeat.HeartbeatServices; +import org.apache.flink.runtime.instance.InstanceID; +import org.apache.flink.runtime.io.network.partition.ResourceManagerPartitionTrackerImpl; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup; +import org.apache.flink.runtime.resourcemanager.ArbitraryWorkerResourceSpecFactory; +import org.apache.flink.runtime.resourcemanager.ResourceManager; +import org.apache.flink.runtime.resourcemanager.ResourceManagerFactory; +import org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServices; +import org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServicesConfiguration; +import org.apache.flink.runtime.resourcemanager.StandaloneResourceManager; +import org.apache.flink.runtime.resourcemanager.StandaloneResourceManagerFactory; +import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.taskexecutor.SlotReport; +import org.apache.flink.runtime.taskexecutor.TaskExecutorResourceUtils; +import org.apache.flink.runtime.taskexecutor.TaskManagerRunner; +import org.apache.flink.test.util.TestProcessBuilder; +import org.apache.flink.util.ConfigurationException; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.time.Duration; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +/** This test ensures the TaskManager disconnects from the ResourceManager on shutdown. */ +public class TaskManagerDisconnectOnShutdownITCase extends TestLogger { + + @Test + public void testTaskManagerProcessFailure() throws Exception { + Configuration config = new Configuration(); + config.setString(JobManagerOptions.ADDRESS, "localhost"); + + // disable heartbeats + config.set(HeartbeatManagerOptions.HEARTBEAT_RPC_FAILURE_THRESHOLD, -1); + config.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, 2); + + config.set(TaskManagerOptions.MANAGED_MEMORY_SIZE, MemorySize.parse("4m")); + config.set(TaskManagerOptions.NETWORK_MEMORY_MIN, MemorySize.parse("3200k")); + config.set(TaskManagerOptions.NETWORK_MEMORY_MAX, MemorySize.parse("3200k")); + config.set(TaskManagerOptions.TASK_HEAP_MEMORY, MemorySize.parse("128m")); + config.set(TaskManagerOptions.CPU_CORES, 1.0); + config.setString(JobManagerOptions.EXECUTION_FAILOVER_STRATEGY, "full"); + config.set(JobManagerOptions.RESOURCE_WAIT_TIMEOUT, Duration.ofSeconds(30L)); + + // check that we run this test only if the java command + // is available on this machine + String javaCommand = getJavaCommandPath(); + if (javaCommand == null) { + fail("cannot find java executable"); + } + + TaskManagerConnectionTracker tracker = new TaskManagerConnectionTracker(); Review comment: nit: just a good practice to follow, always mark all the variables as `final` unless you intend to mutate them. It's bit more verbose, but it gives a good hint to anyone reading the code and it's less error prone in general. ########## File path: flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerDisconnectOnShutdownITCase.java ########## @@ -0,0 +1,304 @@ +/* + * 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.test.recovery; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.AkkaOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ConfigurationUtils; +import org.apache.flink.configuration.HeartbeatManagerOptions; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.core.plugin.PluginManager; +import org.apache.flink.core.plugin.PluginUtils; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.entrypoint.ClusterInformation; +import org.apache.flink.runtime.entrypoint.StandaloneSessionClusterEntrypoint; +import org.apache.flink.runtime.entrypoint.component.DefaultDispatcherResourceManagerComponentFactory; +import org.apache.flink.runtime.heartbeat.HeartbeatServices; +import org.apache.flink.runtime.instance.InstanceID; +import org.apache.flink.runtime.io.network.partition.ResourceManagerPartitionTrackerImpl; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup; +import org.apache.flink.runtime.resourcemanager.ArbitraryWorkerResourceSpecFactory; +import org.apache.flink.runtime.resourcemanager.ResourceManager; +import org.apache.flink.runtime.resourcemanager.ResourceManagerFactory; +import org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServices; +import org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServicesConfiguration; +import org.apache.flink.runtime.resourcemanager.StandaloneResourceManager; +import org.apache.flink.runtime.resourcemanager.StandaloneResourceManagerFactory; +import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.taskexecutor.SlotReport; +import org.apache.flink.runtime.taskexecutor.TaskExecutorResourceUtils; +import org.apache.flink.runtime.taskexecutor.TaskManagerRunner; +import org.apache.flink.test.util.TestProcessBuilder; +import org.apache.flink.util.ConfigurationException; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.time.Duration; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +/** This test ensures the TaskManager disconnects from the ResourceManager on shutdown. */ +public class TaskManagerDisconnectOnShutdownITCase extends TestLogger { + + @Test + public void testTaskManagerProcessFailure() throws Exception { + Configuration config = new Configuration(); + config.setString(JobManagerOptions.ADDRESS, "localhost"); + + // disable heartbeats + config.set(HeartbeatManagerOptions.HEARTBEAT_RPC_FAILURE_THRESHOLD, -1); + config.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, 2); + + config.set(TaskManagerOptions.MANAGED_MEMORY_SIZE, MemorySize.parse("4m")); + config.set(TaskManagerOptions.NETWORK_MEMORY_MIN, MemorySize.parse("3200k")); + config.set(TaskManagerOptions.NETWORK_MEMORY_MAX, MemorySize.parse("3200k")); + config.set(TaskManagerOptions.TASK_HEAP_MEMORY, MemorySize.parse("128m")); + config.set(TaskManagerOptions.CPU_CORES, 1.0); + config.setString(JobManagerOptions.EXECUTION_FAILOVER_STRATEGY, "full"); + config.set(JobManagerOptions.RESOURCE_WAIT_TIMEOUT, Duration.ofSeconds(30L)); + + // check that we run this test only if the java command + // is available on this machine + String javaCommand = getJavaCommandPath(); + if (javaCommand == null) { + fail("cannot find java executable"); + } + + TaskManagerConnectionTracker tracker = new TaskManagerConnectionTracker(); + + TestProcessBuilder.TestProcess taskManagerProcess = null; + try (final StandaloneSessionClusterEntrypoint clusterEntrypoint = + new StandaloneSessionClusterEntrypoint(config) { Review comment: ```suggestion try (final SessionClusterEntrypoint clusterEntrypoint = new SessionClusterEntrypoint(config) { ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
