tillrohrmann commented on a change in pull request #8687: [FLINK-12612][coordination] Track stored partition on the TaskExecutor URL: https://github.com/apache/flink/pull/8687#discussion_r295797271
########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorPartitionLifecycleTest.java ########## @@ -0,0 +1,454 @@ +/* + * 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.ExecutionConfig; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.testutils.BlockerSync; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.runtime.blob.BlobCacheService; +import org.apache.flink.runtime.blob.VoidBlobStore; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.concurrent.Executors; +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.execution.Environment; +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.executiongraph.PartitionInfo; +import org.apache.flink.runtime.heartbeat.HeartbeatServices; +import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; +import org.apache.flink.runtime.io.network.NettyShuffleEnvironmentBuilder; +import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; +import org.apache.flink.runtime.io.network.partition.PartitionProducerStateProvider; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.io.network.partition.ResultPartitionTest; +import org.apache.flink.runtime.io.network.partition.ResultPartitionType; +import org.apache.flink.runtime.io.network.partition.consumer.InputGate; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.jobmaster.JMTMRegistrationSuccess; +import org.apache.flink.runtime.jobmaster.JobMasterGateway; +import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGateway; +import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGatewayBuilder; +import org.apache.flink.runtime.leaderretrieval.SettableLeaderRetrievalService; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; +import org.apache.flink.runtime.rpc.RpcUtils; +import org.apache.flink.runtime.rpc.TestingRpcService; +import org.apache.flink.runtime.state.TaskExecutorLocalStateStoresManager; +import org.apache.flink.runtime.taskexecutor.partition.JobAwareShuffleEnvironment; +import org.apache.flink.runtime.taskexecutor.partition.JobAwareShuffleEnvironmentImpl; +import org.apache.flink.runtime.taskexecutor.slot.TaskSlotTable; +import org.apache.flink.runtime.taskexecutor.slot.TimerService; +import org.apache.flink.runtime.taskmanager.NoOpTaskManagerActions; +import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.apache.flink.runtime.util.TestingFatalErrorHandler; +import org.apache.flink.util.SerializedValue; +import org.apache.flink.util.TestLogger; +import org.apache.flink.util.function.TriConsumer; + +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.IOException; +import java.time.Duration; +import java.util.Collection; +import java.util.Collections; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.BiConsumer; +import java.util.function.Consumer; +import java.util.function.Function; + +import static junit.framework.TestCase.assertTrue; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; + +/** + * Tests for the partition-lifecycle logic in the {@link TaskExecutor}. + */ +public class TaskExecutorPartitionLifecycleTest extends TestLogger { + + private static final Time timeout = Time.seconds(10L); + + private static final TestingHighAvailabilityServices haServices = new TestingHighAvailabilityServices(); + private static final SettableLeaderRetrievalService jobManagerLeaderRetriever = new SettableLeaderRetrievalService(); + private static final JobID jobId = new JobID(); + + static { + haServices.setResourceManagerLeaderRetriever(new SettableLeaderRetrievalService()); + haServices.setJobMasterLeaderRetriever(jobId, jobManagerLeaderRetriever); + } + + @Rule + public final TemporaryFolder tmp = new TemporaryFolder(); + + private TestingRpcService rpc; + + @Before + public void setup() { + rpc = new TestingRpcService(); + } + + @After + public void shutdown() throws ExecutionException, InterruptedException { + if (rpc != null) { + rpc.stopService().get(); + } + } + + @Test + public void testConnectionTerminationAfterExternalRelease() throws IOException, InterruptedException, ExecutionException, TimeoutException { + final JobMasterGateway jobMasterGateway = new TestingJobMasterGatewayBuilder().build(); + + final JobManagerConnection jobManagerConnection = TaskSubmissionTestEnvironment.createJobManagerConnection( + jobId, jobMasterGateway, rpc, new NoOpTaskManagerActions(), timeout); + + final JobManagerTable jobManagerTable = new JobManagerTable(); + jobManagerTable.put(jobId, jobManagerConnection); + + final AtomicBoolean hasPartitionsOccupyingLocalResources = new AtomicBoolean(true); + final TestJobAwareShuffleEnvironment jobAwareShuffleEnvironment = new TestJobAwareShuffleEnvironment(jobId -> hasPartitionsOccupyingLocalResources.get()); + + final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() + .setJobManagerTable(jobManagerTable) + .setShuffleEnvironment(jobAwareShuffleEnvironment) + .setTaskSlotTable(createTaskSlotTable()) + .build(); + + final TestingTaskExecutor taskManager = createTestingTaskExecutor(taskManagerServices); + + try { + taskManager.start(); + taskManager.waitUntilStarted(); + + assertTrue(jobManagerTable.contains(jobId)); + + taskManager.releasePartitions(jobId, Collections.singletonList(new ResultPartitionID())); + // connection should be kept alive since the environment still says we have local resources + assertTrue(jobManagerTable.contains(jobId)); + + hasPartitionsOccupyingLocalResources.set(false); + + // the TM should check whether partitions are still stored, and afterwards terminate the connection + taskManager.releasePartitions(jobId, Collections.singletonList(new ResultPartitionID())); + assertFalse(jobManagerTable.contains(jobId)); + } finally { + RpcUtils.terminateRpcEndpoint(taskManager, timeout); + } + } + + @Test + public void testConnectionTerminationAfterInternalRelease() throws Exception { + final JobMasterGateway jobMasterGateway = new TestingJobMasterGatewayBuilder().build(); + + final JobManagerConnection jobManagerConnection = TaskSubmissionTestEnvironment.createJobManagerConnection( + jobId, jobMasterGateway, rpc, new NoOpTaskManagerActions(), timeout); + + final JobManagerTable jobManagerTable = new JobManagerTable(); + jobManagerTable.put(jobId, jobManagerConnection); + + final AtomicBoolean hasPartitionsOccupyingLocalResources = new AtomicBoolean(true); + final TestJobAwareShuffleEnvironment jobAwareShuffleEnvironment = new TestJobAwareShuffleEnvironment(jobId -> hasPartitionsOccupyingLocalResources.get()); + + final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() + .setJobManagerTable(jobManagerTable) + .setShuffleEnvironment(jobAwareShuffleEnvironment) + .setTaskSlotTable(createTaskSlotTable()) + .build(); + + final TestingTaskExecutor taskManager = createTestingTaskExecutor(taskManagerServices); + + try { + taskManager.start(); + taskManager.waitUntilStarted(); + + assertTrue(jobManagerTable.contains(jobId)); + assertNotNull(jobAwareShuffleEnvironment.listener); + + jobAwareShuffleEnvironment.listener.accept(jobId); + // connection should be kept alive since the environment still says we have local resources + assertTrue(jobManagerTable.contains(jobId)); + + // simulates the release of internal partitions + hasPartitionsOccupyingLocalResources.set(false); + + // the TM should terminate the connection since no more partitions are stored. + jobAwareShuffleEnvironment.listener.accept(jobId); + + // connection termination happens asynchronously + final Deadline deadline = Deadline.fromNow(Duration.ofSeconds(5)); + while (jobManagerTable.contains(jobId) && deadline.hasTimeLeft()){ + Thread.sleep(50); + } + assertFalse(jobManagerTable.contains(jobId)); + } finally { + RpcUtils.terminateRpcEndpoint(taskManager, timeout); + } + } + + @Test + public void testPartitionReleaseAfterDisconnect() throws Exception { + testPartitionRelease( + (jobId, partitionId, taskExecutorGateway) -> taskExecutorGateway.disconnectJobManager(jobId, new Exception("test")), + (jobId, jobAwareShuffleEnvironment) -> assertFalse(jobAwareShuffleEnvironment.hasPartitionsOccupyingLocalResources(jobId))); + } + + @Test + public void testPartitionReleaseAfterReleaseCall() throws Exception { + testPartitionRelease( + (jobId, partitionId, taskExecutorGateway) -> taskExecutorGateway.releasePartitions(jobId, Collections.singletonList(partitionId)), + (jobId, jobAwareShuffleEnvironment) -> assertFalse(jobAwareShuffleEnvironment.hasPartitionsOccupyingLocalResources(jobId))); + } + + @Test + public void testPartitionReleaseAfterShutdown() throws Exception { + // don't do any explicit release action, so that the partition must be cleaned up on shutdown + testPartitionRelease( + (jobId, partitionId, taskExecutorGateway) -> { }, + (jobId, jobAwareShuffleEnvironment) -> { }); + } + + private void testPartitionRelease( + TriConsumer<JobID, ResultPartitionID, TaskExecutorGateway> releaseAction, + BiConsumer<JobID, JobAwareShuffleEnvironmentImpl> postReleaseActionAssertion) throws Exception { + + final ResultPartitionDeploymentDescriptor taskResultPartitionDescriptor = + ResultPartitionTest.createPartitionDeploymentDescriptor(ResultPartitionType.BLOCKING); + final ExecutionAttemptID eid1 = taskResultPartitionDescriptor.getShuffleDescriptor().getResultPartitionID().getProducerId(); + + final TaskDeploymentDescriptor taskDeploymentDescriptor = + TaskExecutorSubmissionTest.createTaskDeploymentDescriptor( + jobId, + "job", + eid1, + new SerializedValue<>(new ExecutionConfig()), + "Sender", + 1, + 0, + 1, + 0, + new Configuration(), + new Configuration(), + TestInvokable.class.getName(), + Collections.singletonList(taskResultPartitionDescriptor), + Collections.emptyList(), + Collections.emptyList(), + Collections.emptyList(), + 0); + + final TaskSlotTable taskSlotTable = createTaskSlotTable(); + + final TaskExecutorLocalStateStoresManager localStateStoresManager = new TaskExecutorLocalStateStoresManager( + false, + new File[]{tmp.newFolder()}, + Executors.directExecutor()); + + final JobAwareShuffleEnvironmentImpl<?> jobAwareShuffleEnvironment = new JobAwareShuffleEnvironmentImpl<>(new NettyShuffleEnvironmentBuilder().build()); + + final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() + .setTaskSlotTable(taskSlotTable) + .setTaskStateManager(localStateStoresManager) + .setShuffleEnvironment(jobAwareShuffleEnvironment) + .build(); + + final CompletableFuture<Void> taskFinishedFuture = new CompletableFuture<>(); + + final TestingJobMasterGateway jobMasterGateway = new TestingJobMasterGatewayBuilder() + .setRegisterTaskManagerFunction((s, location) -> CompletableFuture.completedFuture(new JMTMRegistrationSuccess(ResourceID.generate()))) + .setOfferSlotsFunction((resourceID, slotOffers) -> CompletableFuture.completedFuture(slotOffers)) + .setUpdateTaskExecutionStateFunction(taskExecutionState -> { + if (taskExecutionState.getExecutionState() == ExecutionState.FINISHED) { + taskFinishedFuture.complete(null); + } + return CompletableFuture.completedFuture(Acknowledge.get()); + }) + .build(); + + final TestingTaskExecutor taskManager = createTestingTaskExecutor(taskManagerServices); + + try { + taskManager.start(); + taskManager.waitUntilStarted(); + + final String jobMasterAddress = "jm"; + rpc.registerGateway(jobMasterAddress, jobMasterGateway); + + // inform the task manager about the job leader + taskManagerServices.getJobLeaderService().addJob(jobId, jobMasterAddress); + jobManagerLeaderRetriever.notifyListener(jobMasterAddress, UUID.randomUUID()); + + final Deadline deadline = Deadline.fromNow(Duration.ofSeconds(5)); + while (!taskManagerServices.getJobManagerTable().contains(jobId) && deadline.hasTimeLeft()) { + Thread.sleep(50); + } + assertTrue(taskManagerServices.getJobManagerTable().contains(jobId)); + + taskSlotTable.allocateSlot(0, jobId, taskDeploymentDescriptor.getAllocationId(), Time.seconds(60)); + + TestInvokable.sync = new BlockerSync(); + + taskManager.submitTask(taskDeploymentDescriptor, jobMasterGateway.getFencingToken(), timeout) + .get(); + + TestInvokable.sync.awaitBlocker(); + + // the task is still running, so nothing was actually stored yet + assertTrue(jobAwareShuffleEnvironment.hasPartitionsOccupyingLocalResources(jobId)); + + TestInvokable.sync.releaseBlocker(); + taskFinishedFuture.get(timeout.getSize(), timeout.getUnit()); + + // the task is finished; the partition should be stored locally now + assertTrue(jobAwareShuffleEnvironment.hasPartitionsOccupyingLocalResources(jobId)); + + releaseAction.accept( + jobId, + taskResultPartitionDescriptor.getShuffleDescriptor().getResultPartitionID(), + taskManager); + postReleaseActionAssertion.accept(jobId, jobAwareShuffleEnvironment); + } finally { + RpcUtils.terminateRpcEndpoint(taskManager, timeout); + } + + // the partition table should always be cleaned up on shutdown + assertFalse(jobAwareShuffleEnvironment.hasPartitionsOccupyingLocalResources(jobId)); + } + + /** + * Test invokable which completes the given future when executed. + */ + public static class TestInvokable extends AbstractInvokable { + + static BlockerSync sync; + + public TestInvokable(Environment environment) { + super(environment); + } + + @Override + public void invoke() throws Exception { + sync.block(); + } + } + + private TestingTaskExecutor createTestingTaskExecutor(TaskManagerServices taskManagerServices) throws IOException { + return new TestingTaskExecutor( + rpc, + TaskManagerConfiguration.fromConfiguration(new Configuration()), + haServices, + taskManagerServices, + new HeartbeatServices(Long.MAX_VALUE, Long.MAX_VALUE), + UnregisteredMetricGroups.createUnregisteredTaskManagerMetricGroup(), + null, + new BlobCacheService( + new Configuration(), + new VoidBlobStore(), + null), + new TestingFatalErrorHandler()); Review comment: I think we should test at the end of the test that no fatal error has occurred. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected] With regards, Apache Git Services
