tillrohrmann commented on a change in pull request #14879: URL: https://github.com/apache/flink/pull/14879#discussion_r573607376
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/declarative/StateWithExecutionGraph.java ########## @@ -0,0 +1,348 @@ +/* + * 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.declarative; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.configuration.CheckpointingOptions; +import org.apache.flink.queryablestate.KvStateID; +import org.apache.flink.runtime.accumulators.AccumulatorSnapshot; +import org.apache.flink.runtime.checkpoint.CheckpointCoordinator; +import org.apache.flink.runtime.checkpoint.CheckpointMetrics; +import org.apache.flink.runtime.checkpoint.CompletedCheckpoint; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.executiongraph.ExecutionGraph; +import org.apache.flink.runtime.executiongraph.TaskExecutionStateTransition; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.jobmanager.PartitionProducerDisposedException; +import org.apache.flink.runtime.jobmaster.SerializedInputSplit; +import org.apache.flink.runtime.messages.FlinkJobNotFoundException; +import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint; +import org.apache.flink.runtime.operators.coordination.CoordinationRequest; +import org.apache.flink.runtime.operators.coordination.CoordinationResponse; +import org.apache.flink.runtime.operators.coordination.OperatorEvent; +import org.apache.flink.runtime.query.KvStateLocation; +import org.apache.flink.runtime.query.UnknownKvStateLocation; +import org.apache.flink.runtime.scheduler.ExecutionGraphHandler; +import org.apache.flink.runtime.scheduler.KvStateHandler; +import org.apache.flink.runtime.scheduler.OperatorCoordinatorHandler; +import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.Executor; + +/** + * Abstract state class which contains an {@link ExecutionGraph} and the required handlers to + * execute common operations. + */ +abstract class StateWithExecutionGraph implements State { + private final Context context; + + private final ExecutionGraph executionGraph; + + private final ExecutionGraphHandler executionGraphHandler; + + private final OperatorCoordinatorHandler operatorCoordinatorHandler; + + private final KvStateHandler kvStateHandler; + + private final Logger logger; + + StateWithExecutionGraph( + Context context, + ExecutionGraph executionGraph, + ExecutionGraphHandler executionGraphHandler, + OperatorCoordinatorHandler operatorCoordinatorHandler, + Logger logger) { + this.context = context; + this.executionGraph = executionGraph; + this.executionGraphHandler = executionGraphHandler; + this.operatorCoordinatorHandler = operatorCoordinatorHandler; + this.kvStateHandler = new KvStateHandler(executionGraph); + this.logger = logger; + Preconditions.checkState( + executionGraph.getState() == JobStatus.RUNNING, "Assuming running execution graph"); + + FutureUtils.assertNoException( + executionGraph + .getTerminationFuture() + .thenAcceptAsync( + jobStatus -> { + if (jobStatus.isTerminalState()) { Review comment: I think we need to check whether `jobStatus.isGloballyTerminalState()`. I guess we are lacking test coverage to make sure of this. ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/declarative/MockStateWithExecutionGraphContext.java ########## @@ -0,0 +1,80 @@ +/* + * 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.declarative; + +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor; +import org.apache.flink.runtime.concurrent.ManuallyTriggeredComponentMainThreadExecutor; +import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; + +import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; + +import static org.hamcrest.CoreMatchers.is; +import static org.junit.Assert.assertThat; + +class MockStateWithExecutionGraphContext implements StateWithExecutionGraph.Context, AutoCloseable { + + private final StateValidator<ArchivedExecutionGraph> finishedStateValidator = + new StateValidator<>("Finished"); + + private final ManuallyTriggeredComponentMainThreadExecutor executor = + new ManuallyTriggeredComponentMainThreadExecutor(); + + protected boolean hadStateTransition = false; + + public void setExpectFinished(Consumer<ArchivedExecutionGraph> asserter) { + finishedStateValidator.expectInput(asserter); + } + + @Override + public void runIfState(State expectedState, Runnable action) { + if (!hadStateTransition) { + action.run(); + } + } + + @Override + public boolean isState(State expectedState) { + throw new UnsupportedOperationException("Not covered by this test at the moment"); + } + + @Override + public void goToFinished(ArchivedExecutionGraph archivedExecutionGraph) { + finishedStateValidator.validateInput(archivedExecutionGraph); + hadStateTransition = true; + } + + @Override + public ComponentMainThreadExecutor getMainThreadExecutor() { + return executor; + } + + @Override + public void close() throws Exception { + // trigger executor to make sure there are no outstanding state transitions + executor.triggerAll(); + executor.shutdown(); + executor.awaitTermination(10, TimeUnit.MINUTES); + finishedStateValidator.close(); + } + + protected void assertNoStateTransition() { Review comment: Can this be `final`? ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/concurrent/ManuallyTriggeredComponentMainThreadExecutor.java ########## @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.concurrent; + +/** Testing ComponentMainThreadExecutor which can be manually triggered. */ +public class ManuallyTriggeredComponentMainThreadExecutor + extends ManuallyTriggeredScheduledExecutorService implements ComponentMainThreadExecutor { + + private Thread executorThread = null; Review comment: I meant to set this field when `ManuallyTriggeredComponentMainThreadExecutor` is being created. That way no random thread which calls `assertRunningInMainThread` first will be considered as the main thread. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/declarative/StateWithExecutionGraph.java ########## @@ -0,0 +1,348 @@ +/* + * 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.declarative; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.configuration.CheckpointingOptions; +import org.apache.flink.queryablestate.KvStateID; +import org.apache.flink.runtime.accumulators.AccumulatorSnapshot; +import org.apache.flink.runtime.checkpoint.CheckpointCoordinator; +import org.apache.flink.runtime.checkpoint.CheckpointMetrics; +import org.apache.flink.runtime.checkpoint.CompletedCheckpoint; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.executiongraph.ExecutionGraph; +import org.apache.flink.runtime.executiongraph.TaskExecutionStateTransition; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.jobmanager.PartitionProducerDisposedException; +import org.apache.flink.runtime.jobmaster.SerializedInputSplit; +import org.apache.flink.runtime.messages.FlinkJobNotFoundException; +import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint; +import org.apache.flink.runtime.operators.coordination.CoordinationRequest; +import org.apache.flink.runtime.operators.coordination.CoordinationResponse; +import org.apache.flink.runtime.operators.coordination.OperatorEvent; +import org.apache.flink.runtime.query.KvStateLocation; +import org.apache.flink.runtime.query.UnknownKvStateLocation; +import org.apache.flink.runtime.scheduler.ExecutionGraphHandler; +import org.apache.flink.runtime.scheduler.KvStateHandler; +import org.apache.flink.runtime.scheduler.OperatorCoordinatorHandler; +import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.Executor; + +/** + * Abstract state class which contains an {@link ExecutionGraph} and the required handlers to + * execute common operations. + */ +abstract class StateWithExecutionGraph implements State { + private final Context context; + + private final ExecutionGraph executionGraph; + + private final ExecutionGraphHandler executionGraphHandler; + + private final OperatorCoordinatorHandler operatorCoordinatorHandler; + + private final KvStateHandler kvStateHandler; + + private final Logger logger; + + StateWithExecutionGraph( + Context context, + ExecutionGraph executionGraph, + ExecutionGraphHandler executionGraphHandler, + OperatorCoordinatorHandler operatorCoordinatorHandler, + Logger logger) { + this.context = context; + this.executionGraph = executionGraph; + this.executionGraphHandler = executionGraphHandler; + this.operatorCoordinatorHandler = operatorCoordinatorHandler; + this.kvStateHandler = new KvStateHandler(executionGraph); + this.logger = logger; + Preconditions.checkState( + executionGraph.getState() == JobStatus.RUNNING, "Assuming running execution graph"); + + FutureUtils.assertNoException( + executionGraph + .getTerminationFuture() + .thenAcceptAsync( + jobStatus -> { + if (jobStatus.isTerminalState()) { + context.runIfState(this, () -> onTerminalState(jobStatus)); Review comment: Maybe also rename `onTerminalState` into `onGloballyTerminalState`. ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/declarative/RestartingTest.java ########## @@ -0,0 +1,223 @@ +/* + * 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.declarative; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.JobException; +import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.blob.VoidBlobWriter; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.runtime.executiongraph.ExecutionGraph; +import org.apache.flink.runtime.executiongraph.JobInformation; +import org.apache.flink.runtime.executiongraph.NoOpExecutionDeploymentListener; +import org.apache.flink.runtime.executiongraph.TestingExecutionGraphBuilder; +import org.apache.flink.runtime.executiongraph.failover.flip1.partitionrelease.PartitionReleaseStrategyFactoryLoader; +import org.apache.flink.runtime.io.network.partition.NoOpJobMasterPartitionTracker; +import org.apache.flink.runtime.jobgraph.ScheduleMode; +import org.apache.flink.runtime.scheduler.ExecutionGraphHandler; +import org.apache.flink.runtime.scheduler.OperatorCoordinatorHandler; +import org.apache.flink.runtime.shuffle.NettyShuffleMaster; +import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.apache.flink.util.SerializedValue; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +import java.io.IOException; +import java.time.Duration; +import java.util.Collections; +import java.util.function.Consumer; + +import static org.apache.flink.runtime.scheduler.declarative.WaitingForResourcesTest.assertNonNull; +import static org.hamcrest.CoreMatchers.is; +import static org.junit.Assert.assertThat; + +/** Tests for the {@link Restarting} state of the declarative scheduler. */ +public class RestartingTest extends TestLogger { + + @Test + public void testExecutionGraphCancellationOnEnter() throws Exception { + try (MockRestartingContext ctx = new MockRestartingContext()) { + CancellableExecutionGraph cancellableExecutionGraph = new CancellableExecutionGraph(); + Restarting restarting = createRestartingState(ctx, cancellableExecutionGraph); + + restarting.onEnter(); + assertThat(cancellableExecutionGraph.isCancelled(), is(true)); + } + } + + @Test + public void testTransitionToWaitingForResourcesWhenCancellationComplete() throws Exception { + try (MockRestartingContext ctx = new MockRestartingContext()) { + Restarting restarting = createRestartingState(ctx); + ctx.setExpectWaitingForResources(); + restarting.onTerminalState(JobStatus.CANCELED); + } + } + + @Test + public void testCancel() throws Exception { + try (MockRestartingContext ctx = new MockRestartingContext()) { + Restarting restarting = createRestartingState(ctx); + ctx.setExpectCancelling(assertNonNull()); + restarting.cancel(); + } + } + + @Test + public void testSuspend() throws Exception { + try (MockRestartingContext ctx = new MockRestartingContext()) { + Restarting restarting = createRestartingState(ctx); + ctx.setExpectFinished( + archivedExecutionGraph -> + assertThat(archivedExecutionGraph.getState(), is(JobStatus.SUSPENDED))); + final Throwable cause = new RuntimeException("suspend"); + restarting.suspend(cause); + } + } + + @Test + public void testGlobalFailuresAreIgnored() throws Exception { + try (MockRestartingContext ctx = new MockRestartingContext()) { + Restarting restarting = createRestartingState(ctx); + restarting.handleGlobalFailure(new RuntimeException()); + ctx.assertNoStateTransition(); + } + } + + public Restarting createRestartingState( + MockRestartingContext ctx, ExecutionGraph executionGraph) { + final ExecutionGraphHandler executionGraphHandler = + new ExecutionGraphHandler( + executionGraph, + log, + ctx.getMainThreadExecutor(), + ctx.getMainThreadExecutor()); + final OperatorCoordinatorHandler operatorCoordinatorHandler = + new OperatorCoordinatorHandler( + executionGraph, + (throwable) -> { + throw new RuntimeException("Error in test", throwable); + }); + executionGraph.transitionToRunning(); + return new Restarting( + ctx, + executionGraph, + executionGraphHandler, + operatorCoordinatorHandler, + log, + Duration.ZERO); + } + + public Restarting createRestartingState(MockRestartingContext ctx) + throws JobException, JobExecutionException { + ExecutionGraph executionGraph = TestingExecutionGraphBuilder.newBuilder().build(); + return createRestartingState(ctx, executionGraph); + } + + private static class MockRestartingContext extends MockStateWithExecutionGraphContext + implements Restarting.Context { + + private final StateValidator<ExecutingTest.CancellingArguments> cancellingStateValidator = + new StateValidator<>("Cancelling"); + + private final StateValidator<Void> waitingForResourcesStateValidator = + new StateValidator<>("WaitingForResources"); + + public void setExpectCancelling(Consumer<ExecutingTest.CancellingArguments> asserter) { + cancellingStateValidator.expectInput(asserter); + } + + public void setExpectWaitingForResources() { + waitingForResourcesStateValidator.expectInput((none) -> {}); + } + + @Override + public void goToCanceling( + ExecutionGraph executionGraph, + ExecutionGraphHandler executionGraphHandler, + OperatorCoordinatorHandler operatorCoordinatorHandler) { + cancellingStateValidator.validateInput( + new ExecutingTest.CancellingArguments( + executionGraph, executionGraphHandler, operatorCoordinatorHandler)); + hadStateTransition = true; + } + + @Override + public void goToWaitingForResources() { + waitingForResourcesStateValidator.validateInput(null); + hadStateTransition = true; + } + + @Override + public void runIfState(State expectedState, Runnable action, Duration delay) { + if (!hadStateTransition) { + action.run(); + } + } + + @Override + public void close() throws Exception { + super.close(); + cancellingStateValidator.close(); + waitingForResourcesStateValidator.close(); + } + } + + private static class CancellableExecutionGraph extends ExecutionGraph { + private boolean cancelled = false; + + CancellableExecutionGraph() throws IOException { + super( + new JobInformation( + new JobID(), + "Test Job", + new SerializedValue<>(new ExecutionConfig()), + new Configuration(), + Collections.emptyList(), + Collections.emptyList()), + TestingUtils.defaultExecutor(), + TestingUtils.defaultExecutor(), + AkkaUtils.getDefaultTimeout(), + 1, + ExecutionGraph.class.getClassLoader(), + VoidBlobWriter.getInstance(), + PartitionReleaseStrategyFactoryLoader.loadPartitionReleaseStrategyFactory( + new Configuration()), + NettyShuffleMaster.INSTANCE, + NoOpJobMasterPartitionTracker.INSTANCE, + ScheduleMode.EAGER, + NoOpExecutionDeploymentListener.get(), + (execution, newState) -> {}, + 0L); + } + + @Override + public void cancel() { + cancelled = true; + } + + public boolean isCancelled() { + return cancelled; + } + } Review comment: As a follow up we might wanna extract an interface from the `ExecutionGraph` to make it better testable. Now with every `ExecutionGraph` subclass we need to touch another place whenever we change something on the `ExecutionGraph` constructor. ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/declarative/ExecutingTest.java ########## @@ -0,0 +1,611 @@ +/* + * 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.declarative; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.JobException; +import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.blob.VoidBlobWriter; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.executiongraph.ExecutionGraph; +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.JobInformation; +import org.apache.flink.runtime.executiongraph.NoOpExecutionDeploymentListener; +import org.apache.flink.runtime.executiongraph.TaskExecutionStateTransition; +import org.apache.flink.runtime.executiongraph.TestingExecutionGraphBuilder; +import org.apache.flink.runtime.executiongraph.failover.flip1.partitionrelease.PartitionReleaseStrategyFactoryLoader; +import org.apache.flink.runtime.io.network.partition.NoOpJobMasterPartitionTracker; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobgraph.ScheduleMode; +import org.apache.flink.runtime.scheduler.ExecutionGraphHandler; +import org.apache.flink.runtime.scheduler.OperatorCoordinatorHandler; +import org.apache.flink.runtime.shuffle.NettyShuffleMaster; +import org.apache.flink.runtime.taskmanager.TaskExecutionState; +import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.apache.flink.util.SerializedValue; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; +import org.slf4j.Logger; + +import java.io.IOException; +import java.time.Duration; +import java.util.Collections; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.function.Supplier; + +import static org.apache.flink.runtime.scheduler.declarative.WaitingForResourcesTest.assertNonNull; +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.junit.Assert.assertThat; + +/** Tests for declarative scheduler's {@link Executing} state. */ +public class ExecutingTest extends TestLogger { Review comment: I like the tests. They seem to be very targeted and well to read :-) ---------------------------------------------------------------- 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]
