forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-21254] Add tests for Created state and State interface
This closes apache#14870
- Loading branch information
Showing
2 changed files
with
237 additions
and
0 deletions.
There are no files selected for viewing
135 changes: 135 additions & 0 deletions
135
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/declarative/CreatedTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,135 @@ | ||
/* | ||
* 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:https://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.runtime.executiongraph.ArchivedExecutionGraph; | ||
import org.apache.flink.util.TestLogger; | ||
|
||
import org.junit.Test; | ||
|
||
import javax.annotation.Nullable; | ||
|
||
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 Created} state. */ | ||
public class CreatedTest extends TestLogger { | ||
|
||
@Test | ||
public void testCancel() throws Exception { | ||
try (MockCreatedContext ctx = new MockCreatedContext()) { | ||
Created created = new Created(ctx, log); | ||
|
||
ctx.setExpectFinished(assertNonNull()); | ||
|
||
created.cancel(); | ||
} | ||
} | ||
|
||
@Test | ||
public void testStartScheduling() throws Exception { | ||
try (MockCreatedContext ctx = new MockCreatedContext()) { | ||
Created created = new Created(ctx, log); | ||
|
||
ctx.setExpectWaitingForResources(); | ||
|
||
created.startScheduling(); | ||
} | ||
} | ||
|
||
@Test | ||
public void testSuspend() throws Exception { | ||
try (MockCreatedContext ctx = new MockCreatedContext()) { | ||
Created created = new Created(ctx, log); | ||
|
||
ctx.setExpectFinished( | ||
archivedExecutionGraph -> { | ||
assertThat(archivedExecutionGraph.getState(), is(JobStatus.SUSPENDED)); | ||
}); | ||
|
||
created.suspend(new RuntimeException("Suspend")); | ||
} | ||
} | ||
|
||
@Test | ||
public void testFailure() throws Exception { | ||
try (MockCreatedContext ctx = new MockCreatedContext()) { | ||
Created created = new Created(ctx, log); | ||
|
||
ctx.setExpectFinished( | ||
archivedExecutionGraph -> { | ||
assertThat(archivedExecutionGraph.getState(), is(JobStatus.FAILED)); | ||
}); | ||
|
||
created.handleGlobalFailure(new RuntimeException("Global")); | ||
} | ||
} | ||
|
||
@Test | ||
public void testJobInformation() throws Exception { | ||
try (MockCreatedContext ctx = new MockCreatedContext()) { | ||
Created created = new Created(ctx, log); | ||
ArchivedExecutionGraph job = created.getJob(); | ||
assertThat(job.getState(), is(JobStatus.INITIALIZING)); | ||
} | ||
} | ||
|
||
static class MockCreatedContext implements Created.Context, AutoCloseable { | ||
private final StateValidator<ArchivedExecutionGraph> finishedStateValidator = | ||
new StateValidator<>("finished"); | ||
private final StateValidator<Void> waitingForResourcesStateValidator = | ||
new StateValidator<>("WaitingForResources"); | ||
|
||
public void setExpectFinished(Consumer<ArchivedExecutionGraph> asserter) { | ||
finishedStateValidator.expectInput(asserter); | ||
} | ||
|
||
public void setExpectWaitingForResources() { | ||
waitingForResourcesStateValidator.expectInput((none) -> {}); | ||
} | ||
|
||
@Override | ||
public void goToFinished(ArchivedExecutionGraph archivedExecutionGraph) { | ||
finishedStateValidator.validateInput(archivedExecutionGraph); | ||
} | ||
|
||
@Override | ||
public ArchivedExecutionGraph getArchivedExecutionGraph( | ||
JobStatus jobStatus, @Nullable Throwable cause) { | ||
return ArchivedExecutionGraph.createFromInitializingJob( | ||
new JobID(), "testJob", jobStatus, cause, 0L); | ||
} | ||
|
||
@Override | ||
public void goToWaitingForResources() { | ||
waitingForResourcesStateValidator.validateInput(null); | ||
} | ||
|
||
@Override | ||
public void close() throws Exception { | ||
finishedStateValidator.close(); | ||
waitingForResourcesStateValidator.close(); | ||
} | ||
} | ||
} |
102 changes: 102 additions & 0 deletions
102
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/declarative/StateTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,102 @@ | ||
/* | ||
* 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:https://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.util.TestLogger; | ||
|
||
import org.junit.Test; | ||
|
||
import java.util.Optional; | ||
import java.util.concurrent.atomic.AtomicBoolean; | ||
|
||
import static org.hamcrest.CoreMatchers.is; | ||
import static org.junit.Assert.assertThat; | ||
import static org.junit.Assert.fail; | ||
|
||
/** | ||
* Tests for the default methods on the {@link State} interface, based on the {@link Created} state, | ||
* as it is a simple state. | ||
*/ | ||
public class StateTest extends TestLogger { | ||
@Test | ||
public void testEmptyAs() throws Exception { | ||
try (CreatedTest.MockCreatedContext ctx = new CreatedTest.MockCreatedContext()) { | ||
State state = new Created(ctx, log); | ||
assertThat(state.as(WaitingForResources.class), is(Optional.empty())); | ||
} | ||
} | ||
|
||
@Test | ||
public void testCast() throws Exception { | ||
try (CreatedTest.MockCreatedContext ctx = new CreatedTest.MockCreatedContext()) { | ||
State state = new Created(ctx, log); | ||
assertThat(state.as(Created.class), is(Optional.of(state))); | ||
} | ||
} | ||
|
||
@Test | ||
public void testTryRunStateMismatch() throws Exception { | ||
try (CreatedTest.MockCreatedContext ctx = new CreatedTest.MockCreatedContext()) { | ||
State state = new Created(ctx, log); | ||
state.tryRun( | ||
WaitingForResources.class, (waiting -> fail("Unexpected execution")), "test"); | ||
} | ||
} | ||
|
||
@Test | ||
public void testTryRun() throws Exception { | ||
try (CreatedTest.MockCreatedContext ctx = new CreatedTest.MockCreatedContext()) { | ||
State state = new Created(ctx, log); | ||
AtomicBoolean called = new AtomicBoolean(false); | ||
state.tryRun(Created.class, created -> called.set(true), "test"); | ||
assertThat(called.get(), is(true)); | ||
} | ||
} | ||
|
||
@Test | ||
public void testTryCallStateMismatch() throws Exception { | ||
try (CreatedTest.MockCreatedContext ctx = new CreatedTest.MockCreatedContext()) { | ||
State state = new Created(ctx, log); | ||
Optional<String> result = | ||
state.tryCall( | ||
WaitingForResources.class, | ||
Waiting -> { | ||
fail("Unexpected execution"); | ||
return "nope"; | ||
}, | ||
"test"); | ||
assertThat(result, is(Optional.empty())); | ||
} | ||
} | ||
|
||
@Test | ||
public void testTryCall() throws Exception { | ||
try (CreatedTest.MockCreatedContext ctx = new CreatedTest.MockCreatedContext()) { | ||
State state = new Created(ctx, log); | ||
Optional<String> result = | ||
state.tryCall( | ||
Created.class, | ||
created -> { | ||
return "yes"; | ||
}, | ||
"test"); | ||
assertThat(result, is(Optional.of("yes"))); | ||
} | ||
} | ||
} |