hubcio commented on code in PR #2933: URL: https://github.com/apache/iggy/pull/2933#discussion_r2988038737
########## core/connectors/sinks/influxdb_sink/src/lib.rs: ########## @@ -0,0 +1,1067 @@ +/* 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: `\` → `\\`, `,` → `\,`, ` ` → `\ ` +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) { Review Comment: `write_field_string` escapes `\` and `"` but does not escape `\n` (newline). in influxdb line protocol, newline is the record delimiter - the write API splits on `\n` boundaries *before* parsing individual lines. if any json payload, text payload, or string field value contains a literal newline, it will split one line-protocol record into two malformed fragments. the batch write will either be rejected or - worse - silently produce a corrupt point with garbage measurement/tag/field structure. fix: add `'\n' => buf.push_str("\\n"),` (and `'\r' => buf.push_str("\\r"),` for completeness) to the match arms. ########## core/connectors/sources/influxdb_source/src/lib.rs: ########## @@ -0,0 +1,1166 @@ +/* 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, +} + +// --------------------------------------------------------------------------- +// 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") +} + +/// 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, + 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_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 get_client(&self) -> Result<&ClientWithMiddleware, 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) + } + + /// Single connectivity probe using the provided raw client (no retry). + /// The caller (`check_connectivity_with_retry`) owns the outer retry loop, + /// which uses different delay bounds than per-query 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 separate `max_open_retries` / `open_retry_max_delay` so startup + /// can wait patiently for InfluxDB without affecting the per-query 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 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) { + 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) + } + + /// Execute a Flux query against `/api/v2/query` and return the raw CSV + /// response body. Retry/back-off is handled transparently by the + /// `ClientWithMiddleware` stack (see `build_retry_client`). + async fn run_query(&self, query: &str) -> Result<String, Error> { + let client = self.get_client()?; + let url = self.build_query_url()?; + let token = self.config.token.clone(); + + let body = json!({ + "query": query, + "dialect": { + "annotations": [], + "delimiter": ",", + "header": true, + "commentPrefix": "#" + } + }); + + let response = client + .post(url) + .header("Authorization", format!("Token {token}")) + .header("Content-Type", "application/json") + .header("Accept", "text/csv") + .json(&body) + .send() + .await + .map_err(|e| Error::Storage(format!("InfluxDB query failed: {e}")))?; + + let status = response.status(); + if status.is_success() { + return response + .text() + .await + .map_err(|e| Error::Storage(format!("Failed to read query response: {e}"))); + } + + let body_text = response + .text() + .await + .unwrap_or_else(|_| "failed to read response body".to_string()); + Err(Error::Storage(format!( + "InfluxDB query failed with status {status}: {body_text}" + ))) + } + + fn parse_csv_rows(&self, csv_text: &str) -> Result<Vec<HashMap<String, String>>, Error> { + let mut reader = csv::ReaderBuilder::new() + .has_headers(false) + .from_reader(csv_text.as_bytes()); + + let mut headers: Option<StringRecord> = None; + let mut rows = Vec::new(); + + for result in reader.records() { + let record = result + .map_err(|e| Error::InvalidRecordValue(format!("Invalid CSV record: {e}")))?; + + if record.is_empty() { + continue; + } + + if let Some(first) = record.get(0) + && first.starts_with('#') + { + continue; + } + + if is_header_record(&record) { + headers = Some(record.clone()); + continue; + } + + let Some(active_headers) = headers.as_ref() else { + continue; + }; + + if record == *active_headers { + continue; + } + + let mut mapped = HashMap::new(); + for (idx, key) in active_headers.iter().enumerate() { + if key.is_empty() { + continue; + } + let value = record.get(idx).unwrap_or("").to_string(); + mapped.insert(key.to_string(), value); + } + + if !mapped.is_empty() { + rows.push(mapped); + } + } + + Ok(rows) + } + + fn build_payload( + &self, + row: &HashMap<String, String>, + include_metadata: bool, + ) -> Result<Vec<u8>, Error> { + if let Some(payload_column) = self.config.payload_column.as_deref() { + let raw_value = row.get(payload_column).cloned().ok_or_else(|| { + Error::InvalidRecordValue(format!("Missing payload column '{payload_column}'")) + })?; + + return match self.payload_format() { + PayloadFormat::Json => { + let value: serde_json::Value = + serde_json::from_str(&raw_value).map_err(|e| { + Error::InvalidRecordValue(format!( + "Payload column '{payload_column}' is not valid JSON: {e}" + )) + })?; + serde_json::to_vec(&value).map_err(|e| { + Error::Serialization(format!("JSON serialization failed: {e}")) + }) + } + PayloadFormat::Text => Ok(raw_value.into_bytes()), + PayloadFormat::Raw => general_purpose::STANDARD + .decode(raw_value.as_bytes()) + .map_err(|e| { + Error::InvalidRecordValue(format!( + "Failed to decode payload as base64: {e}" + )) + }), + }; + } + + let mut json_row = serde_json::Map::new(); + for (key, value) in row { + if include_metadata || key == "_value" || key == "_time" || key == "_measurement" { + json_row.insert(key.clone(), parse_scalar(value)); + } + } + + let wrapped = json!({ + "measurement": row.get("_measurement").cloned().unwrap_or_default(), + "field": row.get("_field").cloned().unwrap_or_default(), + "timestamp": row.get("_time").cloned().unwrap_or_default(), + "value": row.get("_value").map(|v| parse_scalar(v)).unwrap_or(serde_json::Value::Null), + "row": json_row, + }); + + serde_json::to_vec(&wrapped) + .map_err(|e| Error::Serialization(format!("JSON serialization failed: {e}"))) + } + + async fn poll_messages(&self) -> Result<(Vec<ProducedMessage>, Option<String>), Error> { + let cursor = self.current_cursor().await; + let query = self.query_with_params(&cursor).map_err(|e| { + error!( + "InfluxDB source ID: {} — invalid cursor, skipping poll: {e}", + self.id + ); + e + })?; + let csv_data = self.run_query(&query).await?; + + let rows = self.parse_csv_rows(&csv_data)?; + let include_metadata = self.config.include_metadata.unwrap_or(true); + let cursor_field = self.cursor_field().to_string(); + + let mut messages = Vec::with_capacity(rows.len()); + let mut max_cursor: Option<String> = None; + + for row in rows { + if let Some(cursor_value) = row.get(&cursor_field) + && max_cursor + .as_ref() + .is_none_or(|current| is_timestamp_after(cursor_value, current)) Review Comment: cursor advancement uses `is_timestamp_after` (strict `>`) to find the max `_time` in the batch, and the example query templates use `r._time > time(v: "$cursor")` (also strict `>`). this means rows with `_time == cursor` are excluded from the next poll. if multiple rows share the exact same max `_time` and the batch limit truncates before all same-timestamp rows are fetched, the remaining rows at that timestamp are permanently skipped - classic cursor-boundary data loss. mitigations: use `>=` in the query and deduplicate already-seen rows by tracking the last-seen offset count at the cursor timestamp, or document the at-most-once limitation clearly so operators know to size batches larger than any expected timestamp collision. ########## core/connectors/sources/influxdb_source/src/lib.rs: ########## @@ -0,0 +1,1166 @@ +/* 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, +} + +// --------------------------------------------------------------------------- +// 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") +} + +/// 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, + 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_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 get_client(&self) -> Result<&ClientWithMiddleware, 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) + } + + /// Single connectivity probe using the provided raw client (no retry). + /// The caller (`check_connectivity_with_retry`) owns the outer retry loop, + /// which uses different delay bounds than per-query 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 separate `max_open_retries` / `open_retry_max_delay` so startup + /// can wait patiently for InfluxDB without affecting the per-query 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 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) { + 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) + } + + /// Execute a Flux query against `/api/v2/query` and return the raw CSV + /// response body. Retry/back-off is handled transparently by the + /// `ClientWithMiddleware` stack (see `build_retry_client`). + async fn run_query(&self, query: &str) -> Result<String, Error> { + let client = self.get_client()?; + let url = self.build_query_url()?; + let token = self.config.token.clone(); + + let body = json!({ + "query": query, + "dialect": { + "annotations": [], + "delimiter": ",", + "header": true, + "commentPrefix": "#" + } + }); + + let response = client + .post(url) + .header("Authorization", format!("Token {token}")) + .header("Content-Type", "application/json") + .header("Accept", "text/csv") + .json(&body) + .send() + .await + .map_err(|e| Error::Storage(format!("InfluxDB query failed: {e}")))?; + + let status = response.status(); + if status.is_success() { + return response + .text() + .await + .map_err(|e| Error::Storage(format!("Failed to read query response: {e}"))); + } + + let body_text = response + .text() + .await + .unwrap_or_else(|_| "failed to read response body".to_string()); + Err(Error::Storage(format!( + "InfluxDB query failed with status {status}: {body_text}" + ))) + } + + fn parse_csv_rows(&self, csv_text: &str) -> Result<Vec<HashMap<String, String>>, Error> { + let mut reader = csv::ReaderBuilder::new() + .has_headers(false) + .from_reader(csv_text.as_bytes()); + + let mut headers: Option<StringRecord> = None; + let mut rows = Vec::new(); + + for result in reader.records() { + let record = result + .map_err(|e| Error::InvalidRecordValue(format!("Invalid CSV record: {e}")))?; + + if record.is_empty() { + continue; + } + + if let Some(first) = record.get(0) + && first.starts_with('#') + { + continue; + } + + if is_header_record(&record) { + headers = Some(record.clone()); + continue; + } + + let Some(active_headers) = headers.as_ref() else { + continue; + }; + + if record == *active_headers { + continue; + } + + let mut mapped = HashMap::new(); + for (idx, key) in active_headers.iter().enumerate() { + if key.is_empty() { + continue; + } + let value = record.get(idx).unwrap_or("").to_string(); + mapped.insert(key.to_string(), value); + } + + if !mapped.is_empty() { + rows.push(mapped); + } + } + + Ok(rows) + } + + fn build_payload( + &self, + row: &HashMap<String, String>, + include_metadata: bool, + ) -> Result<Vec<u8>, Error> { + if let Some(payload_column) = self.config.payload_column.as_deref() { + let raw_value = row.get(payload_column).cloned().ok_or_else(|| { + Error::InvalidRecordValue(format!("Missing payload column '{payload_column}'")) + })?; + + return match self.payload_format() { + PayloadFormat::Json => { + let value: serde_json::Value = + serde_json::from_str(&raw_value).map_err(|e| { + Error::InvalidRecordValue(format!( + "Payload column '{payload_column}' is not valid JSON: {e}" + )) + })?; + serde_json::to_vec(&value).map_err(|e| { + Error::Serialization(format!("JSON serialization failed: {e}")) + }) + } + PayloadFormat::Text => Ok(raw_value.into_bytes()), + PayloadFormat::Raw => general_purpose::STANDARD + .decode(raw_value.as_bytes()) + .map_err(|e| { + Error::InvalidRecordValue(format!( + "Failed to decode payload as base64: {e}" + )) + }), + }; + } + + let mut json_row = serde_json::Map::new(); + for (key, value) in row { + if include_metadata || key == "_value" || key == "_time" || key == "_measurement" { + json_row.insert(key.clone(), parse_scalar(value)); + } + } + + let wrapped = json!({ + "measurement": row.get("_measurement").cloned().unwrap_or_default(), + "field": row.get("_field").cloned().unwrap_or_default(), + "timestamp": row.get("_time").cloned().unwrap_or_default(), + "value": row.get("_value").map(|v| parse_scalar(v)).unwrap_or(serde_json::Value::Null), + "row": json_row, + }); + + serde_json::to_vec(&wrapped) + .map_err(|e| Error::Serialization(format!("JSON serialization failed: {e}"))) + } + + async fn poll_messages(&self) -> Result<(Vec<ProducedMessage>, Option<String>), Error> { + let cursor = self.current_cursor().await; + let query = self.query_with_params(&cursor).map_err(|e| { + error!( + "InfluxDB source ID: {} — invalid cursor, skipping poll: {e}", + self.id + ); + e + })?; + let csv_data = self.run_query(&query).await?; + + let rows = self.parse_csv_rows(&csv_data)?; + let include_metadata = self.config.include_metadata.unwrap_or(true); + let cursor_field = self.cursor_field().to_string(); + + let mut messages = Vec::with_capacity(rows.len()); + let mut max_cursor: Option<String> = None; + + for row in rows { + if let Some(cursor_value) = row.get(&cursor_field) + && max_cursor + .as_ref() + .is_none_or(|current| is_timestamp_after(cursor_value, current)) + { + max_cursor = Some(cursor_value.clone()); + } + + let payload = self.build_payload(&row, include_metadata)?; + // Capture once so timestamp and origin_timestamp are guaranteed identical + // and we make exactly one syscall regardless of how many fields use it. + let now_micros = Utc::now().timestamp_micros() as u64; + + messages.push(ProducedMessage { + id: Some(Uuid::new_v4().as_u128()), + checksum: None, + timestamp: Some(now_micros), + origin_timestamp: Some(now_micros), + headers: None, + payload, + }); + } + + Ok((messages, max_cursor)) + } +} + +// --------------------------------------------------------------------------- +// Source trait implementation +// --------------------------------------------------------------------------- + +#[async_trait] +impl Source for InfluxDbSource { + async fn open(&mut self) -> Result<(), Error> { + info!( + "Opening InfluxDB source connector with ID: {}. Org: {}", + self.id, 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-query middleware retries, so + // we keep them independent. + let raw_client = self.build_raw_client()?; + + // Validate cursor_field before touching the network: string comparison + // is only safe for timestamp columns. See validate_cursor_field for details. + Self::validate_cursor_field(self.cursor_field())?; + + self.check_connectivity_with_retry(&raw_client).await?; + + // Wrap in the retry middleware for all subsequent query 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 query_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, + query_retry_max_delay, + )); + + info!( + "InfluxDB source connector with ID: {} opened successfully", + self.id + ); + Ok(()) + } + + async fn poll(&self) -> Result<ProducedMessages, Error> { + // Skip query if circuit breaker is open + if self.circuit_breaker.is_open().await { + warn!( + "InfluxDB source ID: {} — circuit breaker is OPEN. Skipping poll.", + self.id + ); + return Ok(ProducedMessages { Review Comment: when the circuit breaker is open, `poll()` returns `Ok(ProducedMessages { messages: vec![], .. })` with no sleep. the connector runtime sees a successful empty poll and immediately calls `poll()` again, creating an unbounded cpu spin-loop for the entire circuit breaker cool-down period. `is_open()` acquires a tokio mutex on each iteration, adding contention on top of the cpu burn. add `tokio::time::sleep(self.poll_interval).await;` before the return so the source backs off while the circuit is open. alternatively, return `Err(...)` to match the sink's behavior (sink returns `Err(CannotStoreData(...))` at its equivalent check). -- 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]
