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


##########
core/connectors/sdk/src/retry.rs:
##########
@@ -0,0 +1,159 @@
+/* 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
+//! - [`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 humantime::Duration as HumanDuration;
+use rand::RngExt as _;
+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 {

Review Comment:
   37.87% patch coverage - every function here is pure or near-pure and 
trivially unit-testable. needs a `#[cfg(test)] mod tests` block covering: 
`CircuitBreaker` state transitions (closed -> open -> half-open -> closed), 
`exponential_backoff` scaling + saturation + max_delay clamping, `jitter` range 
bounds + small-duration no-op, `parse_duration` valid/invalid/None, 
`parse_retry_after` integer seconds + invalid input. ~25 tests, zero external 
deps.



##########
core/connectors/sinks/influxdb_sink/src/lib.rs:
##########
@@ -0,0 +1,780 @@
+/* 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, exponential_backoff, jitter, parse_duration, 
parse_retry_after,
+};
+use iggy_connector_sdk::{
+    ConsumedMessage, Error, MessagesMetadata, Sink, TopicMetadata, 
sink_connector,
+};
+use reqwest::{Client, StatusCode, Url};
+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,
+    client: Option<Client>,
+    /// Cached once in `open()` — config fields never change at runtime.
+    write_url: Option<Url>,
+    /// Cached once in `open()` — parsed from `config.retry_max_delay`.
+    /// Controls the backoff cap for per-write retries only; startup retries
+    /// use `config.open_retry_max_delay` and are never affected by this field.
+    write_retry_max_delay: Duration,
+    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
+// ---------------------------------------------------------------------------
+
+fn is_transient_status(status: StatusCode) -> bool {
+    status == StatusCode::TOO_MANY_REQUESTS || status.is_server_error()
+}
+
+/// Write an escaped measurement name into `buf`.
+/// Escapes: `\` → `\\`, `,` → `\,`, ` ` → `\ `
+fn write_measurement(buf: &mut String, value: &str) {
+    for ch in value.chars() {
+        match ch {
+            '\\' => buf.push_str("\\\\"),
+            ',' => buf.push_str("\\,"),
+            ' ' => buf.push_str("\\ "),
+            _ => buf.push(ch),
+        }
+    }
+}
+
+/// Write an escaped tag key/value into `buf`.
+/// Escapes: `\` → `\\`, `,` → `\,`, `=` → `\=`, ` ` → `\ `
+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("\\ "),
+            _ => buf.push(ch),
+        }
+    }
+}
+
+/// Write an escaped string field value (without surrounding quotes) into 
`buf`.
+/// Escapes: `\` → `\\`, `"` → `\"`
+fn write_field_string(buf: &mut String, value: &str) {
+    for ch in value.chars() {
+        match ch {
+            '\\' => buf.push_str("\\\\"),
+            '"' => buf.push_str("\\\""),
+            _ => 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,
+            write_retry_max_delay: Duration::from_secs(0), // overwritten in 
open()
+            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_client(&self) -> Result<Client, Error> {
+        let timeout = parse_duration(self.config.timeout.as_deref(), 
DEFAULT_TIMEOUT);
+        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}")))
+    }
+
+    async fn check_connectivity(&self) -> Result<(), Error> {
+        let client = self.get_client()?;
+        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.
+    async fn check_connectivity_with_retry(&self) -> 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().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<&Client, 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)?;
+        }
+
+        self.write_with_retry(body).await
+    }
+
+    async fn write_with_retry(&self, body: String) -> Result<(), Error> {

Review Comment:
   `write_with_retry()` is a hand-rolled retry loop duplicating what 
`reqwest_retry::RetryTransientMiddleware` already does - transient status 
detection, exponential backoff, jitter. the connector runtime uses 
`reqwest-middleware` for this exact pattern - see `http_provider.rs:82-101`. 
wrapping `reqwest::Client` in `ClientWithMiddleware` would eliminate this loop 
and the source's `run_query_with_retry` equivalent entirely. the circuit 
breaker and Retry-After parsing can be a custom `reqwest_retry::RetryPolicy`.



##########
core/connectors/sdk/src/retry.rs:
##########
@@ -0,0 +1,159 @@
+/* Licensed to the Apache Software Foundation (ASF) under one

Review Comment:
   the connector runtime already uses `reqwest-middleware` + `reqwest-retry` 
(`ExponentialBackoff`, `RetryTransientMiddleware`, `Jitter::Bounded`) in 
`http_provider.rs`. this PR hand-rolls ~160 lines of 
retry/backoff/jitter/circuit-breaker instead of reusing that infrastructure. 
suggest using `reqwest-middleware` with a custom retry policy that handles 
Retry-After and circuit breaking - it would eliminate `write_with_retry()` in 
the sink and `run_query_with_retry()` in the source entirely, and keep the 
connector crates consistent with the runtime.



##########
core/connectors/sources/influxdb_source/src/lib.rs:
##########
@@ -0,0 +1,803 @@
+/* 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, exponential_backoff, jitter, parse_duration, 
parse_retry_after,
+};
+use iggy_connector_sdk::{
+    ConnectorState, Error, ProducedMessage, ProducedMessages, Schema, Source, 
source_connector,
+};
+use regex::Regex;
+use reqwest::{Client, StatusCode, Url};
+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,
+    client: Option<Client>,
+    state: Mutex<State>,
+    verbose: bool,
+    retry_delay: Duration,
+    poll_interval: Duration,
+    /// Cached once in `open()` — parsed from `config.retry_max_delay`.
+    /// Controls the backoff cap for per-query retries only; startup retries
+    /// use `config.open_retry_max_delay` and are never affected by this field.
+    query_retry_max_delay: 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,
+}
+
+// ---------------------------------------------------------------------------
+// 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 {
+    record.iter().any(|v| v == "_time") && record.iter().any(|v| v == "_value")
+}
+
+fn is_transient_status(status: StatusCode) -> bool {
+    status == StatusCode::TOO_MANY_REQUESTS || status.is_server_error()
+}
+
+/// Compare two RFC 3339 timestamp strings chronologically.
+///
+/// InfluxDB strips trailing fractional-second zeros, producing timestamps like
+/// `"2026-03-18T12:00:00.60952Z"` (= 609520µs).  A naïve `>` string comparison
+/// treats this as *greater* than `"2026-03-18T12:00:00.609521Z"` because `'Z'`
+/// (ASCII 90) > `'1'` (ASCII 49), even though the former is chronologically
+/// *earlier*.  Always parse to `DateTime<Utc>` so the comparison is correct.
+fn is_timestamp_after(a: &str, b: &str) -> bool {
+    match (a.parse::<DateTime<Utc>>(), b.parse::<DateTime<Utc>>()) {
+        (Ok(dt_a), Ok(dt_b)) => dt_a > dt_b,
+        _ => a > b,
+    }
+}
+
+// ---------------------------------------------------------------------------
+// InfluxDbSource implementation
+// ---------------------------------------------------------------------------
+
+impl InfluxDbSource {
+    pub fn new(id: u32, config: InfluxDbSourceConfig, state: 
Option<ConnectorState>) -> Self {
+        let verbose = config.verbose_logging.unwrap_or(false);
+        let retry_delay = parse_duration(config.retry_delay.as_deref(), 
DEFAULT_RETRY_DELAY);
+        let poll_interval = parse_duration(config.poll_interval.as_deref(), 
DEFAULT_POLL_INTERVAL);
+
+        // 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,
+        );
+
+        let restored_state = state
+            .and_then(|s| s.deserialize::<State>(CONNECTOR_NAME, id))
+            .inspect(|s| {
+                info!(
+                    "Restored state for {CONNECTOR_NAME} connector with ID: 
{id}. \
+                     Last timestamp: {:?}, processed rows: {}",
+                    s.last_timestamp, s.processed_rows
+                );
+            });
+
+        InfluxDbSource {
+            id,
+            config,
+            client: None,
+            state: Mutex::new(restored_state.unwrap_or(State {
+                last_poll_time: Utc::now(),
+                last_timestamp: None,
+                processed_rows: 0,
+            })),
+            verbose,
+            retry_delay,
+            poll_interval,
+            query_retry_max_delay: Duration::from_secs(0), // overwritten in 
open()
+            circuit_breaker: Arc::new(CircuitBreaker::new(cb_threshold, 
cb_cool_down)),
+        }
+    }
+
+    fn serialize_state(&self, state: &State) -> Option<ConnectorState> {
+        ConnectorState::serialize(state, CONNECTOR_NAME, self.id)
+    }
+
+    fn payload_format(&self) -> PayloadFormat {
+        PayloadFormat::from_config(self.config.payload_format.as_deref())
+    }
+
+    fn cursor_field(&self) -> &str {
+        self.config.cursor_field.as_deref().unwrap_or("_time")
+    }
+
+    fn get_max_retries(&self) -> u32 {
+        self.config
+            .max_retries
+            .unwrap_or(DEFAULT_MAX_RETRIES)
+            .max(1)
+    }
+
+    fn build_client(&self) -> Result<Client, Error> {
+        let timeout = parse_duration(self.config.timeout.as_deref(), 
DEFAULT_TIMEOUT);
+        Client::builder()
+            .timeout(timeout)
+            .build()
+            .map_err(|e| Error::InitError(format!("Failed to create HTTP 
client: {e}")))
+    }
+
+    fn get_client(&self) -> Result<&Client, Error> {
+        self.client
+            .as_ref()
+            .ok_or_else(|| Error::Connection("InfluxDB client is not 
initialized".to_string()))
+    }
+
+    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}")))
+    }
+
+    fn build_query_url(&self) -> Result<Url, Error> {
+        let base = self.config.url.trim_end_matches('/');
+        let mut url = Url::parse(&format!("{base}/api/v2/query"))
+            .map_err(|e| Error::InvalidConfigValue(format!("Invalid InfluxDB 
URL: {e}")))?;
+        url.query_pairs_mut().append_pair("org", &self.config.org);
+        Ok(url)
+    }
+
+    async fn check_connectivity(&self) -> Result<(), Error> {
+        let client = self.get_client()?;
+        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.
+    async fn check_connectivity_with_retry(&self) -> 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().await {
+                Ok(()) => {
+                    if attempt > 0 {
+                        info!(
+                            "InfluxDB connectivity established after {attempt} 
retries \
+                             for connector ID: {}",
+                            self.id
+                        );
+                    }
+                    return Ok(());
+                }
+                Err(e) => {
+                    attempt += 1;
+                    if attempt >= max_open_retries {
+                        error!(
+                            "InfluxDB connectivity check failed after 
{attempt} attempts \
+                             for 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 connector ID: {}. Retrying in {backoff:?}: {e}",
+                        self.id
+                    );
+                    tokio::time::sleep(backoff).await;
+                }
+            }
+        }
+    }
+
+    async fn current_cursor(&self) -> String {
+        let state = self.state.lock().await;
+        state
+            .last_timestamp
+            .clone()
+            .or_else(|| self.config.initial_offset.clone())
+            .unwrap_or_else(|| DEFAULT_CURSOR.to_string())
+    }
+
+    fn cursor_re() -> &'static Regex {
+        CURSOR_RE.get_or_init(|| {
+            
Regex::new(r"^\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}(\.\d+)?(Z|[+-]\d{2}:\d{2})$")
+                .expect("hardcoded regex is valid")
+        })
+    }
+
+    fn validate_cursor(cursor: &str) -> Result<(), Error> {
+        if Self::cursor_re().is_match(cursor) {
+            // ← call on &Regex, not on OnceLock
+            Ok(())
+        } else {
+            Err(Error::InvalidConfigValue(format!(
+                "cursor value {:?} is not a valid RFC 3339 timestamp; \
+             refusing substitution to prevent Flux query injection",
+                cursor
+            )))
+        }
+    }
+
+    /// Reject cursor fields that would produce incorrect results.
+    ///
+    /// Cursor advancement compares values as `String`s (lexicographic order).
+    /// This is correct for ISO 8601 / RFC 3339 timestamps — the default
+    /// `cursor_field` of `"_time"` — because their fixed-width format makes
+    /// lexicographic and chronological order identical.
+    fn validate_cursor_field(field: &str) -> Result<(), Error> {
+        match field {
+            "_time" | "time" => Ok(()),
+            other => Err(Error::InvalidConfigValue(format!(
+                "cursor_field {:?} is not supported — cursor values are 
compared as strings \
+                 (lexicographic order), which is only correct for ISO 8601 
timestamp columns. \
+                 Use the default \"_time\" column, or omit cursor_field 
entirely.",
+                other
+            ))),
+        }
+    }
+
+    fn query_with_params(&self, cursor: &str) -> Result<String, Error> {
+        // Reject anything that is not a well-formed RFC 3339 timestamp.
+        // This prevents a crafted or corrupted _time value (e.g. containing
+        // Flux syntax like `") |> drop() //`) from being injected into the
+        // query string before it is sent to /api/v2/query.
+        // Note: InfluxDB OSS v2 does not support the `params` JSON field for
+        // parameterized queries (Cloud-only feature), so substitution is
+        // unavoidable for OSS — validation is the correct mitigation here.
+        Self::validate_cursor(cursor)?;
+        let limit = self.config.batch_size.unwrap_or(500).to_string();
+        let mut query = self.config.query.clone();
+        if query.contains("$cursor") {
+            query = query.replace("$cursor", cursor);
+        }
+        if query.contains("$limit") {
+            query = query.replace("$limit", &limit);
+        }
+        Ok(query)
+    }
+
+    async fn run_query_with_retry(&self, query: &str) -> Result<String, Error> 
{

Review Comment:
   61.74% patch coverage. untested critical paths: `run_query_with_retry` HTTP 
429/5xx/network-error retry branches, circuit breaker open-state skip in 
`poll`, `validate_cursor` rejection of Flux injection payloads, 
`validate_cursor_field` rejection of non-timestamp fields, `parse_csv_rows` 
with blank lines / `#` annotations / multi-table results / missing columns, 
`build_payload` with missing `payload_column` / invalid base64 / invalid JSON, 
`close()` client cleanup.



##########
core/connectors/sinks/influxdb_sink/src/lib.rs:
##########
@@ -0,0 +1,780 @@
+/* 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, exponential_backoff, jitter, parse_duration, 
parse_retry_after,
+};
+use iggy_connector_sdk::{
+    ConsumedMessage, Error, MessagesMetadata, Sink, TopicMetadata, 
sink_connector,
+};
+use reqwest::{Client, StatusCode, Url};
+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,
+    client: Option<Client>,
+    /// Cached once in `open()` — config fields never change at runtime.
+    write_url: Option<Url>,
+    /// Cached once in `open()` — parsed from `config.retry_max_delay`.
+    /// Controls the backoff cap for per-write retries only; startup retries
+    /// use `config.open_retry_max_delay` and are never affected by this field.
+    write_retry_max_delay: Duration,
+    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
+// ---------------------------------------------------------------------------
+
+fn is_transient_status(status: StatusCode) -> bool {
+    status == StatusCode::TOO_MANY_REQUESTS || status.is_server_error()
+}
+
+/// Write an escaped measurement name into `buf`.
+/// Escapes: `\` → `\\`, `,` → `\,`, ` ` → `\ `
+fn write_measurement(buf: &mut String, value: &str) {
+    for ch in value.chars() {
+        match ch {
+            '\\' => buf.push_str("\\\\"),
+            ',' => buf.push_str("\\,"),
+            ' ' => buf.push_str("\\ "),
+            _ => buf.push(ch),
+        }
+    }
+}
+
+/// Write an escaped tag key/value into `buf`.
+/// Escapes: `\` → `\\`, `,` → `\,`, `=` → `\=`, ` ` → `\ `
+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("\\ "),
+            _ => buf.push(ch),
+        }
+    }
+}
+
+/// Write an escaped string field value (without surrounding quotes) into 
`buf`.
+/// Escapes: `\` → `\\`, `"` → `\"`
+fn write_field_string(buf: &mut String, value: &str) {
+    for ch in value.chars() {
+        match ch {
+            '\\' => buf.push_str("\\\\"),
+            '"' => buf.push_str("\\\""),
+            _ => 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,
+            write_retry_max_delay: Duration::from_secs(0), // overwritten in 
open()
+            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_client(&self) -> Result<Client, Error> {
+        let timeout = parse_duration(self.config.timeout.as_deref(), 
DEFAULT_TIMEOUT);
+        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}")))
+    }
+
+    async fn check_connectivity(&self) -> Result<(), Error> {
+        let client = self.get_client()?;
+        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.
+    async fn check_connectivity_with_retry(&self) -> 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().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<&Client, 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)?;
+        }
+
+        self.write_with_retry(body).await
+    }
+
+    async fn write_with_retry(&self, body: String) -> Result<(), Error> {
+        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 max_retries = self.get_max_retries();
+        let token = self.config.token.clone();
+
+        // Cap for per-write backoff — cached in open(), no re-parse needed.
+        let max_delay = self.write_retry_max_delay;
+
+        // Convert once before the loop — Bytes is a reference-counted view 
over
+        // the same allocation. Every .clone() inside the loop is a pointer 
bump
+        // (O(1)), not a deep copy of the string data (O(n)).
+        let body: Bytes = Bytes::from(body);
+
+        let mut attempts = 0u32;
+        loop {
+            let response_result = client
+                .post(url.clone())
+                .header("Authorization", format!("Token {token}"))
+                .header("Content-Type", "text/plain; charset=utf-8")
+                .body(body.clone())
+                .send()
+                .await;
+
+            match response_result {
+                Ok(response) => {
+                    let status = response.status();
+                    if status == StatusCode::NO_CONTENT || status == 
StatusCode::OK {
+                        return Ok(());
+                    }
+
+                    // Honour Retry-After on 429 before our own backoff
+                    let retry_after = if status == 
StatusCode::TOO_MANY_REQUESTS {
+                        response
+                            .headers()
+                            .get("Retry-After")
+                            .and_then(|v| v.to_str().ok())
+                            .and_then(parse_retry_after)
+                    } else {
+                        None
+                    };
+
+                    let body_text = response
+                        .text()
+                        .await
+                        .unwrap_or_else(|_| "failed to read response 
body".to_string());
+
+                    attempts += 1;
+                    if is_transient_status(status) && attempts < max_retries {
+                        // Use server-supplied delay when available
+                        let delay = retry_after.unwrap_or_else(|| {
+                            // Exponential, with jitter
+                            jitter(exponential_backoff(self.retry_delay, 
attempts, max_delay))
+                        });
+                        warn!(
+                            "Transient InfluxDB write error (attempt 
{attempts}/{max_retries}): \
+                             {status}. Retrying in {delay:?}..."
+                        );
+                        tokio::time::sleep(delay).await;
+                        continue;
+                    }
+
+                    return Err(Error::CannotStoreData(format!(
+                        "InfluxDB write failed with status {status}: 
{body_text}"
+                    )));
+                }
+                Err(e) => {
+                    attempts += 1;
+                    if attempts < max_retries {
+                        // Exponential, with jitter
+                        let delay =
+                            jitter(exponential_backoff(self.retry_delay, 
attempts, max_delay));
+                        warn!(
+                            "Failed to send write request to InfluxDB \
+                             (attempt {attempts}/{max_retries}): {e}. Retrying 
in {delay:?}..."
+                        );
+                        tokio::time::sleep(delay).await;
+                        continue;
+                    }
+
+                    return Err(Error::CannotStoreData(format!(
+                        "InfluxDB write failed after {attempts} attempts: {e}"
+                    )));
+                }
+            }
+        }
+    }
+}
+
+// ---------------------------------------------------------------------------
+// Sink trait implementation
+// ---------------------------------------------------------------------------
+
+#[async_trait]
+impl Sink for InfluxDbSink {

Review Comment:
   60.46% patch coverage. untested critical paths: `write_with_retry` HTTP 
429/5xx/network-error retry branches, circuit breaker open-state skip in 
`consume`, `check_connectivity_with_retry` failure loop, 
`to_precision_timestamp` for `"ms"` and `"s"` precision, `append_line` error 
paths (invalid JSON, invalid UTF-8), escaping edge cases (`=` in tags, `\` and 
`,` in measurement names), partial batch failure (first chunk fails, rest 
succeed).



-- 
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