StephanEwen commented on a change in pull request #10483: [FLINK-15099][runtime] (FLIP-27) Add Operator Coordinators and Events URL: https://github.com/apache/flink/pull/10483#discussion_r355182595
########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/OperatorCoordinatorSchedulerTest.java ########## @@ -0,0 +1,226 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.scheduler; + +import org.apache.flink.runtime.checkpoint.CompletedCheckpoint; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter; +import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutorService; +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; +import org.apache.flink.runtime.operators.coordination.TestingOperatorCoordinator; +import org.apache.flink.runtime.taskmanager.TaskExecutionState; +import org.apache.flink.runtime.testtasks.NoOpInvokable; +import org.apache.flink.util.SerializedValue; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +import static org.apache.flink.runtime.scheduler.SchedulerTestingUtils.takeCheckpoint; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.not; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + + +/** + * Tests for the integration of the {@link OperatorCoordinator} with the scheduler, to ensure the relevant + * actions are leading to the right method invocations on the coordinator. + */ +public class OperatorCoordinatorSchedulerTest extends TestLogger { + + private final JobVertexID testVertexId = new JobVertexID(); + private final OperatorID testOperatorId = new OperatorID(); + + private final ManuallyTriggeredScheduledExecutorService executor = new ManuallyTriggeredScheduledExecutorService(); + + // ------------------------------------------------------------------------ + // tests + // ------------------------------------------------------------------------ + + @Test + public void testCoordinatorStartedWhenSchedulerStarts() throws Exception { + final DefaultScheduler scheduler = createAndStartScheduler(); + final TestingOperatorCoordinator coordinator = getCoordinator(scheduler); + + assertTrue(coordinator.isStarted()); + } + + @Test + public void testCoordinatorDisposedWhenSchedulerStops() throws Exception { + final DefaultScheduler scheduler = createAndStartScheduler(); + final TestingOperatorCoordinator coordinator = getCoordinator(scheduler); + + scheduler.suspend(new Exception("test suspend")); + + assertTrue(coordinator.isClosed()); + } + + @Test + public void testFailureToStartPropagatesExceptions() throws Exception { + final OperatorCoordinator.Provider failingCoordinatorProvider = + new TestingOperatorCoordinator.Provider(testOperatorId, CoordinatorThatFailsInStart::new); + final DefaultScheduler scheduler = createScheduler(failingCoordinatorProvider); + + try { Review comment: I have made some bad experience with that pattern where the exception was thrown in a different place (broken mick instantiation), which made the test pass but useless. This pattern is more safe in precise, in my opinion. ---------------------------------------------------------------- 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
