[python] Make HTTP timeout/retry/keep-alive configurable via CatalogOptions#7732
[python] Make HTTP timeout/retry/keep-alive configurable via CatalogOptions#7732TheR1sing3un wants to merge 1 commit intoapache:masterfrom
Conversation
…ptions The REST HttpClient hardcoded its retry count and ignored its own session timeout (the requests library does not honour Session.timeout; the timeout must be passed to session.request()). Introduce five CatalogOptions so users can tune REST behaviour without monkey-patching: * http.connect-timeout (int, default 180) * http.read-timeout (int, default 180) * http.max-connect-retries (int, default 3) * http.max-read-retries (int, default 3) * http.keep-alive (bool, default true) HttpClient now accepts an optional Options argument, reads these keys, applies the timeout via session.request(timeout=...), separates connect and read retry counters in ExponentialRetry (total=None lets each type of retry govern independently), and emits Connection: close when keep-alive is disabled. RESTApi forwards its options through. Update token_loader.py to use the new ExponentialRetry signature, and add HttpClient option-coverage tests alongside the retry tests.
|
Why can't it be consistent with Java implementation? |
Thanks for the review @JingsongLi. I do want to keep both the bug fix and the new options in this PR. The reason isn't just convenience — without these as catalog options the client's actual behaviour is not under our control. requests falls back to the OS / kernel for
Exposing the five keys (http.connect-timeout / http.read-timeout / http.max-connect-retries / http.max-read-retries / http.keep-alive) is what lets us pin those down deterministically per-catalog. The defaults already match Java's On the Java consistency point: I take that seriously. My plan is, once this PR lands, to follow up with a dedicated PR that introduces the same option keys on the Java side (extending RESTCatalogOptions and wiring them through If that two-step plan works for you, I'll keep this PR as-is. Otherwise happy to discuss alternatives. |
Purpose
The REST
HttpClientcurrently hardcodes its retry count and ignores its own session timeout. Two practical issues fall out of this:requests.Session.timeoutis not consulted by the library — onlysession.request(timeout=...)is. The previousself.session.timeout = (180, 180)had no effect, so requests could hang indefinitely on a slow server.max_retries=3mixed connect / read retries; users could not disable connect retries (which often shouldn't retry) or boost read retries against flaky upstreams.This PR introduces five
CatalogOptionsso REST behaviour can be tuned via standard catalog options:http.connect-timeouthttp.read-timeouthttp.max-connect-retrieshttp.max-read-retrieshttp.keep-aliveConnection: closeHttpClientnow accepts an optionalOptionsargument, applies the timeout viasession.request(timeout=...), separatesconnect/readretry counters inExponentialRetry(withtotal=Noneso each type governs independently), and setsConnection: closewhen keep-alive is disabled.RESTApiforwards its options through.token_loader.pyis updated to the newExponentialRetry(connect_retries, read_retries)signature.Linked issue
N/A — discovered while tuning REST clients against high-latency catalog servers and verifying that
session.timeoutis dead code.Tests
pypaimon/tests/rest/test_exponential_retry_strategy.py— refreshed for the new signature; coverstotal=None, separated connect/read counters, zero-retries, and an end-to-end retry-on-connect-error case.pypaimon/tests/rest/client_test.py— newHttpClientHttpOptionsTest:options=Nonehttp.connect-timeout/http.read-timeoutreachclient._timeouthttp.keep-alive=falsesetsConnection: closehttp.max-connect-retries/http.max-read-retriesreach the mounted adapter'sRetryLocal:
pytest pypaimon/tests/rest/client_test.py pypaimon/tests/rest/test_exponential_retry_strategy.py→ 10 passed;flake8 --config=dev/cfg.iniclean.API and format
HttpClient.__init__adds an optionaloptionskwarg (backward compatible — existingHttpClient(uri)callers still work and use the same default behaviour as before, except the timeout is now actually honoured).ExponentialRetry.__init__now takesconnect_retries/read_retriesinstead of a singlemax_retries. The only internal caller (token_loader.py) is updated; no public API forExponentialRetry.No file format change.
Documentation
Option keys are self-described via
with_description(...)inCatalogOptions. No additional doc change required.Generative AI disclosure
Drafted with assistance from an AI coding tool; all logic reviewed by the author and validated by the tests above.