Skip to content

Commit

Permalink
[hotfix][batch] Fix incorrectly use of ExecutionMode in AdaptiveBatch…
Browse files Browse the repository at this point in the history
…SchedulerFactory
  • Loading branch information
reswqa committed Sep 27, 2024
1 parent 6a290ca commit 2f888ba
Showing 1 changed file with 3 additions and 3 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.common.BatchShuffleMode;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.ExecutionMode;
import org.apache.flink.api.common.RuntimeExecutionMode;
import org.apache.flink.configuration.BatchExecutionOptions;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.ExecutionOptions;
Expand Down Expand Up @@ -367,12 +367,12 @@ private static void checkAllExchangesAreSupported(final JobGraph jobGraph) {
+ "to be executed with types of all edges being BLOCKING or HYBRID_FULL/HYBRID_SELECTIVE. "
+ "To do that, you need to configure '%s' to '%s' or '%s/%s'. "
+ "Note that for DataSet jobs which do not recognize the aforementioned shuffle mode, "
+ "the ExecutionMode needs to be %s to force BLOCKING shuffle",
+ "the RuntimeExecutionMode needs to be %s to force BLOCKING shuffle",
ExecutionOptions.BATCH_SHUFFLE_MODE.key(),
BatchShuffleMode.ALL_EXCHANGES_BLOCKING,
BatchShuffleMode.ALL_EXCHANGES_HYBRID_FULL,
BatchShuffleMode.ALL_EXCHANGES_HYBRID_SELECTIVE,
ExecutionMode.BATCH_FORCED));
RuntimeExecutionMode.BATCH));
}
}
}
Expand Down

0 comments on commit 2f888ba

Please sign in to comment.