-
Notifications
You must be signed in to change notification settings - Fork 590
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
Bug: consumption of connector source will hang forever if encounters an error #7192
Comments
+1. The upper-layer streaming logic has no way to handle the connector error, so we expect the connector itself to retry in the case of temporary service unavailable as much as possible. For those unrecoverable errors, we catch the error by So we may also need to report the error as a user error? cc @jon-chuang @fuyufjh |
I believe that in this case, it's not an unrecoverable error. Is this accurate? @tabVersion
Perhaps this particular case should actually not be handled as an error. Unclear if we should even report it as a user error. Consuming the Kafka partition temporarily should be expected in normal operation.
EDIT: I guess this is an artificial error and we should not focus too much on it. |
It also seems that we need to classify errors into recoverable and unrecoverable errors. Question: should this be defined in the connector, or in the stream layer? |
+1 for this. Connector itself should not propagate the error to stream, rather, it should itself retry, and once it terminates the retry, emit an error that the stream actor will decide what to do with. I think we should bounded exponential retry (default: up to few hours? - to give the user/SRE time to respond, perhaps restart the source) and report the user error in the meantime. |
~~Currently, due to `try_stream`, every executor should first propagate its msg errors into the stream. This will eventually be intercepted by `impl StreamConsumer for DispatchExecutor`, the error from `StreamExecutorResult` will be propagated into `BarrierStream`, and `Actor::run_consumer` will terminate, triggering `context.lock_barrier_manager().notify_failure(actor_id, err);`~~
Edit: yes, |
After discussion with @tabVersion , we are in agreement on the following:
|
Describe the bug
If the connector source throws an error up to the source executor, the consumption of the source will stop forever:
risingwave/src/stream/src/executor/source/reader.rs
Lines 61 to 62 in 6f39f43
But the barrier messages can still pass to downstream executors:
risingwave/src/stream/src/executor/source/reader.rs
Lines 90 to 95 in 98ae6ce
I am not sure whether this problem is a by-design feature.
To Reproduce
Manually construct an error in the
KafkaSplitReader
and propagate it to the upper layer.Then start the cluster with the
full
config and run the tpch-bench script to ingest data.Expected behavior
IIUC, when executors encounter an error in other scenarios, the error will report to the Meta via the barrier collection mechanism (#6319). Then Meta will enter the recovery process to recover the whole streaming graph.
But for the scenario of connector source failure, the upstream system may become available after a while or the failure is unrecoverable indeed. So it may be a waste to let Meta recover the cluster blindly:
risingwave/src/meta/src/barrier/mod.rs
Lines 772 to 780 in 1ba6981
Candidate solutions:
Employ a bounded retry strategy for connector sources
When a connector source encounters an error, we will try our best to recover the consumer client. For example, we can drop the current consumer and create a new one to try to resume consumption. If we fail to recover the consumer, we can choose to hang up the connector source stream and prompt users to drop the source and troubleshoot the upstream system.
Hang up the connector source stream forever as the current implementation did and prompt users to drop the source and troubleshoot the upstream system. For example, we can prompt an error to users when they query the downstream MVs of the broken source.
Additional context
No response
The text was updated successfully, but these errors were encountered: