Details
-
Improvement
-
Status: Open
-
Major
-
Resolution: Unresolved
-
None
-
None
Description
Discussion thread: https://lists.apache.org/thread/yrf0mmbch0lhk3rgkz94fr0x5qz2417l
Currently the Elasticsearch 8 connector retries all items if the request fails as a whole, and retries failed items if the request has partial failures [1]. I think this infinitely retries might be problematic in some cases when retrying can never eventually succeed. For example, if the request is 400 (bad request) or 404 (not found), retries do not help. If there are too many failed items non-retriable, new requests will get processed less effectively. In extreme cases, it may stall the pipeline if in-flight requests are occupied by those failed items.
FLIP-451 proposes timeout for retrying which helps with un-acknowledged requests, but not addressing the case when request gets processed and failed items keep failing no matter how many times we retry. Correct me if I'm wrong.
One opinionated option is to fail fast for non-retriable errors like 400 / 404 and to drop items for 409. Or we can allow users to configure "drop/fail" behavior for non-retriable errors. I prefer the latter. I checked how LogStash ingests data to Elasticsearch and it takes a similar approach for non-retriable errors [2]. In my day job, we have a dead-letter-queue in AsynSinkWriter for failed entries that exhaust retries. I guess that is too specific to our setup and seems an overkill here for Elasticsearch connector.
Attachments
Issue Links
- links to