Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[hotfix] Forward root cause in DebeziumSourceFunction #1791

Merged
merged 2 commits into from
Dec 5, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -305,7 +305,8 @@ public void snapshotState(FunctionSnapshotContext functionSnapshotContext) throw
if (handover.hasError()) {
LOG.debug("snapshotState() called on closed source");
throw new FlinkRuntimeException(
"Call snapshotState() on closed source, checkpoint failed.");
"Call snapshotState() on closed source, checkpoint failed.",
handover.getError());
} else {
snapshotOffsetState(functionSnapshotContext.getCheckpointId());
snapshotHistoryRecordsState();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import javax.annotation.Nullable;
import javax.annotation.concurrent.GuardedBy;
import javax.annotation.concurrent.ThreadSafe;

Expand Down Expand Up @@ -56,6 +57,7 @@ public class Handover implements Closeable {
private List<ChangeEvent<SourceRecord, SourceRecord>> next;

@GuardedBy("lock")
@Nullable
private Throwable error;

private boolean wakeupProducer;
Expand Down Expand Up @@ -161,6 +163,18 @@ public boolean hasError() {
return error != null;
}

/**
* Return the error, if its set.
*
* @return the error, if its set
*/
@Nullable
public Throwable getError() {
synchronized (lock) {
return this.error;
}
}

/**
* Closes the handover. Both the {@link #produce(List)} method and the {@link #pollNext()} will
* throw a {@link ClosedException} on any currently blocking and future invocations.
Expand Down