Skip to content

Commit

Permalink
[FLINK-21535][test] Adds proper comment to test methods that are disa…
Browse files Browse the repository at this point in the history
…bled for the AdaptiveScheduler
  • Loading branch information
XComp committed Mar 25, 2024
1 parent 8f06fb4 commit 9614240
Show file tree
Hide file tree
Showing 2 changed files with 10 additions and 5 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -38,12 +38,10 @@
import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
import org.apache.flink.testutils.junit.FailsWithAdaptiveScheduler;
import org.apache.flink.util.Collector;

import org.apache.commons.lang3.ArrayUtils;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;

Expand Down Expand Up @@ -105,7 +103,6 @@
* </ul>
*/
@RunWith(Parameterized.class)
@Category(FailsWithAdaptiveScheduler.class) // FLINK-21689
public class UnalignedCheckpointITCase extends UnalignedCheckpointTestBase {
enum Topology implements DagCreator {
PIPELINE {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -104,8 +104,16 @@
import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.CHECKPOINTING_TIMEOUT;
import static org.apache.flink.util.Preconditions.checkState;

/** Base class for tests related to unaligned checkpoints. */
@Category(FailsWithAdaptiveScheduler.class) // FLINK-21689
/**
* Base class for tests related to unaligned checkpoints.
*
* <p>This test base relies on restarting the subtasks within the scheduler to trigger a reset of
* the operators. The operator reset is counted in the LongSource. The job will terminate if the
* number of expected restarts is reached. The AdaptiveScheduler won't trigger the operator reset
* resulting in the test running forever. This is why this test suite is disabled for the {@link
* org.apache.flink.runtime.scheduler.adaptive.AdaptiveScheduler}.
*/
@Category(FailsWithAdaptiveScheduler.class)
public abstract class UnalignedCheckpointTestBase extends TestLogger {
protected static final Logger LOG = LoggerFactory.getLogger(UnalignedCheckpointTestBase.class);
protected static final String NUM_INPUTS = "inputs_";
Expand Down

0 comments on commit 9614240

Please sign in to comment.