hubcio commented on code in PR #2933: URL: https://github.com/apache/iggy/pull/2933#discussion_r2980192260
########## core/connectors/sources/influxdb_source/src/lib.rs: ########## @@ -0,0 +1,779 @@ +/* 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"; +// 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, + 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 + pub open_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, + _ => PayloadFormat::Json, Review Comment: source `PayloadFormat::from_config` silently falls through to `Json` on unrecognized input with no warning, while the sink correctly logs a `warn!()` (sink `lib.rs:124-131`). asymmetric fix from hubcio's earlier comment - should match the sink behavior. ########## core/connectors/runtime/example_config/connectors/influxdb_source.toml: ########## @@ -0,0 +1,54 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +type = "source" +key = "influxdb" +enabled = true +version = 0 +name = "InfluxDB source" +path = "target/release/libiggy_connector_influxdb_source" +plugin_config_format = "toml" +verbose = false + +[[streams]] +stream = "events" +topic = "influx_events" +schema = "json" +batch_length = 100 + +[plugin_config] +url = "http://localhost:8086" +org = "iggy" +token = "replace-with-token" Review Comment: this file uses `"replace_token_with_secret_token"` while the top-level `influxdb_source.toml` uses `"replace-with-token"` (different placeholder style). also unclear why both files exist - is the top-level one intentional or a leftover? ########## core/integration/tests/connectors/fixtures/influxdb/container.rs: ########## @@ -0,0 +1,238 @@ +/* + * 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 integration::harness::TestBinaryError; +use reqwest_middleware::ClientWithMiddleware as HttpClient; +use reqwest_retry::RetryTransientMiddleware; +use reqwest_retry::policies::ExponentialBackoff; +use testcontainers_modules::testcontainers::core::{IntoContainerPort, WaitFor}; +use testcontainers_modules::testcontainers::runners::AsyncRunner; +use testcontainers_modules::testcontainers::{ContainerAsync, GenericImage, ImageExt}; +use tracing::info; + +const INFLUXDB_IMAGE: &str = "influxdb"; +const INFLUXDB_TAG: &str = "2.7-alpine"; +const INFLUXDB_PORT: u16 = 8086; + +pub const INFLUXDB_ORG: &str = "iggy-test-org"; +pub const INFLUXDB_BUCKET: &str = "iggy-test-bucket"; +pub const INFLUXDB_TOKEN: &str = "iggy-test-secret-token"; +pub const INFLUXDB_USERNAME: &str = "iggy-admin"; +pub const INFLUXDB_PASSWORD: &str = "iggy-password"; + +/// Number of attempts to poll `/ping` before giving up. +pub const HEALTH_CHECK_ATTEMPTS: usize = 60; +/// Milliseconds between each `/ping` attempt. +pub const HEALTH_CHECK_INTERVAL_MS: u64 = 1_000; + +pub const DEFAULT_TEST_STREAM: &str = "test_stream"; +pub const DEFAULT_TEST_TOPIC: &str = "test_topic"; + +// ── env-var keys injected into the connectors runtime ──────────────────────── + +pub const ENV_SOURCE_URL: &str = "IGGY_CONNECTORS_SOURCE_INFLUXDB_PLUGIN_CONFIG_URL"; +pub const ENV_SOURCE_ORG: &str = "IGGY_CONNECTORS_SOURCE_INFLUXDB_PLUGIN_CONFIG_ORG"; +pub const ENV_SOURCE_TOKEN: &str = "IGGY_CONNECTORS_SOURCE_INFLUXDB_PLUGIN_CONFIG_TOKEN"; +pub const ENV_SOURCE_QUERY: &str = "IGGY_CONNECTORS_SOURCE_INFLUXDB_PLUGIN_CONFIG_QUERY"; +pub const ENV_SOURCE_POLL_INTERVAL: &str = + "IGGY_CONNECTORS_SOURCE_INFLUXDB_PLUGIN_CONFIG_POLL_INTERVAL"; +pub const ENV_SOURCE_BATCH_SIZE: &str = "IGGY_CONNECTORS_SOURCE_INFLUXDB_PLUGIN_CONFIG_BATCH_SIZE"; +pub const ENV_SOURCE_STREAMS_0_STREAM: &str = "IGGY_CONNECTORS_SOURCE_INFLUXDB_STREAMS_0_STREAM"; +pub const ENV_SOURCE_STREAMS_0_TOPIC: &str = "IGGY_CONNECTORS_SOURCE_INFLUXDB_STREAMS_0_TOPIC"; +pub const ENV_SOURCE_STREAMS_0_SCHEMA: &str = "IGGY_CONNECTORS_SOURCE_INFLUXDB_STREAMS_0_SCHEMA"; +pub const ENV_SOURCE_PATH: &str = "IGGY_CONNECTORS_SOURCE_INFLUXDB_PATH"; + +pub const ENV_SINK_URL: &str = "IGGY_CONNECTORS_SINK_INFLUXDB_PLUGIN_CONFIG_URL"; +pub const ENV_SINK_ORG: &str = "IGGY_CONNECTORS_SINK_INFLUXDB_PLUGIN_CONFIG_ORG"; +pub const ENV_SINK_TOKEN: &str = "IGGY_CONNECTORS_SINK_INFLUXDB_PLUGIN_CONFIG_TOKEN"; +pub const ENV_SINK_BUCKET: &str = "IGGY_CONNECTORS_SINK_INFLUXDB_PLUGIN_CONFIG_BUCKET"; +pub const ENV_SINK_STREAMS_0_STREAM: &str = "IGGY_CONNECTORS_SINK_INFLUXDB_STREAMS_0_STREAM"; +pub const ENV_SINK_STREAMS_0_TOPICS: &str = "IGGY_CONNECTORS_SINK_INFLUXDB_STREAMS_0_TOPICS"; +pub const ENV_SINK_STREAMS_0_SCHEMA: &str = "IGGY_CONNECTORS_SINK_INFLUXDB_STREAMS_0_SCHEMA"; +pub const ENV_SINK_STREAMS_0_CONSUMER_GROUP: &str = + "IGGY_CONNECTORS_SINK_INFLUXDB_STREAMS_0_CONSUMER_GROUP"; +pub const ENV_SINK_PATH: &str = "IGGY_CONNECTORS_SINK_INFLUXDB_PATH"; + +// ── Container ──────────────────────────────────────────────────────────────── + +pub struct InfluxDbContainer { + #[allow(dead_code)] + container: ContainerAsync<GenericImage>, + pub base_url: String, +} + +impl InfluxDbContainer { + pub async fn start() -> Result<Self, TestBinaryError> { + let container: ContainerAsync<GenericImage> = + GenericImage::new(INFLUXDB_IMAGE, INFLUXDB_TAG) + .with_exposed_port(INFLUXDB_PORT.tcp()) + // "Listening" appears in stdout before the HTTP API is ready on + // aarch64/Apple Silicon; we add a real /ping probe below. + .with_wait_for(WaitFor::message_on_stdout("Listening")) + .with_mapped_port(0, INFLUXDB_PORT.tcp()) + .with_env_var("DOCKER_INFLUXDB_INIT_MODE", "setup") + .with_env_var("DOCKER_INFLUXDB_INIT_USERNAME", INFLUXDB_USERNAME) + .with_env_var("DOCKER_INFLUXDB_INIT_PASSWORD", INFLUXDB_PASSWORD) + .with_env_var("DOCKER_INFLUXDB_INIT_ORG", INFLUXDB_ORG) + .with_env_var("DOCKER_INFLUXDB_INIT_BUCKET", INFLUXDB_BUCKET) + .with_env_var("DOCKER_INFLUXDB_INIT_ADMIN_TOKEN", INFLUXDB_TOKEN) + .start() + .await + .map_err(|e| TestBinaryError::FixtureSetup { + fixture_type: "InfluxDbContainer".to_string(), + message: format!("Failed to start container: {e}"), + })?; + + let mapped_port = container + .ports() + .await + .map_err(|e| TestBinaryError::FixtureSetup { + fixture_type: "InfluxDbContainer".to_string(), + message: format!("Failed to get ports: {e}"), + })? + .map_to_host_port_ipv4(INFLUXDB_PORT) + .ok_or_else(|| TestBinaryError::FixtureSetup { + fixture_type: "InfluxDbContainer".to_string(), + message: "No mapping for InfluxDB port".to_string(), + })?; + + let base_url = format!("http://localhost:{mapped_port}"); + info!("InfluxDB container available at {base_url}"); + + Ok(Self { + container, + base_url, + }) + } +} + +// ── HTTP client ─────────────────────────────────────────────────────────────── + +pub fn create_http_client() -> HttpClient { + let retry_policy = ExponentialBackoff::builder().build_with_max_retries(3); + let client = reqwest::Client::builder() + .timeout(std::time::Duration::from_secs(30)) + .build() + .expect("Failed to build HTTP client"); + reqwest_middleware::ClientBuilder::new(client) + .with(RetryTransientMiddleware::new_with_policy(retry_policy)) + .build() +} + +// ── Shared InfluxDB operations ──────────────────────────────────────────────── + +pub trait InfluxDbOps: Sync { + fn container(&self) -> &InfluxDbContainer; + fn http_client(&self) -> &HttpClient; + + /// Write line-protocol lines into the test bucket. + fn write_lines( + &self, + lines: &[&str], + ) -> impl std::future::Future<Output = Result<(), TestBinaryError>> + Send { + async move { + let url = format!( + "{}/api/v2/write?org={}&bucket={}&precision=ns", + self.container().base_url, + INFLUXDB_ORG, + INFLUXDB_BUCKET, + ); + let body = lines.join("\n"); + + let response = self + .http_client() + .post(&url) + .header("Authorization", format!("Token {INFLUXDB_TOKEN}")) + .header("Content-Type", "text/plain; charset=utf-8") + .body(body) + .send() + .await + .map_err(|e| TestBinaryError::FixtureSetup { + fixture_type: "InfluxDbOps".to_string(), + message: format!("Failed to write lines: {e}"), + })?; + + let status = response.status(); + if !status.is_success() && status.as_u16() != 204 { + let body = response.text().await.unwrap_or_default(); + return Err(TestBinaryError::FixtureSetup { + fixture_type: "InfluxDbOps".to_string(), + message: format!("Write error: status={status}, body={body}"), + }); + } + + Ok(()) + } + } + + fn query_count( + &self, + flux: &str, + ) -> impl std::future::Future<Output = Result<usize, TestBinaryError>> + Send { + async move { + let url = format!( + "{}/api/v2/query?org={}", + self.container().base_url, + INFLUXDB_ORG + ); + let body = serde_json::json!({ "query": flux, "type": "flux" }); + + let response = self + .http_client() + .post(&url) + .header("Authorization", format!("Token {INFLUXDB_TOKEN}")) + .header("Content-Type", "application/json") + .header("Accept", "application/csv") + .json(&body) + .send() + .await + .map_err(|e| TestBinaryError::InvalidState { + message: format!("Failed to query InfluxDB: {e}"), + })?; + + let text = response.text().await.unwrap_or_default(); + eprintln!( Review Comment: debug `eprintln!` left in test fixture code. should use `tracing::debug!` or be removed. ########## core/connectors/sources/influxdb_source/src/lib.rs: ########## @@ -0,0 +1,779 @@ +/* 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"; +// 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, + 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 + pub open_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, + _ => 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, + 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> { + let client = self.get_client()?; + let url = self.build_query_url()?; + let max_retries = self.get_max_retries(); + let token = self.config.token.clone(); + + // Cap for per-query backoff (reuse open_retry_max_delay config) + let max_delay = parse_duration( Review Comment: `run_query_with_retry` reuses `open_retry_max_delay` for per-query backoff. hubcio flagged this exact pattern for the sink and it was fixed with a separate `retry_max_delay` config field + `write_retry_max_delay` cached in `open()`. the source has the same issue unfixed - `InfluxDbSourceConfig` has no `retry_max_delay` field, so a user setting `open_retry_max_delay = "60s"` for patient startup also gets 60s caps on per-query retries. -- 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]
