NicoK closed pull request #7033: [FLINK-10419][checkpoint] fix
ClassNotFoundException while deserializing user exceptions
URL: https://github.com/apache/flink/pull/7033
This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:
As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):
diff --git
a/flink-runtime/src/main/java/org/apache/flink/runtime/messages/checkpoint/DeclineCheckpoint.java
b/flink-runtime/src/main/java/org/apache/flink/runtime/messages/checkpoint/DeclineCheckpoint.java
index 7b0b55c9a1b..f8f3c0117bd 100644
---
a/flink-runtime/src/main/java/org/apache/flink/runtime/messages/checkpoint/DeclineCheckpoint.java
+++
b/flink-runtime/src/main/java/org/apache/flink/runtime/messages/checkpoint/DeclineCheckpoint.java
@@ -19,12 +19,6 @@
package org.apache.flink.runtime.messages.checkpoint;
import org.apache.flink.api.common.JobID;
-import
org.apache.flink.runtime.checkpoint.decline.AlignmentLimitExceededException;
-import
org.apache.flink.runtime.checkpoint.decline.CheckpointDeclineOnCancellationBarrierException;
-import
org.apache.flink.runtime.checkpoint.decline.CheckpointDeclineSubsumedException;
-import
org.apache.flink.runtime.checkpoint.decline.CheckpointDeclineTaskNotCheckpointingException;
-import
org.apache.flink.runtime.checkpoint.decline.CheckpointDeclineTaskNotReadyException;
-import org.apache.flink.runtime.checkpoint.decline.InputEndOfStreamException;
import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
import org.apache.flink.util.SerializedThrowable;
@@ -38,7 +32,7 @@
private static final long serialVersionUID = 2094094662279578953L;
- /** The reason why the checkpoint was declined */
+ /** The reason why the checkpoint was declined. */
private final Throwable reason;
public DeclineCheckpoint(JobID job, ExecutionAttemptID taskExecutionId,
long checkpointId) {
@@ -47,19 +41,12 @@ public DeclineCheckpoint(JobID job, ExecutionAttemptID
taskExecutionId, long che
public DeclineCheckpoint(JobID job, ExecutionAttemptID taskExecutionId,
long checkpointId, Throwable reason) {
super(job, taskExecutionId, checkpointId);
-
- if (reason == null ||
- reason.getClass() ==
AlignmentLimitExceededException.class ||
- reason.getClass() ==
CheckpointDeclineOnCancellationBarrierException.class ||
- reason.getClass() ==
CheckpointDeclineSubsumedException.class ||
- reason.getClass() ==
CheckpointDeclineTaskNotCheckpointingException.class ||
- reason.getClass() ==
CheckpointDeclineTaskNotReadyException.class ||
- reason.getClass() == InputEndOfStreamException.class)
- {
- // null or known common exceptions that cannot
reference any dynamically loaded code
+
+ if (reason == null) {
this.reason = reason;
} else {
- // some other exception. replace with a serialized
throwable, to be on the safe side
+ // exceptions may reference dynamically loaded code
(exception itself, cause, suppressed)
+ // -> replace with a serialized throwable, to be on the
safe side
this.reason = new SerializedThrowable(reason);
}
}
@@ -68,7 +55,7 @@ public DeclineCheckpoint(JobID job, ExecutionAttemptID
taskExecutionId, long che
/**
* Gets the reason why the checkpoint was declined.
- *
+ *
* @return The reason why the checkpoint was declined
*/
public Throwable getReason() {
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on 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