hubcio commented on code in PR #2933:
URL: https://github.com/apache/iggy/pull/2933#discussion_r2993933012


##########
core/connectors/sources/influxdb_source/src/lib.rs:
##########
@@ -0,0 +1,1265 @@
+/* Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use async_trait::async_trait;
+use base64::{Engine as _, engine::general_purpose};
+use csv::StringRecord;
+use iggy_common::{DateTime, Utc};
+use iggy_connector_sdk::retry::{
+    CircuitBreaker, build_retry_client, exponential_backoff, jitter, 
parse_duration,
+};
+use iggy_connector_sdk::{
+    ConnectorState, Error, ProducedMessage, ProducedMessages, Schema, Source, 
source_connector,
+};
+use regex::Regex;
+use reqwest::Url;
+use reqwest_middleware::ClientWithMiddleware;
+use serde::{Deserialize, Serialize};
+use serde_json::json;
+use std::collections::HashMap;
+use std::sync::Arc;
+use std::sync::OnceLock;
+use std::time::Duration;
+use tokio::sync::Mutex;
+use tracing::{debug, error, info, warn};
+use uuid::Uuid;
+
+source_connector!(InfluxDbSource);
+
+const CONNECTOR_NAME: &str = "InfluxDB source";
+const DEFAULT_MAX_RETRIES: u32 = 3;
+const DEFAULT_RETRY_DELAY: &str = "1s";
+const DEFAULT_POLL_INTERVAL: &str = "5s";
+const DEFAULT_TIMEOUT: &str = "10s";
+const DEFAULT_CURSOR: &str = "1970-01-01T00:00:00Z";
+// Maximum attempts for open() connectivity retries
+const DEFAULT_MAX_OPEN_RETRIES: u32 = 10;
+// Cap for exponential backoff in open() — never wait longer than this
+const DEFAULT_OPEN_RETRY_MAX_DELAY: &str = "60s";
+// Cap for exponential backoff on per-query retries — kept short so a
+// transient InfluxDB blip does not stall polling for too long
+const DEFAULT_RETRY_MAX_DELAY: &str = "5s";
+// How many consecutive poll failures open the circuit breaker
+const DEFAULT_CIRCUIT_BREAKER_THRESHOLD: u32 = 5;
+// How long the circuit stays open before allowing a probe attempt
+const DEFAULT_CIRCUIT_COOL_DOWN: &str = "30s";
+
+/// RFC 3339 / ISO 8601 datetime pattern.
+/// Matches the forms InfluxDB stores in `_time`:
+///   "2024-01-15T10:30:00Z"
+///   "2024-01-15T10:30:00.123456789Z"
+///   "2024-01-15T10:30:00+05:30"
+/// Intentionally strict: only digits, T, Z, colon, dot, plus, hyphen.
+/// Any Flux syntax character (pipe, quote, paren, space, slash) is rejected.
+static CURSOR_RE: OnceLock<Regex> = OnceLock::new();
+
+// ---------------------------------------------------------------------------
+// Main connector structs
+// ---------------------------------------------------------------------------
+
+#[derive(Debug)]
+pub struct InfluxDbSource {
+    pub id: u32,
+    config: InfluxDbSourceConfig,
+    /// `None` until `open()` is called. Wraps `reqwest::Client` with
+    /// [`InfluxDbRetryMiddleware`] so retry/back-off/jitter is handled
+    /// transparently by the middleware stack instead of a hand-rolled loop.
+    client: Option<ClientWithMiddleware>,
+    state: Mutex<State>,
+    verbose: bool,
+    retry_delay: Duration,
+    poll_interval: Duration,
+    circuit_breaker: Arc<CircuitBreaker>,
+}
+
+#[derive(Debug, Clone, Serialize, Deserialize)]
+pub struct InfluxDbSourceConfig {
+    pub url: String,
+    pub org: String,
+    pub token: String,
+    pub query: String,
+    pub poll_interval: Option<String>,
+    pub batch_size: Option<u32>,
+    pub cursor_field: Option<String>,
+    pub initial_offset: Option<String>,
+    pub payload_column: Option<String>,
+    pub payload_format: Option<String>,
+    pub include_metadata: Option<bool>,
+    pub verbose_logging: Option<bool>,
+    pub max_retries: Option<u32>,
+    pub retry_delay: Option<String>,
+    pub timeout: Option<String>,
+    // How many times open() will retry before giving up
+    pub max_open_retries: Option<u32>,
+    // Upper cap on open() backoff delay — can be set high (e.g. "60s") for
+    // patient startup without affecting per-query retry behaviour
+    pub open_retry_max_delay: Option<String>,
+    // Upper cap on per-query retry backoff — kept short so a transient blip
+    // does not stall polling; independent of open_retry_max_delay
+    pub retry_max_delay: Option<String>,
+    // Circuit breaker configuration
+    pub circuit_breaker_threshold: Option<u32>,
+    pub circuit_breaker_cool_down: Option<String>,
+}
+
+#[derive(Debug, Clone, Copy, PartialEq, Eq, Default)]
+enum PayloadFormat {
+    #[default]
+    Json,
+    Text,
+    Raw,
+}
+
+impl PayloadFormat {
+    fn from_config(value: Option<&str>) -> Self {
+        match value.map(|v| v.to_ascii_lowercase()).as_deref() {
+            Some("text") | Some("utf8") => PayloadFormat::Text,
+            Some("raw") | Some("base64") => PayloadFormat::Raw,
+            Some("json") => PayloadFormat::Json,
+            other => {
+                warn!(
+                    "Unrecognized payload_format value {:?}, falling back to 
JSON. \
+                     Valid values are: \"json\", \"text\", \"utf8\", 
\"base64\", \"raw\".",
+                    other
+                );
+                PayloadFormat::Json
+            }
+        }
+    }
+
+    fn schema(self) -> Schema {
+        match self {
+            PayloadFormat::Json => Schema::Json,
+            PayloadFormat::Text => Schema::Text,
+            PayloadFormat::Raw => Schema::Raw,
+        }
+    }
+}
+
+#[derive(Debug, Serialize, Deserialize)]
+struct State {
+    last_poll_time: DateTime<Utc>,
+    last_timestamp: Option<String>,
+    processed_rows: u64,
+    /// How many rows at `last_timestamp` have already been delivered 
downstream.
+    ///
+    /// When the user's Flux query uses `>= $cursor`, consecutive polls may
+    /// return the same rows for the current cursor timestamp.  This counter
+    /// lets `poll_messages` skip those already-delivered rows and inflate
+    /// `$limit` accordingly, preventing both duplicates and data loss at
+    /// batch boundaries where multiple rows share the same timestamp.
+    ///
+    /// `#[serde(default)]` keeps existing persisted state files 
forward-compatible:
+    /// the field defaults to 0 when the state was saved by an older version.
+    #[serde(default)]
+    cursor_row_count: u64,
+}
+
+// ---------------------------------------------------------------------------
+// Helpers
+// ---------------------------------------------------------------------------
+
+fn parse_scalar(value: &str) -> serde_json::Value {
+    if value.is_empty() {
+        return serde_json::Value::Null;
+    }
+    if let Ok(v) = value.parse::<bool>() {
+        return serde_json::Value::Bool(v);
+    }
+    if let Ok(v) = value.parse::<i64>() {
+        return serde_json::Value::Number(v.into());
+    }
+    if let Ok(v) = value.parse::<f64>()
+        && let Some(number) = serde_json::Number::from_f64(v)
+    {
+        return serde_json::Value::Number(number);
+    }
+    serde_json::Value::String(value.to_string())
+}
+
+fn is_header_record(record: &StringRecord) -> bool {

Review Comment:
   this requires both `_time` AND `_value` to be present, but flux aggregation 
queries (`count()`, `mean()`, `group()`) can produce result tables with 
different column layouts - e.g. `_time` + `_count` instead of `_value`. when 
that happens the header row won't be recognized, and all subsequent data rows 
until the next recognized header will be silently skipped. checking for `_time` 
alone (or detecting influxdb CSV annotation rows like `#group`, `#datatype`, 
`#default`) would be more robust.



##########
core/connectors/runtime/example_config/connectors/influxdb_source.toml:
##########
@@ -0,0 +1,54 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+type = "source"
+key = "influxdb"
+enabled = true
+version = 0
+name = "InfluxDB source"
+path = "<BASE_DIR>/target/release/libiggy_connector_influxdb_source"
+plugin_config_format = "toml"
+verbose = false
+
+[[streams]]
+stream = "events"
+topic = "influx_events"
+schema = "json"
+batch_length = 100
+
+[plugin_config]
+url = "http://localhost:8086";
+org = "iggy_org"
+token = "replace_token_with_secret_token"
+query = '''
+from(bucket: "iggy_bucket")
+  |> range(start: -1h)
+  |> filter(fn: (r) => r._measurement == "iggy_messages")
+  |> filter(fn: (r) => r._time > time(v: "$cursor"))
+  |> sort(columns: ["_time"])
+  |> limit(n: $limit)
+'''
+poll_interval = "5s"
+batch_size = 500
+cursor_field = "_time"
+initial_offset = "1970-01-01T00:00:00Z"
+include_metadata = true
+payload_format = "json"
+max_retries = 3
+retry_delay = "1s"
+timeout = "10s"
+precision = "us"

Review Comment:
   `InfluxDbSourceConfig` has no `precision` field - this key is silently 
ignored by serde deserialization. a user copying this example and setting 
`precision = "ns"` will get no effect and no error. either add the field to the 
config struct and use it, or remove this line.



##########
core/connectors/sdk/src/retry.rs:
##########
@@ -0,0 +1,463 @@
+/* Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+//! Shared retry and resilience utilities for connector implementations.
+//!
+//! Provides:
+//! - [`CircuitBreaker`] — consecutive-failure circuit breaker
+//! - [`InfluxDbRetryMiddleware`] — `reqwest-middleware` middleware with
+//!   exponential back-off, jitter, and `Retry-After` header support
+//! - [`build_retry_client`] — wraps a `reqwest::Client` with the middleware
+//! - [`is_transient_status`] — transient HTTP status predicate
+//! - [`parse_duration`] — humantime duration parsing with fallback
+//! - [`jitter`] — ±20 % random jitter for retry delays
+//! - [`exponential_backoff`] — capped exponential backoff
+//! - [`parse_retry_after`] — HTTP `Retry-After` header parsing
+
+use anyhow::anyhow;
+use http::Extensions;
+use humantime::Duration as HumanDuration;
+use rand::RngExt as _;
+use reqwest_middleware::{ClientBuilder, ClientWithMiddleware, Middleware, 
Next};
+use std::str::FromStr;
+use std::time::Duration;
+use tokio::sync::Mutex;
+use tracing::{info, warn};
+
+// ---------------------------------------------------------------------------
+// Circuit breaker
+// ---------------------------------------------------------------------------
+
+#[derive(Debug)]
+struct CircuitState {
+    consecutive_failures: u32,
+    open_until: Option<tokio::time::Instant>,
+}
+
+/// A simple consecutive-failure circuit breaker.
+///
+/// All mutable state is held under a single [`Mutex`] so that
+/// `consecutive_failures` and `open_until` are always updated atomically,
+/// preventing races between concurrent `record_failure` / `is_open` callers.
+#[derive(Debug)]
+pub struct CircuitBreaker {
+    threshold: u32,
+    cool_down: Duration,
+    state: Mutex<CircuitState>,
+}
+
+impl CircuitBreaker {
+    pub fn new(threshold: u32, cool_down: Duration) -> Self {
+        Self {
+            threshold,
+            cool_down,
+            state: Mutex::new(CircuitState {
+                consecutive_failures: 0,
+                open_until: None,
+            }),
+        }
+    }
+
+    /// Called on every successful operation — resets the failure counter and
+    /// closes the circuit atomically.
+    ///
+    /// Uses `try_lock` so success never blocks on the hot path; at worst one
+    /// extra failure is needed to re-open an already-closing circuit.
+    pub fn record_success(&self) {
+        if let Ok(mut s) = self.state.try_lock() {
+            s.consecutive_failures = 0;
+            s.open_until = None;
+        }
+    }
+
+    /// Called after all retries for one operation have failed. May open the
+    /// circuit once the failure count reaches the configured threshold.
+    pub async fn record_failure(&self) {
+        let mut s = self.state.lock().await;
+        s.consecutive_failures = s.consecutive_failures.saturating_add(1);
+        if s.consecutive_failures >= self.threshold {
+            let deadline = tokio::time::Instant::now() + self.cool_down;
+            s.open_until = Some(deadline);
+            warn!(
+                "Circuit breaker OPENED after {} consecutive failures. \
+                 Pausing for {:?}.",
+                s.consecutive_failures, self.cool_down
+            );
+        }
+    }
+
+    /// Returns `true` if the circuit is open (callers should skip the
+    /// operation). Transitions to half-open automatically once the cool-down
+    /// has elapsed.
+    pub async fn is_open(&self) -> bool {
+        let mut s = self.state.lock().await;
+        match s.open_until {
+            None => false,
+            Some(deadline) if tokio::time::Instant::now() < deadline => true,
+            Some(_) => {
+                // Cool-down elapsed: half-open — let one probe through.
+                s.open_until = None;
+                s.consecutive_failures = 0;
+                info!("Circuit breaker entering HALF-OPEN state.");
+                false
+            }
+        }
+    }
+}
+
+// ---------------------------------------------------------------------------
+// Duration / backoff helpers
+// ---------------------------------------------------------------------------
+
+/// Parse a human-readable duration string (e.g. `"5s"`, `"1m30s"`) using
+/// [`humantime`]. Falls back to 1 second if parsing fails.
+pub fn parse_duration(value: Option<&str>, default_value: &str) -> Duration {
+    let raw = value.unwrap_or(default_value);
+    HumanDuration::from_str(raw)
+        .map(|d| d.into())
+        .unwrap_or_else(|_| Duration::from_secs(1))
+}
+
+/// Apply ±20 % random jitter to `base` to spread retry storms.
+pub fn jitter(base: Duration) -> Duration {
+    let millis = base.as_millis() as u64;
+    let jitter_range = millis / 5; // 20% of base
+    if jitter_range == 0 {
+        return base;
+    }
+    let delta = rand::rng().random_range(0..=jitter_range * 2);
+    
Duration::from_millis(millis.saturating_sub(jitter_range).saturating_add(delta))
+}
+
+/// True exponential backoff: `base × 2^attempt`, capped at `max_delay`.
+pub fn exponential_backoff(base: Duration, attempt: u32, max_delay: Duration) 
-> Duration {
+    let factor = 2u64.saturating_pow(attempt);
+    let millis = base
+        .as_millis()
+        .saturating_mul(factor as u128)
+        .min(max_delay.as_millis());
+    let millis_u64 = u64::try_from(millis).unwrap_or(u64::MAX);
+    Duration::from_millis(millis_u64)
+}
+
+/// Parse a `Retry-After` header value (integer seconds).
+/// Returns `None` for HTTP-date values — callers should fall back to their
+/// own backoff strategy.
+pub fn parse_retry_after(value: &str) -> Option<Duration> {
+    if let Ok(secs) = value.trim().parse::<u64>() {
+        return Some(Duration::from_secs(secs));
+    }
+    None
+}
+
+// ---------------------------------------------------------------------------
+// reqwest-middleware retry implementation
+// ---------------------------------------------------------------------------
+
+/// Returns `true` for HTTP status codes that are worth retrying:
+/// `429 Too Many Requests` and all `5xx` server errors.
+pub fn is_transient_status(status: reqwest::StatusCode) -> bool {
+    status == reqwest::StatusCode::TOO_MANY_REQUESTS || 
status.is_server_error()
+}
+
+/// Per-request retry middleware for InfluxDB connectors.
+///
+/// Wraps a `reqwest-middleware` stack and retries transient failures
+/// (HTTP 429, 5xx, network errors) with exponential back-off and ±20 % jitter.
+/// An `Retry-After` response header on a 429 overrides the calculated delay.
+///
+/// The `max_retries` parameter is the *total attempt count* (not the number
+/// of extra attempts), consistent with the rest of the connector retry config:
+/// - `max_retries = 1` → one attempt, no retries on failure
+/// - `max_retries = 3` → up to three attempts (two retries after a failure)
+///
+/// Non-transient error responses (4xx except 429) are returned as-is so
+/// callers can inspect the status and body to build a meaningful error.
+#[derive(Debug, Clone)]
+pub struct InfluxDbRetryMiddleware {

Review Comment:
   this struct lives in `iggy_connector_sdk::retry` - the shared SDK crate for 
all connectors - but the name and log messages (e.g. line 256: `"InfluxDB 
error"`) are influxdb-specific. the behavior itself is fully generic (retry 
transient HTTP errors with exponential backoff). rename to 
`HttpRetryMiddleware` (or similar) and parameterize the log prefix so the next 
connector author can reuse it without importing something misleadingly named or 
copy-pasting.



##########
core/connectors/sinks/influxdb_sink/src/lib.rs:
##########
@@ -0,0 +1,1104 @@
+/* Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use async_trait::async_trait;
+use base64::{Engine as _, engine::general_purpose};
+use bytes::Bytes;
+use iggy_connector_sdk::retry::{
+    CircuitBreaker, build_retry_client, exponential_backoff, jitter, 
parse_duration,
+};
+use iggy_connector_sdk::{
+    ConsumedMessage, Error, MessagesMetadata, Sink, TopicMetadata, 
sink_connector,
+};
+use reqwest::Url;
+use reqwest_middleware::ClientWithMiddleware;
+use serde::{Deserialize, Serialize};
+use std::sync::Arc;
+use std::sync::atomic::{AtomicU64, Ordering};
+use std::time::Duration;
+use std::time::SystemTime;
+use std::time::UNIX_EPOCH;
+use tracing::{debug, error, info, warn};
+sink_connector!(InfluxDbSink);
+
+const DEFAULT_MAX_RETRIES: u32 = 3;
+const DEFAULT_RETRY_DELAY: &str = "1s";
+const DEFAULT_TIMEOUT: &str = "30s";
+const DEFAULT_PRECISION: &str = "us";
+// Maximum attempts for open() connectivity retries
+const DEFAULT_MAX_OPEN_RETRIES: u32 = 10;
+// Cap for exponential backoff in open() — never wait longer than this
+const DEFAULT_OPEN_RETRY_MAX_DELAY: &str = "60s";
+// Cap for exponential backoff on per-write retries — kept short so a
+// transient InfluxDB blip does not stall message delivery for too long
+const DEFAULT_RETRY_MAX_DELAY: &str = "5s";
+// How many consecutive batch failures open the circuit breaker
+const DEFAULT_CIRCUIT_BREAKER_THRESHOLD: u32 = 5;
+// How long the circuit stays open before allowing a probe attempt
+const DEFAULT_CIRCUIT_COOL_DOWN: &str = "30s";
+
+// ---------------------------------------------------------------------------
+// Main connector structs
+// ---------------------------------------------------------------------------
+
+#[derive(Debug)]
+pub struct InfluxDbSink {
+    pub id: u32,
+    config: InfluxDbSinkConfig,
+    /// `None` until `open()` is called. Wraps `reqwest::Client` with
+    /// [`InfluxDbRetryMiddleware`] so retry/back-off/jitter is handled
+    /// transparently by the middleware stack instead of a hand-rolled loop.
+    client: Option<ClientWithMiddleware>,
+    /// Cached once in `open()` — config fields never change at runtime.
+    write_url: Option<Url>,
+    messages_attempted: AtomicU64,
+    write_success: AtomicU64,
+    write_errors: AtomicU64,
+    verbose: bool,
+    retry_delay: Duration,
+    circuit_breaker: Arc<CircuitBreaker>,
+}
+
+#[derive(Debug, Clone, Serialize, Deserialize)]
+pub struct InfluxDbSinkConfig {
+    pub url: String,
+    pub org: String,
+    pub bucket: String,
+    pub token: String,

Review Comment:
   also at `core/connectors/sources/influxdb_source/src/lib.rs:94`.
   
   every other connector in the codebase uses `SecretString` from the `secrecy` 
crate for credentials - elasticsearch uses `Option<SecretString>` for password, 
mongodb uses `SecretString` for connection_uri, postgres uses `SecretString` 
for connection_string. the influxdb API token grants full bucket read/write 
access and will show up in debug logs and serialized config dumps as a plain 
string. switch to `SecretString` with `serialize_secret` and use 
`.expose_secret()` at the two points where it's interpolated into the 
`Authorization` header.



##########
core/connectors/sdk/src/retry.rs:
##########
@@ -0,0 +1,463 @@
+/* Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+//! Shared retry and resilience utilities for connector implementations.
+//!
+//! Provides:
+//! - [`CircuitBreaker`] — consecutive-failure circuit breaker
+//! - [`InfluxDbRetryMiddleware`] — `reqwest-middleware` middleware with
+//!   exponential back-off, jitter, and `Retry-After` header support
+//! - [`build_retry_client`] — wraps a `reqwest::Client` with the middleware
+//! - [`is_transient_status`] — transient HTTP status predicate
+//! - [`parse_duration`] — humantime duration parsing with fallback
+//! - [`jitter`] — ±20 % random jitter for retry delays
+//! - [`exponential_backoff`] — capped exponential backoff
+//! - [`parse_retry_after`] — HTTP `Retry-After` header parsing
+
+use anyhow::anyhow;
+use http::Extensions;
+use humantime::Duration as HumanDuration;
+use rand::RngExt as _;
+use reqwest_middleware::{ClientBuilder, ClientWithMiddleware, Middleware, 
Next};
+use std::str::FromStr;
+use std::time::Duration;
+use tokio::sync::Mutex;
+use tracing::{info, warn};
+
+// ---------------------------------------------------------------------------
+// Circuit breaker
+// ---------------------------------------------------------------------------
+
+#[derive(Debug)]
+struct CircuitState {
+    consecutive_failures: u32,
+    open_until: Option<tokio::time::Instant>,
+}
+
+/// A simple consecutive-failure circuit breaker.
+///
+/// All mutable state is held under a single [`Mutex`] so that
+/// `consecutive_failures` and `open_until` are always updated atomically,
+/// preventing races between concurrent `record_failure` / `is_open` callers.
+#[derive(Debug)]
+pub struct CircuitBreaker {
+    threshold: u32,
+    cool_down: Duration,
+    state: Mutex<CircuitState>,
+}
+
+impl CircuitBreaker {
+    pub fn new(threshold: u32, cool_down: Duration) -> Self {
+        Self {
+            threshold,
+            cool_down,
+            state: Mutex::new(CircuitState {
+                consecutive_failures: 0,
+                open_until: None,
+            }),
+        }
+    }
+
+    /// Called on every successful operation — resets the failure counter and
+    /// closes the circuit atomically.
+    ///
+    /// Uses `try_lock` so success never blocks on the hot path; at worst one
+    /// extra failure is needed to re-open an already-closing circuit.
+    pub fn record_success(&self) {
+        if let Ok(mut s) = self.state.try_lock() {
+            s.consecutive_failures = 0;
+            s.open_until = None;
+        }
+    }
+
+    /// Called after all retries for one operation have failed. May open the
+    /// circuit once the failure count reaches the configured threshold.
+    pub async fn record_failure(&self) {
+        let mut s = self.state.lock().await;
+        s.consecutive_failures = s.consecutive_failures.saturating_add(1);
+        if s.consecutive_failures >= self.threshold {
+            let deadline = tokio::time::Instant::now() + self.cool_down;
+            s.open_until = Some(deadline);
+            warn!(
+                "Circuit breaker OPENED after {} consecutive failures. \
+                 Pausing for {:?}.",
+                s.consecutive_failures, self.cool_down
+            );
+        }
+    }
+
+    /// Returns `true` if the circuit is open (callers should skip the
+    /// operation). Transitions to half-open automatically once the cool-down
+    /// has elapsed.
+    pub async fn is_open(&self) -> bool {
+        let mut s = self.state.lock().await;
+        match s.open_until {
+            None => false,
+            Some(deadline) if tokio::time::Instant::now() < deadline => true,
+            Some(_) => {
+                // Cool-down elapsed: half-open — let one probe through.
+                s.open_until = None;
+                s.consecutive_failures = 0;
+                info!("Circuit breaker entering HALF-OPEN state.");
+                false
+            }
+        }
+    }
+}
+
+// ---------------------------------------------------------------------------
+// Duration / backoff helpers
+// ---------------------------------------------------------------------------
+
+/// Parse a human-readable duration string (e.g. `"5s"`, `"1m30s"`) using
+/// [`humantime`]. Falls back to 1 second if parsing fails.
+pub fn parse_duration(value: Option<&str>, default_value: &str) -> Duration {

Review Comment:
   if the user-provided value is malformed (e.g. `retry_delay = "5sec"` instead 
of `"5s"`), this silently returns 1 second with no log. a `warn!()` on the 
fallback path would save users from debugging mysterious retry timing.



##########
core/connectors/sinks/influxdb_sink/src/lib.rs:
##########
@@ -0,0 +1,1104 @@
+/* Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use async_trait::async_trait;
+use base64::{Engine as _, engine::general_purpose};
+use bytes::Bytes;
+use iggy_connector_sdk::retry::{
+    CircuitBreaker, build_retry_client, exponential_backoff, jitter, 
parse_duration,
+};
+use iggy_connector_sdk::{
+    ConsumedMessage, Error, MessagesMetadata, Sink, TopicMetadata, 
sink_connector,
+};
+use reqwest::Url;
+use reqwest_middleware::ClientWithMiddleware;
+use serde::{Deserialize, Serialize};
+use std::sync::Arc;
+use std::sync::atomic::{AtomicU64, Ordering};
+use std::time::Duration;
+use std::time::SystemTime;
+use std::time::UNIX_EPOCH;
+use tracing::{debug, error, info, warn};
+sink_connector!(InfluxDbSink);
+
+const DEFAULT_MAX_RETRIES: u32 = 3;
+const DEFAULT_RETRY_DELAY: &str = "1s";
+const DEFAULT_TIMEOUT: &str = "30s";
+const DEFAULT_PRECISION: &str = "us";
+// Maximum attempts for open() connectivity retries
+const DEFAULT_MAX_OPEN_RETRIES: u32 = 10;
+// Cap for exponential backoff in open() — never wait longer than this
+const DEFAULT_OPEN_RETRY_MAX_DELAY: &str = "60s";
+// Cap for exponential backoff on per-write retries — kept short so a
+// transient InfluxDB blip does not stall message delivery for too long
+const DEFAULT_RETRY_MAX_DELAY: &str = "5s";
+// How many consecutive batch failures open the circuit breaker
+const DEFAULT_CIRCUIT_BREAKER_THRESHOLD: u32 = 5;
+// How long the circuit stays open before allowing a probe attempt
+const DEFAULT_CIRCUIT_COOL_DOWN: &str = "30s";
+
+// ---------------------------------------------------------------------------
+// Main connector structs
+// ---------------------------------------------------------------------------
+
+#[derive(Debug)]
+pub struct InfluxDbSink {
+    pub id: u32,
+    config: InfluxDbSinkConfig,
+    /// `None` until `open()` is called. Wraps `reqwest::Client` with
+    /// [`InfluxDbRetryMiddleware`] so retry/back-off/jitter is handled
+    /// transparently by the middleware stack instead of a hand-rolled loop.
+    client: Option<ClientWithMiddleware>,
+    /// Cached once in `open()` — config fields never change at runtime.
+    write_url: Option<Url>,
+    messages_attempted: AtomicU64,
+    write_success: AtomicU64,
+    write_errors: AtomicU64,
+    verbose: bool,
+    retry_delay: Duration,
+    circuit_breaker: Arc<CircuitBreaker>,
+}
+
+#[derive(Debug, Clone, Serialize, Deserialize)]
+pub struct InfluxDbSinkConfig {
+    pub url: String,
+    pub org: String,
+    pub bucket: String,
+    pub token: String,
+    pub measurement: Option<String>,
+    pub precision: Option<String>,
+    pub batch_size: Option<u32>,
+    pub include_metadata: Option<bool>,
+    pub include_checksum: Option<bool>,
+    pub include_origin_timestamp: Option<bool>,
+    pub include_stream_tag: Option<bool>,
+    pub include_topic_tag: Option<bool>,
+    pub include_partition_tag: Option<bool>,
+    pub payload_format: Option<String>,
+    pub verbose_logging: Option<bool>,
+    pub max_retries: Option<u32>,
+    pub retry_delay: Option<String>,
+    pub timeout: Option<String>,
+    // How many times open() will retry before giving up
+    pub max_open_retries: Option<u32>,
+    // Upper cap on open() backoff delay — can be set high (e.g. "60s") for
+    // patient startup without affecting per-write retry behaviour
+    pub open_retry_max_delay: Option<String>,
+    // Upper cap on per-write retry backoff — kept short so a transient blip
+    // does not stall message delivery; independent of open_retry_max_delay
+    pub retry_max_delay: Option<String>,
+    // Circuit breaker configuration
+    pub circuit_breaker_threshold: Option<u32>,
+    pub circuit_breaker_cool_down: Option<String>,
+}
+
+#[derive(Debug, Clone, Copy, PartialEq, Eq, Default)]
+enum PayloadFormat {
+    #[default]
+    Json,
+    Text,
+    Base64,
+}
+
+impl PayloadFormat {
+    fn from_config(value: Option<&str>) -> Self {
+        match value.map(|v| v.to_ascii_lowercase()).as_deref() {
+            Some("text") | Some("utf8") => PayloadFormat::Text,
+            Some("base64") | Some("raw") => PayloadFormat::Base64,
+            Some("json") => PayloadFormat::Json,
+            other => {
+                warn!(
+                    "Unrecognized payload_format value {:?}, falling back to 
JSON. \
+                     Valid values are: \"json\", \"text\", \"utf8\", 
\"base64\", \"raw\".",
+                    other
+                );
+                PayloadFormat::Json
+            }
+        }
+    }
+}
+
+// ---------------------------------------------------------------------------
+// Helpers
+// ---------------------------------------------------------------------------
+
+/// Write an escaped measurement name into `buf`.
+/// Escapes: `\` → `\\`, `,` → `\,`, ` ` → `\ `, `\n` → `\\n`, `\r` → `\\r`
+///
+/// Newline (`\n`) and carriage-return (`\r`) are the InfluxDB line-protocol
+/// record delimiters; a literal newline inside a measurement name would split
+/// the line and corrupt the batch.
+fn write_measurement(buf: &mut String, value: &str) {
+    for ch in value.chars() {
+        match ch {
+            '\\' => buf.push_str("\\\\"),
+            ',' => buf.push_str("\\,"),
+            ' ' => buf.push_str("\\ "),
+            '\n' => buf.push_str("\\n"),
+            '\r' => buf.push_str("\\r"),
+            _ => buf.push(ch),
+        }
+    }
+}
+
+/// Write an escaped tag key/value into `buf`.
+/// Escapes: `\` → `\\`, `,` → `\,`, `=` → `\=`, ` ` → `\ `, `\n` → `\\n`, 
`\r` → `\\r`
+///
+/// Newline and carriage-return are escaped for the same reason as in
+/// [`write_measurement`]: they are InfluxDB line-protocol record delimiters.
+fn write_tag_value(buf: &mut String, value: &str) {
+    for ch in value.chars() {
+        match ch {
+            '\\' => buf.push_str("\\\\"),
+            ',' => buf.push_str("\\,"),
+            '=' => buf.push_str("\\="),
+            ' ' => buf.push_str("\\ "),
+            '\n' => buf.push_str("\\n"),
+            '\r' => buf.push_str("\\r"),
+            _ => buf.push(ch),
+        }
+    }
+}
+
+/// Write an escaped string field value (without surrounding quotes) into 
`buf`.
+/// Escapes: `\` → `\\`, `"` → `\"`, `\n` → `\\n`, `\r` → `\\r`
+///
+/// Newline and carriage-return are the InfluxDB line-protocol record
+/// delimiters; a literal newline inside a string field value (e.g. from a
+/// multi-line text payload) would split the line and corrupt the batch.
+fn write_field_string(buf: &mut String, value: &str) {
+    for ch in value.chars() {
+        match ch {
+            '\\' => buf.push_str("\\\\"),
+            '"' => buf.push_str("\\\""),
+            '\n' => buf.push_str("\\n"),
+            '\r' => buf.push_str("\\r"),
+            _ => buf.push(ch),
+        }
+    }
+}
+
+// ---------------------------------------------------------------------------
+// InfluxDbSink implementation
+// ---------------------------------------------------------------------------
+
+impl InfluxDbSink {
+    pub fn new(id: u32, config: InfluxDbSinkConfig) -> Self {
+        let verbose = config.verbose_logging.unwrap_or(false);
+        let retry_delay = parse_duration(config.retry_delay.as_deref(), 
DEFAULT_RETRY_DELAY);
+
+        // Build circuit breaker from config
+        let cb_threshold = config
+            .circuit_breaker_threshold
+            .unwrap_or(DEFAULT_CIRCUIT_BREAKER_THRESHOLD);
+        let cb_cool_down = parse_duration(
+            config.circuit_breaker_cool_down.as_deref(),
+            DEFAULT_CIRCUIT_COOL_DOWN,
+        );
+
+        InfluxDbSink {
+            id,
+            config,
+            client: None,
+            write_url: None,
+            messages_attempted: AtomicU64::new(0),
+            write_success: AtomicU64::new(0),
+            write_errors: AtomicU64::new(0),
+            verbose,
+            retry_delay,
+            circuit_breaker: Arc::new(CircuitBreaker::new(cb_threshold, 
cb_cool_down)),
+        }
+    }
+
+    fn build_raw_client(&self) -> Result<reqwest::Client, Error> {
+        let timeout = parse_duration(self.config.timeout.as_deref(), 
DEFAULT_TIMEOUT);
+        reqwest::Client::builder()
+            .timeout(timeout)
+            .build()
+            .map_err(|e| Error::InitError(format!("Failed to create HTTP 
client: {e}")))
+    }
+
+    fn build_write_url(&self) -> Result<Url, Error> {
+        let base = self.config.url.trim_end_matches('/');
+        let mut url = Url::parse(&format!("{base}/api/v2/write"))
+            .map_err(|e| Error::InvalidConfigValue(format!("Invalid InfluxDB 
URL: {e}")))?;
+
+        let precision = self
+            .config
+            .precision
+            .as_deref()
+            .unwrap_or(DEFAULT_PRECISION);
+        url.query_pairs_mut()
+            .append_pair("org", &self.config.org)
+            .append_pair("bucket", &self.config.bucket)
+            .append_pair("precision", precision);
+
+        Ok(url)
+    }
+
+    fn build_health_url(&self) -> Result<Url, Error> {
+        let base = self.config.url.trim_end_matches('/');
+        Url::parse(&format!("{base}/health"))
+            .map_err(|e| Error::InvalidConfigValue(format!("Invalid InfluxDB 
URL: {e}")))
+    }
+
+    /// Single connectivity probe using the provided raw client (no retry).
+    /// The caller (`check_connectivity_with_retry`) is responsible for the
+    /// outer retry loop, which uses different delay bounds than per-write 
retries.
+    async fn check_connectivity(&self, client: &reqwest::Client) -> Result<(), 
Error> {
+        let url = self.build_health_url()?;
+
+        let response = client
+            .get(url)
+            .send()
+            .await
+            .map_err(|e| Error::Connection(format!("InfluxDB health check 
failed: {e}")))?;
+
+        if !response.status().is_success() {
+            let status = response.status();
+            let body = response
+                .text()
+                .await
+                .unwrap_or_else(|_| "failed to read response 
body".to_string());
+            return Err(Error::Connection(format!(
+                "InfluxDB health check returned status {status}: {body}"
+            )));
+        }
+
+        Ok(())
+    }
+
+    /// Retry connectivity check with exponential backoff + jitter instead of
+    /// failing hard on the first attempt.
+    ///
+    /// Uses a separate `max_open_retries` / `open_retry_max_delay` so startup
+    /// can wait patiently for InfluxDB without affecting the per-write retry
+    /// parameters used by the middleware during normal operation.
+    async fn check_connectivity_with_retry(&self, client: &reqwest::Client) -> 
Result<(), Error> {
+        let max_open_retries = self
+            .config
+            .max_open_retries
+            .unwrap_or(DEFAULT_MAX_OPEN_RETRIES)
+            .max(1);
+
+        let max_delay = parse_duration(
+            self.config.open_retry_max_delay.as_deref(),
+            DEFAULT_OPEN_RETRY_MAX_DELAY,
+        );
+
+        let mut attempt = 0u32;
+        loop {
+            match self.check_connectivity(client).await {
+                Ok(()) => {
+                    if attempt > 0 {
+                        info!(
+                            "InfluxDB connectivity established after {attempt} 
retries \
+                             for sink connector ID: {}",
+                            self.id
+                        );
+                    }
+                    return Ok(());
+                }
+                Err(e) => {
+                    attempt += 1;
+                    if attempt >= max_open_retries {
+                        error!(
+                            "InfluxDB connectivity check failed after 
{attempt} attempts \
+                             for sink connector ID: {}. Giving up: {e}",
+                            self.id
+                        );
+                        return Err(e);
+                    }
+                    // Exponential backoff, with jitter
+                    let backoff = jitter(exponential_backoff(self.retry_delay, 
attempt, max_delay));
+                    warn!(
+                        "InfluxDB health check failed (attempt 
{attempt}/{max_open_retries}) \
+                         for sink connector ID: {}. Retrying in {backoff:?}: 
{e}",
+                        self.id
+                    );
+                    tokio::time::sleep(backoff).await;
+                }
+            }
+        }
+    }
+
+    fn get_client(&self) -> Result<&ClientWithMiddleware, Error> {
+        self.client
+            .as_ref()
+            .ok_or_else(|| Error::Connection("InfluxDB client is not 
initialized".to_string()))
+    }
+
+    fn measurement(&self) -> &str {
+        self.config
+            .measurement
+            .as_deref()
+            .unwrap_or("iggy_messages")
+    }
+
+    fn payload_format(&self) -> PayloadFormat {
+        PayloadFormat::from_config(self.config.payload_format.as_deref())
+    }
+
+    fn timestamp_precision(&self) -> &str {
+        self.config
+            .precision
+            .as_deref()
+            .unwrap_or(DEFAULT_PRECISION)
+    }
+
+    fn get_max_retries(&self) -> u32 {
+        self.config
+            .max_retries
+            .unwrap_or(DEFAULT_MAX_RETRIES)
+            .max(1)
+    }
+
+    fn to_precision_timestamp(&self, micros: u64) -> u64 {
+        match self.timestamp_precision() {
+            "ns" => micros.saturating_mul(1_000),
+            "us" => micros,
+            "ms" => micros / 1_000,
+            "s" => micros / 1_000_000,
+            _ => micros,
+        }
+    }
+
+    /// Serialise one message as a line-protocol line, appending directly into
+    /// `buf` with no intermediate `Vec<String>` for tags or fields.
+    ///
+    /// # Allocation budget (per message, happy path)
+    /// - Zero `Vec` allocations for tags or fields.
+    /// - Zero per-tag/per-field `format!` allocations.
+    /// - One `Vec<u8>` for `payload_bytes` (unavoidable — payload must be
+    ///   decoded/serialised before it can be escaped into the buffer).
+    /// - The caller's `buf` grows in place; if it was pre-allocated with
+    ///   `with_capacity` it will not reallocate for typical message sizes.
+    fn append_line(
+        &self,
+        buf: &mut String,
+        topic_metadata: &TopicMetadata,
+        messages_metadata: &MessagesMetadata,
+        message: &ConsumedMessage,
+    ) -> Result<(), Error> {
+        let include_metadata = self.config.include_metadata.unwrap_or(true);
+        let include_checksum = self.config.include_checksum.unwrap_or(true);
+        let include_origin_timestamp = 
self.config.include_origin_timestamp.unwrap_or(true);
+        let include_stream_tag = 
self.config.include_stream_tag.unwrap_or(true);
+        let include_topic_tag = self.config.include_topic_tag.unwrap_or(true);
+        let include_partition_tag = 
self.config.include_partition_tag.unwrap_or(true);
+
+        // ── Measurement 
──────────────────────────────────────────────────────
+        write_measurement(buf, self.measurement());
+
+        // ── Tag set 
──────────────────────────────────────────────────────────
+        // Tags are written as ",key=value" pairs directly into buf.
+        // The offset tag is always present — it makes every point unique in
+        // InfluxDB's deduplication key (measurement + tag set + timestamp),
+        // regardless of precision or how many messages share a timestamp.
+        if include_metadata && include_stream_tag {
+            buf.push_str(",stream=");
+            write_tag_value(buf, &topic_metadata.stream);
+        }
+        if include_metadata && include_topic_tag {
+            buf.push_str(",topic=");
+            write_tag_value(buf, &topic_metadata.topic);
+        }
+        if include_metadata && include_partition_tag {
+            use std::fmt::Write as _;
+            write!(buf, ",partition={}", messages_metadata.partition_id)
+                .expect("write to String is infallible");
+        }
+        // offset tag — always written, ensures point uniqueness
+        {
+            use std::fmt::Write as _;
+            write!(buf, ",offset={}", message.offset).expect("write to String 
is infallible");
+        }
+
+        // ── Field set 
────────────────────────────────────────────────────────
+        // First field: no leading comma.  All subsequent fields: leading 
comma.
+        buf.push(' ');
+
+        buf.push_str("message_id=\"");
+        write_field_string(buf, &message.id.to_string());
+        buf.push('"');
+
+        // offset as a numeric field (queryable in Flux) in addition to the tag
+        {
+            use std::fmt::Write as _;
+            write!(buf, ",offset={}u", message.offset).expect("write to String 
is infallible");
+        }
+
+        // Optional metadata fields written when the corresponding tag is
+        // disabled (so the value is still queryable as a field).
+        if include_metadata && !include_stream_tag {
+            buf.push_str(",iggy_stream=\"");
+            write_field_string(buf, &topic_metadata.stream);
+            buf.push('"');
+        }
+        if include_metadata && !include_topic_tag {
+            buf.push_str(",iggy_topic=\"");
+            write_field_string(buf, &topic_metadata.topic);
+            buf.push('"');
+        }
+        if include_metadata && !include_partition_tag {
+            use std::fmt::Write as _;
+            write!(
+                buf,
+                ",iggy_partition={}u",
+                messages_metadata.partition_id as u64
+            )
+            .expect("write to String is infallible");
+        }
+        if include_checksum {
+            use std::fmt::Write as _;
+            write!(buf, ",iggy_checksum={}u", message.checksum)
+                .expect("write to String is infallible");
+        }
+        if include_origin_timestamp {
+            use std::fmt::Write as _;
+            write!(buf, ",iggy_origin_timestamp={}u", message.origin_timestamp)
+                .expect("write to String is infallible");
+        }
+
+        // ── Payload field 
────────────────────────────────────────────────────
+        // try_as_bytes() serialises in-place without cloning the Payload tree.
+        let payload_bytes = message.payload.try_as_bytes().map_err(|e| {
+            Error::CannotStoreData(format!("Failed to convert payload to 
bytes: {e}"))
+        })?;
+
+        match self.payload_format() {
+            PayloadFormat::Json => {
+                let value: serde_json::Value =
+                    serde_json::from_slice(&payload_bytes).map_err(|e| {
+                        Error::CannotStoreData(format!(
+                            "Payload format is json but payload is invalid 
JSON: {e}"
+                        ))
+                    })?;
+                let compact = serde_json::to_string(&value).map_err(|e| {
+                    Error::CannotStoreData(format!("Failed to serialize JSON 
payload: {e}"))
+                })?;
+                buf.push_str(",payload_json=\"");
+                write_field_string(buf, &compact);
+                buf.push('"');
+            }
+            PayloadFormat::Text => {
+                let text = String::from_utf8(payload_bytes).map_err(|e| {
+                    Error::CannotStoreData(format!(
+                        "Payload format is text but payload is invalid UTF-8: 
{e}"
+                    ))
+                })?;
+                buf.push_str(",payload_text=\"");
+                write_field_string(buf, &text);
+                buf.push('"');
+            }
+            PayloadFormat::Base64 => {
+                let encoded = general_purpose::STANDARD.encode(&payload_bytes);
+                buf.push_str(",payload_base64=\"");
+                write_field_string(buf, &encoded);
+                buf.push('"');
+            }
+        }
+
+        // ── Timestamp 
────────────────────────────────────────────────────────
+        // message.timestamp is microseconds since Unix epoch.
+        // Fall back to now() when unset (0) so points are not stored at the
+        // Unix epoch (year 1970), which falls outside every range(start:-1h).
+        let base_micros = if message.timestamp == 0 {
+            SystemTime::now()
+                .duration_since(UNIX_EPOCH)
+                .unwrap_or_default()
+                .as_micros() as u64
+        } else {
+            message.timestamp
+        };
+        let ts = self.to_precision_timestamp(base_micros);
+
+        {
+            use std::fmt::Write as _;
+            write!(buf, " {ts}").expect("write to String is infallible");
+        }
+
+        debug!(
+            "InfluxDB sink ID: {} point — offset={}, raw_ts={}, 
influx_ts={ts}",
+            self.id, message.offset, message.timestamp
+        );
+
+        Ok(())
+    }
+
+    async fn process_batch(
+        &self,
+        topic_metadata: &TopicMetadata,
+        messages_metadata: &MessagesMetadata,
+        messages: &[ConsumedMessage],
+    ) -> Result<(), Error> {
+        if messages.is_empty() {
+            return Ok(());
+        }
+
+        // Single buffer for the entire batch — reused across all messages.
+        // Pre-allocate a generous estimate (256 bytes per message) to avoid
+        // reallocation in the common case.  The buffer is passed into
+        // append_line() which writes each line directly, with '\n' separators
+        // between lines.  No per-message String is allocated.
+        let mut body = String::with_capacity(messages.len() * 256);
+
+        for (i, message) in messages.iter().enumerate() {
+            if i > 0 {
+                body.push('\n');
+            }
+            self.append_line(&mut body, topic_metadata, messages_metadata, 
message)?;
+        }
+
+        let client = self.get_client()?;
+        let url = self.write_url.clone().ok_or_else(|| {
+            Error::Connection("write_url not initialised — was open() 
called?".to_string())
+        })?;
+        let token = self.config.token.clone();
+
+        // Convert once before sending — Bytes is reference-counted so any
+        // retry inside the middleware clones the pointer, not the payload 
data.
+        let body: Bytes = Bytes::from(body);
+
+        let response = client
+            .post(url)
+            .header("Authorization", format!("Token {token}"))
+            .header("Content-Type", "text/plain; charset=utf-8")
+            .body(body)
+            .send()
+            .await
+            .map_err(|e| Error::CannotStoreData(format!("InfluxDB write 
failed: {e}")))?;
+
+        let status = response.status();
+        if status.is_success() {
+            return Ok(());
+        }
+
+        let body_text = response
+            .text()
+            .await
+            .unwrap_or_else(|_| "failed to read response body".to_string());
+        Err(Error::CannotStoreData(format!(
+            "InfluxDB write failed with status {status}: {body_text}"
+        )))
+    }
+}
+
+// ---------------------------------------------------------------------------
+// Sink trait implementation
+// ---------------------------------------------------------------------------
+
+#[async_trait]
+impl Sink for InfluxDbSink {
+    async fn open(&mut self) -> Result<(), Error> {
+        info!(
+            "Opening InfluxDB sink connector with ID: {}. Bucket: {}, org: {}",
+            self.id, self.config.bucket, self.config.org
+        );
+
+        // Build the raw client first and use it for the startup connectivity
+        // check. The connectivity retry loop uses separate delay bounds
+        // (open_retry_max_delay) from the per-write middleware retries, so
+        // we keep them independent rather than routing health checks through
+        // the write-tuned middleware.
+        let raw_client = self.build_raw_client()?;
+        self.check_connectivity_with_retry(&raw_client).await?;
+
+        // Wrap in the retry middleware for all subsequent write operations.
+        // The middleware handles transient 429 / 5xx retries with
+        // exponential back-off, jitter, and Retry-After header support.
+        let max_retries = self.get_max_retries();
+        let write_retry_max_delay = parse_duration(
+            self.config.retry_max_delay.as_deref(),
+            DEFAULT_RETRY_MAX_DELAY,
+        );
+        self.client = Some(build_retry_client(
+            raw_client,
+            max_retries,
+            self.retry_delay,
+            write_retry_max_delay,
+        ));
+
+        // Cache once — both are derived purely from config fields that
+        // never change at runtime.
+        self.write_url = Some(self.build_write_url()?);
+
+        info!(
+            "InfluxDB sink connector with ID: {} opened successfully",
+            self.id
+        );
+        Ok(())
+    }
+
+    async fn consume(
+        &self,
+        topic_metadata: &TopicMetadata,
+        messages_metadata: MessagesMetadata,
+        messages: Vec<ConsumedMessage>,
+    ) -> Result<(), Error> {
+        let batch_size = self.config.batch_size.unwrap_or(500) as usize;
+        let total_messages = messages.len();
+
+        // Skip writes entirely if circuit breaker is open
+        if self.circuit_breaker.is_open().await {
+            warn!(
+                "InfluxDB sink ID: {} — circuit breaker is OPEN. \
+                 Skipping {} messages to avoid hammering a down InfluxDB.",
+                self.id, total_messages
+            );
+            // Return an error so the runtime knows messages were not written
+            return Err(Error::CannotStoreData(
+                "Circuit breaker is open — InfluxDB write skipped".to_string(),
+            ));
+        }
+
+        // Collect the first batch error rather than silently dropping
+        let mut first_error: Option<Error> = None;
+
+        for batch in messages.chunks(batch_size.max(1)) {
+            match self
+                .process_batch(topic_metadata, &messages_metadata, batch)
+                .await
+            {
+                Ok(()) => {
+                    // Successful write — reset circuit breaker
+                    self.circuit_breaker.record_success();
+                    self.write_success
+                        .fetch_add(batch.len() as u64, Ordering::Relaxed);
+                }
+                Err(e) => {
+                    // Failed write — notify circuit breaker
+                    self.circuit_breaker.record_failure().await;

Review Comment:
   `record_failure()` is called for all errors including non-transient ones 
like 400 Bad Request (malformed line protocol) or 422 (schema conflict). the 
retry middleware already handles transient errors (429, 5xx), so errors that 
reach here are likely permanent data issues. counting them toward the circuit 
breaker threshold means a burst of bad data can open the circuit and block 
delivery of valid messages in subsequent `consume()` calls. consider only 
calling `record_failure()` for transient/connectivity errors.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to