zhuzhurk commented on a change in pull request #16314: URL: https://github.com/apache/flink/pull/16314#discussion_r675531633
########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorFactoryTest.java ########## @@ -0,0 +1,398 @@ +/* + * 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.deployment; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.JobException; +import org.apache.flink.runtime.blob.BlobWriter; +import org.apache.flink.runtime.blob.VoidBlobWriter; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter; +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.Execution; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.executiongraph.ExecutionGraph; +import org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils; +import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; +import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.executiongraph.IntermediateResult; +import org.apache.flink.runtime.executiongraph.TaskExecutionStateTransition; +import org.apache.flink.runtime.executiongraph.TestingDefaultExecutionGraphBuilder; +import org.apache.flink.runtime.executiongraph.failover.flip1.TestRestartBackoffTimeStrategy; +import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.io.network.partition.ResultPartitionType; +import org.apache.flink.runtime.jobgraph.DistributionPattern; +import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobGraphBuilder; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.jobmaster.LogicalSlot; +import org.apache.flink.runtime.jobmaster.TestingLogicalSlotBuilder; +import org.apache.flink.runtime.scheduler.DefaultScheduler; +import org.apache.flink.runtime.scheduler.SchedulerTestingUtils; +import org.apache.flink.runtime.shuffle.ShuffleDescriptor; +import org.apache.flink.runtime.taskmanager.TaskExecutionState; +import org.apache.flink.util.SerializedValue; +import org.apache.flink.util.TestLogger; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeoutException; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +/** Tests for {@link TaskDeploymentDescriptorFactory}. */ +public class TaskDeploymentDescriptorFactoryTest extends TestLogger { + + private static final int PARALLELISM = 4; + + private ScheduledExecutorService scheduledExecutorService; + + private ComponentMainThreadExecutor mainThreadExecutor; + + @Before + public void setup() { + + scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(); + + mainThreadExecutor = + ComponentMainThreadExecutorServiceAdapter.forSingleThreadExecutor( + scheduledExecutorService); + } + + @After + public void teardown() { + if (scheduledExecutorService != null) { + scheduledExecutorService.shutdownNow(); + } + } + + @Test + public void testCacheShuffleDescriptor() throws Exception { + final ExecutionJobVertex ejv = + setupExecutionGraphAndGetVertex(new JobID(), new VoidBlobWriter()); + + final IntermediateResult consumedResult = ejv.getInputs().get(0); + + final ExecutionVertex ev21 = ejv.getTaskVertices()[0]; + final InputGateDeploymentDescriptor igdd21 = createTaskDeploymentDescriptorAndGet(ev21); + final ShuffleDescriptor[] shuffleDescriptors = igdd21.getShuffleDescriptors(); + + // ShuffleDescriptors should be cached + final SerializedValue<ShuffleDescriptor[]> compressedSerializedValue = + consumedResult.getCachedShuffleDescriptors(ev21.getConsumedPartitionGroup(0)); + + final ShuffleDescriptor[] cachedShuffleDescriptors = + deserializeShuffleDescriptors(compressedSerializedValue); + + assertEquals(shuffleDescriptors.length, cachedShuffleDescriptors.length); + for (int i = 0; i < cachedShuffleDescriptors.length; i++) { + assertEquals( + shuffleDescriptors[i].getResultPartitionID(), + cachedShuffleDescriptors[i].getResultPartitionID()); + } + + // The second TaskDeploymentDescriptor should uses cached ShuffleDescriptors + final ExecutionVertex ev22 = ejv.getTaskVertices()[1]; + final InputGateDeploymentDescriptor igdd22 = createTaskDeploymentDescriptorAndGet(ev22); + final ShuffleDescriptor[] otherShuffleDescriptors = igdd22.getShuffleDescriptors(); + + assertEquals(shuffleDescriptors.length, otherShuffleDescriptors.length); + for (int i = 0; i < shuffleDescriptors.length; i++) { + assertEquals( + shuffleDescriptors[i].getResultPartitionID(), + otherShuffleDescriptors[i].getResultPartitionID()); + } + } + + private ExecutionJobVertex setupExecutionGraphAndGetVertex(JobID jobId, BlobWriter blobWriter) + throws JobException, JobExecutionException { + final JobVertex v1 = createJobVertex("v1", PARALLELISM); + final JobVertex v2 = createJobVertex("v2", PARALLELISM); + + v2.connectNewDataSetAsInput( + v1, DistributionPattern.ALL_TO_ALL, ResultPartitionType.BLOCKING); + + final List<JobVertex> ordered = new ArrayList<>(Arrays.asList(v1, v2)); + + final ExecutionGraph executionGraph = + createExecutionGraphDirectly(jobId, ordered, blobWriter); + + return executionGraph.getJobVertex(v2.getID()); + } + + @Test + public void testRemoveShuffleDescriptorCacheAfterFinished() throws Exception { Review comment: This case should not be in `TaskDeploymentDescriptorFactoryTest`. ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorFactoryTest.java ########## @@ -0,0 +1,398 @@ +/* + * 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.deployment; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.JobException; +import org.apache.flink.runtime.blob.BlobWriter; +import org.apache.flink.runtime.blob.VoidBlobWriter; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter; +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.Execution; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.executiongraph.ExecutionGraph; +import org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils; +import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; +import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.executiongraph.IntermediateResult; +import org.apache.flink.runtime.executiongraph.TaskExecutionStateTransition; +import org.apache.flink.runtime.executiongraph.TestingDefaultExecutionGraphBuilder; +import org.apache.flink.runtime.executiongraph.failover.flip1.TestRestartBackoffTimeStrategy; +import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.io.network.partition.ResultPartitionType; +import org.apache.flink.runtime.jobgraph.DistributionPattern; +import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobGraphBuilder; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.jobmaster.LogicalSlot; +import org.apache.flink.runtime.jobmaster.TestingLogicalSlotBuilder; +import org.apache.flink.runtime.scheduler.DefaultScheduler; +import org.apache.flink.runtime.scheduler.SchedulerTestingUtils; +import org.apache.flink.runtime.shuffle.ShuffleDescriptor; +import org.apache.flink.runtime.taskmanager.TaskExecutionState; +import org.apache.flink.util.SerializedValue; +import org.apache.flink.util.TestLogger; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeoutException; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +/** Tests for {@link TaskDeploymentDescriptorFactory}. */ +public class TaskDeploymentDescriptorFactoryTest extends TestLogger { + + private static final int PARALLELISM = 4; + + private ScheduledExecutorService scheduledExecutorService; + + private ComponentMainThreadExecutor mainThreadExecutor; + + @Before + public void setup() { + + scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(); + + mainThreadExecutor = + ComponentMainThreadExecutorServiceAdapter.forSingleThreadExecutor( + scheduledExecutorService); + } + + @After + public void teardown() { + if (scheduledExecutorService != null) { + scheduledExecutorService.shutdownNow(); + } + } + + @Test + public void testCacheShuffleDescriptor() throws Exception { + final ExecutionJobVertex ejv = + setupExecutionGraphAndGetVertex(new JobID(), new VoidBlobWriter()); + + final IntermediateResult consumedResult = ejv.getInputs().get(0); + + final ExecutionVertex ev21 = ejv.getTaskVertices()[0]; + final InputGateDeploymentDescriptor igdd21 = createTaskDeploymentDescriptorAndGet(ev21); + final ShuffleDescriptor[] shuffleDescriptors = igdd21.getShuffleDescriptors(); + + // ShuffleDescriptors should be cached + final SerializedValue<ShuffleDescriptor[]> compressedSerializedValue = + consumedResult.getCachedShuffleDescriptors(ev21.getConsumedPartitionGroup(0)); + + final ShuffleDescriptor[] cachedShuffleDescriptors = + deserializeShuffleDescriptors(compressedSerializedValue); + + assertEquals(shuffleDescriptors.length, cachedShuffleDescriptors.length); + for (int i = 0; i < cachedShuffleDescriptors.length; i++) { + assertEquals( + shuffleDescriptors[i].getResultPartitionID(), + cachedShuffleDescriptors[i].getResultPartitionID()); + } + + // The second TaskDeploymentDescriptor should uses cached ShuffleDescriptors + final ExecutionVertex ev22 = ejv.getTaskVertices()[1]; + final InputGateDeploymentDescriptor igdd22 = createTaskDeploymentDescriptorAndGet(ev22); + final ShuffleDescriptor[] otherShuffleDescriptors = igdd22.getShuffleDescriptors(); + + assertEquals(shuffleDescriptors.length, otherShuffleDescriptors.length); + for (int i = 0; i < shuffleDescriptors.length; i++) { + assertEquals( + shuffleDescriptors[i].getResultPartitionID(), + otherShuffleDescriptors[i].getResultPartitionID()); + } + } + + private ExecutionJobVertex setupExecutionGraphAndGetVertex(JobID jobId, BlobWriter blobWriter) + throws JobException, JobExecutionException { + final JobVertex v1 = createJobVertex("v1", PARALLELISM); + final JobVertex v2 = createJobVertex("v2", PARALLELISM); + + v2.connectNewDataSetAsInput( + v1, DistributionPattern.ALL_TO_ALL, ResultPartitionType.BLOCKING); + + final List<JobVertex> ordered = new ArrayList<>(Arrays.asList(v1, v2)); + + final ExecutionGraph executionGraph = + createExecutionGraphDirectly(jobId, ordered, blobWriter); + + return executionGraph.getJobVertex(v2.getID()); + } + + @Test + public void testRemoveShuffleDescriptorCacheAfterFinished() throws Exception { + + final JobVertex v1 = createJobVertex("v1", PARALLELISM); + final JobVertex v2 = createJobVertex("v2", PARALLELISM); + + final ExecutionGraph executionGraph = createExecutionGraphAndDeploy(v1, v2); + + // ShuffleDescriptors should be cached during the deployment + final ShuffleDescriptor[] shuffleDescriptors = + deserializeShuffleDescriptors(getCachedShuffleDescriptor(executionGraph, v2)); + assertEquals(PARALLELISM, shuffleDescriptors.length); + + CompletableFuture.runAsync( + () -> transitionTasksToFinished(executionGraph, v2.getID()), + mainThreadExecutor) + .join(); + + // Cache should be removed when partitions are released + assertNull(getCachedShuffleDescriptor(executionGraph, v2)); + } + + @Test + public void testCacheRemovedCorrectlyAfterFailover() throws Exception { Review comment: This case should not be in `TaskDeploymentDescriptorFactoryTest`. -- 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]
