From 63491d9770669b344802cc105263ca86ba520c61 Mon Sep 17 00:00:00 2001 From: paomian Date: Wed, 22 May 2024 17:27:19 +0800 Subject: [PATCH 01/51] chore: add log http ingester scaffold --- src/frontend/src/server.rs | 3 +++ src/servers/Cargo.toml | 1 + src/servers/src/http.rs | 18 ++++++++++++++++++ src/servers/src/http/handler.rs | 26 ++++++++++++++++++++++++++ 4 files changed, 48 insertions(+) diff --git a/src/frontend/src/server.rs b/src/frontend/src/server.rs index f5a0afb53016..063c6de015c8 100644 --- a/src/frontend/src/server.rs +++ b/src/frontend/src/server.rs @@ -90,6 +90,9 @@ where Some(self.instance.clone()), ); + builder = builder + .with_log_ingest_handler(ServerGrpcQueryHandlerAdapter::arc(self.instance.clone())); + if let Some(user_provider) = self.plugins.get::() { builder = builder.with_user_provider(user_provider); } diff --git a/src/servers/Cargo.toml b/src/servers/Cargo.toml index b355539ea713..371b25bae648 100644 --- a/src/servers/Cargo.toml +++ b/src/servers/Cargo.toml @@ -104,6 +104,7 @@ tower = { workspace = true, features = ["full"] } tower-http = { version = "0.4", features = ["full"] } urlencoding = "2.1" zstd.workspace = true +#pipeline = { git = "https://github.com/GreptimeTeam/pipeline.git", rev = "3cdbe92239303b3208ddd71a7ff7630730c91ad5" } [target.'cfg(not(windows))'.dependencies] tikv-jemalloc-ctl = { version = "0.5", features = ["use_std"] } diff --git a/src/servers/src/http.rs b/src/servers/src/http.rs index 5ef29b8b38c1..99bee15507a2 100644 --- a/src/servers/src/http.rs +++ b/src/servers/src/http.rs @@ -65,6 +65,7 @@ use crate::http::prometheus::{ use crate::metrics::http_metrics_layer; use crate::metrics_handler::MetricsHandler; use crate::prometheus_handler::PrometheusHandlerRef; +use crate::query_handler::grpc::ServerGrpcQueryHandlerRef; use crate::query_handler::sql::ServerSqlQueryHandlerRef; use crate::query_handler::{ InfluxdbLineProtocolHandlerRef, OpenTelemetryProtocolHandlerRef, OpentsdbProtocolHandlerRef, @@ -587,6 +588,17 @@ impl HttpServerBuilder { } } + // FIXME(qtang): This is a temporary solution to handle the log ingest metrics. + pub fn with_log_ingest_handler(self, handler: ServerGrpcQueryHandlerRef) -> Self { + Self { + router: self.router.nest( + &format!("/{HTTP_API_VERSION}/event"), + HttpServer::route_log(handler), + ), + ..self + } + } + pub fn with_plugins(self, plugins: Plugins) -> Self { Self { plugins, ..self } } @@ -699,6 +711,12 @@ impl HttpServer { .with_state(metrics_handler) } + fn route_log(grpc_handler: ServerGrpcQueryHandlerRef) -> Router { + Router::new() + .route("/logs", routing::get(handler::log_ingester)) + .with_state(grpc_handler) + } + fn route_sql(api_state: ApiState) -> ApiRouter { ApiRouter::new() .api_route( diff --git a/src/servers/src/http/handler.rs b/src/servers/src/http/handler.rs index fa8fe98e4cf1..858498955c5e 100644 --- a/src/servers/src/http/handler.rs +++ b/src/servers/src/http/handler.rs @@ -17,6 +17,7 @@ use std::env; use std::time::Instant; use aide::transform::TransformOperation; +use api::v1::RowInsertRequests; use axum::extract::{Json, Query, State}; use axum::response::{IntoResponse, Response}; use axum::{Extension, Form}; @@ -44,6 +45,7 @@ use crate::http::{ HttpResponse, ResponseFormat, }; use crate::metrics_handler::MetricsHandler; +use crate::query_handler::grpc::ServerGrpcQueryHandlerRef; use crate::query_handler::sql::ServerSqlQueryHandlerRef; #[derive(Debug, Default, Serialize, Deserialize, JsonSchema)] @@ -64,6 +66,30 @@ pub struct SqlQuery { pub epoch: Option, pub limit: Option, } +/// handler to log ingester +#[axum_macros::debug_handler] +pub async fn log_ingester( + State(_state): State, + Query(_query_params): Query, + Extension(_query_ctx): Extension, + Json(_payload): Json, +) -> String { + // TODO (paomian): implement log ingester + // transform payload to RowInsertRequest + // maybe we need a new trait for log ingester like `LogIngester` instead of `ServerGrpcQueryHandlerRef` + // let request = pileline::transform(payload); + // state.do_query(payload, query_ctx).await.to_string(); + let _processors = _payload["processors"].as_str().unwrap(); + let _data = _payload["data"].as_array().unwrap(); + + let insert_request = + api::v1::greptime_request::Request::RowInserts(RowInsertRequests::default()); + let insert_result = _state.do_query(insert_request, _query_ctx).await; + match insert_result { + Ok(_) => String::from("ok"), + Err(e) => e.to_string(), + } +} /// Handler to execute sql #[axum_macros::debug_handler] From 4d2ec3bb54a2d01d13d7549269b903ee1ad7f182 Mon Sep 17 00:00:00 2001 From: paomian Date: Wed, 22 May 2024 20:18:26 +0800 Subject: [PATCH 02/51] chore: add some example code --- src/servers/src/http.rs | 2 +- src/servers/src/http/handler.rs | 62 +++++++++++++++++++++++++++++---- 2 files changed, 56 insertions(+), 8 deletions(-) diff --git a/src/servers/src/http.rs b/src/servers/src/http.rs index 99bee15507a2..8354db475de6 100644 --- a/src/servers/src/http.rs +++ b/src/servers/src/http.rs @@ -713,7 +713,7 @@ impl HttpServer { fn route_log(grpc_handler: ServerGrpcQueryHandlerRef) -> Router { Router::new() - .route("/logs", routing::get(handler::log_ingester)) + .route("/logs", routing::post(handler::log_ingester)) .with_state(grpc_handler) } diff --git a/src/servers/src/http/handler.rs b/src/servers/src/http/handler.rs index 858498955c5e..d9fd28aaa3d2 100644 --- a/src/servers/src/http/handler.rs +++ b/src/servers/src/http/handler.rs @@ -14,10 +14,14 @@ use std::collections::HashMap; use std::env; -use std::time::Instant; +use std::time::{Instant, SystemTime, UNIX_EPOCH}; use aide::transform::TransformOperation; -use api::v1::RowInsertRequests; +use api::v1::value::ValueData; +use api::v1::{ + ColumnDataType, ColumnSchema, Row, RowInsertRequest, RowInsertRequests, Rows, SemanticType, + Value as ApiValue, +}; use axum::extract::{Json, Query, State}; use axum::response::{IntoResponse, Response}; use axum::{Extension, Form}; @@ -81,13 +85,57 @@ pub async fn log_ingester( // state.do_query(payload, query_ctx).await.to_string(); let _processors = _payload["processors"].as_str().unwrap(); let _data = _payload["data"].as_array().unwrap(); - - let insert_request = - api::v1::greptime_request::Request::RowInserts(RowInsertRequests::default()); - let insert_result = _state.do_query(insert_request, _query_ctx).await; + let mut rows = Rows::default(); + + // need a ColumnSchema for rows + rows.schema = vec![ + ColumnSchema { + column_name: "log".to_string(), + datatype: ColumnDataType::String as i32, + semantic_type: SemanticType::Tag as i32, + datatype_extension: None, + }, + ColumnSchema { + column_name: "ts".to_string(), + datatype: ColumnDataType::TimestampSecond as i32, + semantic_type: SemanticType::Timestamp as i32, + datatype_extension: None, + }, + ]; + + for row in _data { + let _row = row.as_str().unwrap(); + let mut value = ApiValue::default(); + value.value_data = Some(ValueData::StringValue(_row.to_string())); + let start = SystemTime::now(); + let since_the_epoch = start + .duration_since(UNIX_EPOCH) + .expect("Time went backwards"); + let now = + since_the_epoch.as_secs(); + rows.rows.push(Row { + values: vec![ + value, + ApiValue { + value_data: Some(ValueData::TimestampSecondValue(now as i64)), + }, + ], + }) + } + let mut insert_request = RowInsertRequest::default(); + insert_request.rows = Some(rows); + insert_request.table_name = "log".to_string(); + let insert_requests = RowInsertRequests { + inserts: vec![insert_request], + }; + let test_insert_request = api::v1::greptime_request::Request::RowInserts(insert_requests); + let insert_result = _state.do_query(test_insert_request, _query_ctx).await; match insert_result { Ok(_) => String::from("ok"), - Err(e) => e.to_string(), + Err(e) => { + tracing::error!(error = ?e, "error in log ingester"); + e.to_string() + } } } From 2e51c1667a3144e47dd51202461e094eca411b52 Mon Sep 17 00:00:00 2001 From: paomian Date: Mon, 27 May 2024 15:40:50 +0800 Subject: [PATCH 03/51] chore: add log inserter --- Cargo.lock | 72 ++++++++++++-- src/frontend/src/instance.rs | 4 +- src/frontend/src/server.rs | 3 +- src/operator/src/insert.rs | 152 ++++++++++++++++++++++------- src/servers/Cargo.toml | 3 +- src/servers/src/http.rs | 12 +-- src/servers/src/http/handler.rs | 158 +++++++++++++++++++------------ src/servers/src/query_handler.rs | 7 ++ 8 files changed, 296 insertions(+), 115 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 16bf0e911aad..1bfde2622540 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -222,7 +222,7 @@ dependencies = [ "common-macro", "common-time", "datatypes", - "greptime-proto", + "greptime-proto 0.1.0 (git+https://github.com/GreptimeTeam/greptime-proto.git?rev=a11db14b8502f55ca5348917fd18e6fcf140f55e)", "paste", "prost 0.12.4", "snafu 0.8.2", @@ -339,7 +339,7 @@ dependencies = [ "arrow-data", "arrow-schema", "chrono", - "chrono-tz", + "chrono-tz 0.8.6", "half 2.4.1", "hashbrown 0.14.5", "num", @@ -1387,7 +1387,18 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d59ae0466b83e838b81a54256c39d5d7c20b9d7daa10510a242d9b75abd5936e" dependencies = [ "chrono", - "chrono-tz-build", + "chrono-tz-build 0.2.1", + "phf", +] + +[[package]] +name = "chrono-tz" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "93698b29de5e97ad0ae26447b344c482a7284c737d9ddc5f9e52b74a336671bb" +dependencies = [ + "chrono", + "chrono-tz-build 0.3.0", "phf", ] @@ -1402,6 +1413,17 @@ dependencies = [ "phf_codegen", ] +[[package]] +name = "chrono-tz-build" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0c088aee841df9c3041febbb73934cfc39708749bf96dc827e3359cd39ef11b1" +dependencies = [ + "parse-zoneinfo", + "phf", + "phf_codegen", +] + [[package]] name = "chunked_transfer" version = "1.5.0" @@ -2091,7 +2113,7 @@ dependencies = [ "backtrace", "common-error", "console-subscriber", - "greptime-proto", + "greptime-proto 0.1.0 (git+https://github.com/GreptimeTeam/greptime-proto.git?rev=a11db14b8502f55ca5348917fd18e6fcf140f55e)", "lazy_static", "once_cell", "opentelemetry 0.21.0", @@ -2128,7 +2150,7 @@ version = "0.8.0" dependencies = [ "arrow", "chrono", - "chrono-tz", + "chrono-tz 0.8.6", "common-error", "common-macro", "once_cell", @@ -3824,7 +3846,7 @@ dependencies = [ "enum-as-inner", "enum_dispatch", "futures", - "greptime-proto", + "greptime-proto 0.1.0 (git+https://github.com/GreptimeTeam/greptime-proto.git?rev=a11db14b8502f55ca5348917fd18e6fcf140f55e)", "hydroflow", "itertools 0.10.5", "minstant", @@ -4206,6 +4228,20 @@ dependencies = [ "tonic-build 0.11.0", ] +[[package]] +name = "greptime-proto" +version = "0.1.0" +source = "git+https://github.com/GreptimeTeam/greptime-proto.git#31dd060fc3c84836d0b4997b11c22da122084dae" +dependencies = [ + "prost 0.12.4", + "serde", + "serde_json", + "strum 0.25.0", + "strum_macros 0.25.3", + "tonic 0.11.0", + "tonic-build 0.11.0", +] + [[package]] name = "h2" version = "0.3.26" @@ -4697,7 +4733,7 @@ dependencies = [ "common-telemetry", "fst", "futures", - "greptime-proto", + "greptime-proto 0.1.0 (git+https://github.com/GreptimeTeam/greptime-proto.git?rev=a11db14b8502f55ca5348917fd18e6fcf140f55e)", "mockall", "pin-project", "prost 0.12.4", @@ -6669,7 +6705,7 @@ dependencies = [ "async-trait", "bytes", "chrono", - "chrono-tz", + "chrono-tz 0.8.6", "datafusion 37.1.0", "datafusion-expr 37.1.0", "datafusion-physical-expr 37.1.0", @@ -7142,6 +7178,21 @@ version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" +[[package]] +name = "pipeline" +version = "0.1.0" +source = "git+ssh://git@github.com/GreptimeTeam/pipeline.git?rev=d3b700c226abc10b5030b38f2a20cc30c1983970#d3b700c226abc10b5030b38f2a20cc30c1983970" +dependencies = [ + "chrono", + "chrono-tz 0.9.0", + "csv", + "greptime-proto 0.1.0 (git+https://github.com/GreptimeTeam/greptime-proto.git)", + "itertools 0.12.1", + "serde_json", + "urlencoding", + "yaml-rust", +] + [[package]] name = "pkcs1" version = "0.3.3" @@ -7534,7 +7585,7 @@ dependencies = [ "datafusion-functions 37.0.0", "datatypes", "futures", - "greptime-proto", + "greptime-proto 0.1.0 (git+https://github.com/GreptimeTeam/greptime-proto.git?rev=a11db14b8502f55ca5348917fd18e6fcf140f55e)", "itertools 0.10.5", "lazy_static", "prometheus", @@ -7875,7 +7926,7 @@ dependencies = [ "format_num", "futures", "futures-util", - "greptime-proto", + "greptime-proto 0.1.0 (git+https://github.com/GreptimeTeam/greptime-proto.git?rev=a11db14b8502f55ca5348917fd18e6fcf140f55e)", "humantime", "lazy_static", "meter-core", @@ -9536,6 +9587,7 @@ dependencies = [ "permutation", "pgwire", "pin-project", + "pipeline", "postgres-types", "pprof", "prometheus", diff --git a/src/frontend/src/instance.rs b/src/frontend/src/instance.rs index c0ae6b64c2f8..860ed700bb6c 100644 --- a/src/frontend/src/instance.rs +++ b/src/frontend/src/instance.rs @@ -15,6 +15,7 @@ pub mod builder; mod grpc; mod influxdb; +mod log_handler; mod opentsdb; mod otlp; mod prom_store; @@ -66,7 +67,7 @@ use servers::prometheus_handler::PrometheusHandler; use servers::query_handler::grpc::GrpcQueryHandler; use servers::query_handler::sql::SqlQueryHandler; use servers::query_handler::{ - InfluxdbLineProtocolHandler, OpenTelemetryProtocolHandler, OpentsdbProtocolHandler, + InfluxdbLineProtocolHandler, LogHandler, OpenTelemetryProtocolHandler, OpentsdbProtocolHandler, PromStoreProtocolHandler, ScriptHandler, }; use servers::server::ServerHandlers; @@ -100,6 +101,7 @@ pub trait FrontendInstance: + OpenTelemetryProtocolHandler + ScriptHandler + PrometheusHandler + + LogHandler + Send + Sync + 'static diff --git a/src/frontend/src/server.rs b/src/frontend/src/server.rs index 063c6de015c8..e2f0fdb56254 100644 --- a/src/frontend/src/server.rs +++ b/src/frontend/src/server.rs @@ -90,8 +90,7 @@ where Some(self.instance.clone()), ); - builder = builder - .with_log_ingest_handler(ServerGrpcQueryHandlerAdapter::arc(self.instance.clone())); + builder = builder.with_log_ingest_handler(self.instance.clone()); if let Some(user_provider) = self.plugins.get::() { builder = builder.with_user_provider(user_provider); diff --git a/src/operator/src/insert.rs b/src/operator/src/insert.rs index a68ed9b6be18..f4ec1f0996ab 100644 --- a/src/operator/src/insert.rs +++ b/src/operator/src/insert.rs @@ -67,6 +67,12 @@ pub struct Inserter { pub type InserterRef = Arc; +enum TableType { + Logical(String), + Physical, + Log, +} + impl Inserter { pub fn new( catalog_manager: CatalogManagerRef, @@ -109,7 +115,37 @@ impl Inserter { validate_column_count_match(&requests)?; let table_name_to_ids = self - .create_or_alter_tables_on_demand(&requests, &ctx, None, statement_executor) + .create_or_alter_tables_on_demand( + &requests, + &ctx, + TableType::Physical, + statement_executor, + ) + .await?; + let inserts = RowToRegion::new(table_name_to_ids, self.partition_manager.as_ref()) + .convert(requests) + .await?; + + self.do_request(inserts, &ctx).await + } + + pub async fn handle_log_inserts( + &self, + mut requests: RowInsertRequests, + ctx: QueryContextRef, + statement_executor: &StatementExecutor, + ) -> Result { + // remove empty requests + requests.inserts.retain(|req| { + req.rows + .as_ref() + .map(|r| !r.rows.is_empty()) + .unwrap_or_default() + }); + validate_column_count_match(&requests)?; + + let table_name_to_ids = self + .create_or_alter_tables_on_demand(&requests, &ctx, TableType::Log, statement_executor) .await?; let inserts = RowToRegion::new(table_name_to_ids, self.partition_manager.as_ref()) .convert(requests) @@ -144,7 +180,7 @@ impl Inserter { .create_or_alter_tables_on_demand( &requests, &ctx, - Some(physical_table.to_string()), + TableType::Logical(physical_table.to_string()), statement_executor, ) .await?; @@ -366,7 +402,7 @@ impl Inserter { &self, requests: &RowInsertRequests, ctx: &QueryContextRef, - on_physical_table: Option, + table_type: TableType, statement_executor: &StatementExecutor, ) -> Result> { let mut table_name_to_ids = HashMap::with_capacity(requests.inserts.len()); @@ -394,42 +430,56 @@ impl Inserter { } } - if let Some(on_physical_table) = on_physical_table { - if !create_tables.is_empty() { - // Creates logical tables in batch. - let tables = self - .create_logical_tables( - create_tables, - ctx, - &on_physical_table, - statement_executor, - ) - .await?; + match table_type { + TableType::Logical(on_physical_table) => { + if !create_tables.is_empty() { + // Creates logical tables in batch. + let tables = self + .create_logical_tables( + create_tables, + ctx, + &on_physical_table, + statement_executor, + ) + .await?; - for table in tables { + for table in tables { + let table_info = table.table_info(); + table_name_to_ids.insert(table_info.name.clone(), table_info.table_id()); + } + } + if !alter_tables.is_empty() { + // Alter logical tables in batch. + statement_executor + .alter_logical_tables(alter_tables, ctx.clone()) + .await?; + } + } + TableType::Physical => { + for req in create_tables { + let table = self.create_table(req, ctx, statement_executor).await?; let table_info = table.table_info(); table_name_to_ids.insert(table_info.name.clone(), table_info.table_id()); } + for alter_expr in alter_tables.into_iter() { + statement_executor + .alter_table_inner(alter_expr, ctx.clone()) + .await?; + } } - if !alter_tables.is_empty() { - // Alter logical tables in batch. - statement_executor - .alter_logical_tables(alter_tables, ctx.clone()) - .await?; - } - } else { - for req in create_tables { - let table = self.create_table(req, ctx, statement_executor).await?; - let table_info = table.table_info(); - table_name_to_ids.insert(table_info.name.clone(), table_info.table_id()); - } - for alter_expr in alter_tables.into_iter() { - statement_executor - .alter_table_inner(alter_expr, ctx.clone()) - .await?; + TableType::Log => { + for req in create_tables { + let table = self.create_log_table(req, ctx, statement_executor).await?; + let table_info = table.table_info(); + table_name_to_ids.insert(table_info.name.clone(), table_info.table_id()); + } + for alter_expr in alter_tables.into_iter() { + statement_executor + .alter_table_inner(alter_expr, ctx.clone()) + .await?; + } } } - Ok(table_name_to_ids) } @@ -571,6 +621,44 @@ impl Inserter { } } + async fn create_log_table( + &self, + req: &RowInsertRequest, + ctx: &QueryContextRef, + statement_executor: &StatementExecutor, + ) -> Result { + let table_ref = + TableReference::full(ctx.current_catalog(), ctx.current_schema(), &req.table_name); + + let request_schema = req.rows.as_ref().unwrap().schema.as_slice(); + let create_table_expr = &mut build_create_table_expr(&table_ref, request_schema)?; + + info!("Table `{table_ref}` does not exist, try creating table"); + create_table_expr + .table_options + .insert("append_mode".to_string(), "true".to_string()); + let res = statement_executor + .create_table_inner(create_table_expr, None, ctx.clone()) + .await; + + match res { + Ok(table) => { + info!( + "Successfully created table {}.{}.{}", + table_ref.catalog, table_ref.schema, table_ref.table, + ); + Ok(table) + } + Err(err) => { + error!( + "Failed to create table {}.{}.{}: {}", + table_ref.catalog, table_ref.schema, table_ref.table, err + ); + Err(err) + } + } + } + async fn create_logical_tables( &self, create_tables: Vec<&RowInsertRequest>, diff --git a/src/servers/Cargo.toml b/src/servers/Cargo.toml index 371b25bae648..7421aa01f630 100644 --- a/src/servers/Cargo.toml +++ b/src/servers/Cargo.toml @@ -104,8 +104,7 @@ tower = { workspace = true, features = ["full"] } tower-http = { version = "0.4", features = ["full"] } urlencoding = "2.1" zstd.workspace = true -#pipeline = { git = "https://github.com/GreptimeTeam/pipeline.git", rev = "3cdbe92239303b3208ddd71a7ff7630730c91ad5" } - +pipeline = { git = "ssh://git@github.com/GreptimeTeam/pipeline.git", rev = "d3b700c226abc10b5030b38f2a20cc30c1983970" } [target.'cfg(not(windows))'.dependencies] tikv-jemalloc-ctl = { version = "0.5", features = ["use_std"] } diff --git a/src/servers/src/http.rs b/src/servers/src/http.rs index 8354db475de6..d112b63e4ec6 100644 --- a/src/servers/src/http.rs +++ b/src/servers/src/http.rs @@ -65,11 +65,10 @@ use crate::http::prometheus::{ use crate::metrics::http_metrics_layer; use crate::metrics_handler::MetricsHandler; use crate::prometheus_handler::PrometheusHandlerRef; -use crate::query_handler::grpc::ServerGrpcQueryHandlerRef; use crate::query_handler::sql::ServerSqlQueryHandlerRef; use crate::query_handler::{ - InfluxdbLineProtocolHandlerRef, OpenTelemetryProtocolHandlerRef, OpentsdbProtocolHandlerRef, - PromStoreProtocolHandlerRef, ScriptHandlerRef, + InfluxdbLineProtocolHandlerRef, LogHandlerRef, OpenTelemetryProtocolHandlerRef, + OpentsdbProtocolHandlerRef, PromStoreProtocolHandlerRef, ScriptHandlerRef, }; use crate::server::Server; @@ -588,8 +587,7 @@ impl HttpServerBuilder { } } - // FIXME(qtang): This is a temporary solution to handle the log ingest metrics. - pub fn with_log_ingest_handler(self, handler: ServerGrpcQueryHandlerRef) -> Self { + pub fn with_log_ingest_handler(self, handler: LogHandlerRef) -> Self { Self { router: self.router.nest( &format!("/{HTTP_API_VERSION}/event"), @@ -711,10 +709,10 @@ impl HttpServer { .with_state(metrics_handler) } - fn route_log(grpc_handler: ServerGrpcQueryHandlerRef) -> Router { + fn route_log(log_handler: LogHandlerRef) -> Router { Router::new() .route("/logs", routing::post(handler::log_ingester)) - .with_state(grpc_handler) + .with_state(log_handler) } fn route_sql(api_state: ApiState) -> ApiRouter { diff --git a/src/servers/src/http/handler.rs b/src/servers/src/http/handler.rs index d9fd28aaa3d2..265e315fbc39 100644 --- a/src/servers/src/http/handler.rs +++ b/src/servers/src/http/handler.rs @@ -14,14 +14,10 @@ use std::collections::HashMap; use std::env; -use std::time::{Instant, SystemTime, UNIX_EPOCH}; +use std::time::Instant; use aide::transform::TransformOperation; -use api::v1::value::ValueData; -use api::v1::{ - ColumnDataType, ColumnSchema, Row, RowInsertRequest, RowInsertRequests, Rows, SemanticType, - Value as ApiValue, -}; +use api::v1::{RowInsertRequest, RowInsertRequests}; use axum::extract::{Json, Query, State}; use axum::response::{IntoResponse, Response}; use axum::{Extension, Form}; @@ -31,6 +27,9 @@ use common_plugins::GREPTIME_EXEC_WRITE_COST; use common_query::{Output, OutputData}; use common_recordbatch::util; use common_telemetry::tracing; +use pipeline::transform::GreptimeTransformer; +use pipeline::value::Value as PipelineValue; +use pipeline::{parse, Content, Pipeline}; use query::parser::{PromQuery, DEFAULT_LOOKBACK_STRING}; use schemars::JsonSchema; use serde::{Deserialize, Serialize}; @@ -49,8 +48,8 @@ use crate::http::{ HttpResponse, ResponseFormat, }; use crate::metrics_handler::MetricsHandler; -use crate::query_handler::grpc::ServerGrpcQueryHandlerRef; use crate::query_handler::sql::ServerSqlQueryHandlerRef; +use crate::query_handler::LogHandlerRef; #[derive(Debug, Default, Serialize, Deserialize, JsonSchema)] pub struct SqlQuery { @@ -70,71 +69,108 @@ pub struct SqlQuery { pub epoch: Option, pub limit: Option, } + +#[derive(Debug, Default, Serialize, Deserialize, JsonSchema)] +pub struct LogIngesterQueryParams { + pub table_name: String, +} + +fn validate_log_ingester_payload(payload: &Value) -> Result<(), String> { + if !payload.is_object() { + return Err("payload must be an object".to_string()); + } + + if payload["processors"].as_str().is_none() { + return Err("processors field is required".to_string()); + } + + if payload["data"].as_array().is_none() { + return Err("data field is required".to_string()); + } + + Ok(()) +} + /// handler to log ingester #[axum_macros::debug_handler] pub async fn log_ingester( - State(_state): State, - Query(_query_params): Query, + State(_state): State, + Query(_query_params): Query, Extension(_query_ctx): Extension, Json(_payload): Json, -) -> String { - // TODO (paomian): implement log ingester +) -> HttpResponse { + // TODO (qtang): implement log ingester // transform payload to RowInsertRequest // maybe we need a new trait for log ingester like `LogIngester` instead of `ServerGrpcQueryHandlerRef` // let request = pileline::transform(payload); // state.do_query(payload, query_ctx).await.to_string(); - let _processors = _payload["processors"].as_str().unwrap(); - let _data = _payload["data"].as_array().unwrap(); - let mut rows = Rows::default(); - - // need a ColumnSchema for rows - rows.schema = vec![ - ColumnSchema { - column_name: "log".to_string(), - datatype: ColumnDataType::String as i32, - semantic_type: SemanticType::Tag as i32, - datatype_extension: None, - }, - ColumnSchema { - column_name: "ts".to_string(), - datatype: ColumnDataType::TimestampSecond as i32, - semantic_type: SemanticType::Timestamp as i32, - datatype_extension: None, - }, - ]; - - for row in _data { - let _row = row.as_str().unwrap(); - let mut value = ApiValue::default(); - value.value_data = Some(ValueData::StringValue(_row.to_string())); - let start = SystemTime::now(); - let since_the_epoch = start - .duration_since(UNIX_EPOCH) - .expect("Time went backwards"); - let now = - since_the_epoch.as_secs(); - rows.rows.push(Row { - values: vec![ - value, - ApiValue { - value_data: Some(ValueData::TimestampSecondValue(now as i64)), - }, - ], - }) - } - let mut insert_request = RowInsertRequest::default(); - insert_request.rows = Some(rows); - insert_request.table_name = "log".to_string(); - let insert_requests = RowInsertRequests { - inserts: vec![insert_request], + match validate_log_ingester_payload(&_payload) { + Ok(_) => (), + Err(e) => { + return HttpResponse::Error(ErrorResponse::from_error_message( + StatusCode::InvalidArguments, + e, + )) + } }; - let test_insert_request = api::v1::greptime_request::Request::RowInserts(insert_requests); - let insert_result = _state.do_query(test_insert_request, _query_ctx).await; - match insert_result { - Ok(_) => String::from("ok"), + let processors_ = _payload["processors"].as_str().unwrap(); + let data = _payload["data"]; + let yaml_content = Content::Yaml(processors_.into()); + let pipeline_: Result, String> = parse(&yaml_content); + match pipeline_ { + Ok(pipeline) => { + let pipeline_data = PipelineValue::try_from(data); + match pipeline_data { + Ok(pipeline_data) => { + let transformed_data = pipeline.exec(pipeline_data); + match transformed_data { + Ok(rows) => { + let insert_request = RowInsertRequest { + rows: Some(rows), + table_name: _query_params.table_name.clone(), + }; + let insert_requests = RowInsertRequests { + inserts: vec![insert_request], + }; + let insert_result = + _state.insert_log(insert_requests, _query_ctx).await; + match insert_result { + Ok(_) => { + return HttpResponse::GreptimedbV1(GreptimedbV1Response { + output: vec![], + execution_time_ms: 0, + resp_metrics: HashMap::new(), + }) + } + Err(e) => { + return HttpResponse::Error(ErrorResponse::from_error_message( + StatusCode::InvalidArguments, + e.to_string(), + )); + } + } + } + Err(e) => { + return HttpResponse::Error(ErrorResponse::from_error_message( + StatusCode::InvalidArguments, + e, + )) + } + } + } + Err(e) => { + return HttpResponse::Error(ErrorResponse::from_error_message( + StatusCode::InvalidArguments, + e, + )) + } + } + } Err(e) => { - tracing::error!(error = ?e, "error in log ingester"); - e.to_string() + return HttpResponse::Error(ErrorResponse::from_error_message( + StatusCode::InvalidArguments, + e, + )) } } } diff --git a/src/servers/src/query_handler.rs b/src/servers/src/query_handler.rs index 0430005aed7d..f15f945df865 100644 --- a/src/servers/src/query_handler.rs +++ b/src/servers/src/query_handler.rs @@ -48,6 +48,7 @@ pub type InfluxdbLineProtocolHandlerRef = Arc; pub type OpenTelemetryProtocolHandlerRef = Arc; pub type ScriptHandlerRef = Arc; +pub type LogHandlerRef = Arc; #[async_trait] pub trait ScriptHandler { @@ -118,3 +119,9 @@ pub trait OpenTelemetryProtocolHandler { ctx: QueryContextRef, ) -> Result; } + +#[async_trait] + +pub trait LogHandler { + async fn insert_log(&self, log: RowInsertRequests, ctx: QueryContextRef) -> Result; +} From fbc66ec51424ea07931b47c143e538467871b9af Mon Sep 17 00:00:00 2001 From: paomian Date: Mon, 27 May 2024 16:53:26 +0800 Subject: [PATCH 04/51] chore: add log handler file --- Cargo.lock | 14 +++--- src/frontend/src/instance/log_handler.rs | 57 ++++++++++++++++++++++++ 2 files changed, 64 insertions(+), 7 deletions(-) create mode 100644 src/frontend/src/instance/log_handler.rs diff --git a/Cargo.lock b/Cargo.lock index 1bfde2622540..7db551ab64d8 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -222,7 +222,7 @@ dependencies = [ "common-macro", "common-time", "datatypes", - "greptime-proto 0.1.0 (git+https://github.com/GreptimeTeam/greptime-proto.git?rev=a11db14b8502f55ca5348917fd18e6fcf140f55e)", + "greptime-proto 0.1.0 (git+https://github.com/GreptimeTeam/greptime-proto.git?rev=902f75fdd170c572e90b1f640161d90995f20218)", "paste", "prost 0.12.4", "snafu 0.8.2", @@ -2113,7 +2113,7 @@ dependencies = [ "backtrace", "common-error", "console-subscriber", - "greptime-proto 0.1.0 (git+https://github.com/GreptimeTeam/greptime-proto.git?rev=a11db14b8502f55ca5348917fd18e6fcf140f55e)", + "greptime-proto 0.1.0 (git+https://github.com/GreptimeTeam/greptime-proto.git?rev=902f75fdd170c572e90b1f640161d90995f20218)", "lazy_static", "once_cell", "opentelemetry 0.21.0", @@ -3846,7 +3846,7 @@ dependencies = [ "enum-as-inner", "enum_dispatch", "futures", - "greptime-proto 0.1.0 (git+https://github.com/GreptimeTeam/greptime-proto.git?rev=a11db14b8502f55ca5348917fd18e6fcf140f55e)", + "greptime-proto 0.1.0 (git+https://github.com/GreptimeTeam/greptime-proto.git?rev=902f75fdd170c572e90b1f640161d90995f20218)", "hydroflow", "itertools 0.10.5", "minstant", @@ -4733,7 +4733,7 @@ dependencies = [ "common-telemetry", "fst", "futures", - "greptime-proto 0.1.0 (git+https://github.com/GreptimeTeam/greptime-proto.git?rev=a11db14b8502f55ca5348917fd18e6fcf140f55e)", + "greptime-proto 0.1.0 (git+https://github.com/GreptimeTeam/greptime-proto.git?rev=902f75fdd170c572e90b1f640161d90995f20218)", "mockall", "pin-project", "prost 0.12.4", @@ -7585,7 +7585,7 @@ dependencies = [ "datafusion-functions 37.0.0", "datatypes", "futures", - "greptime-proto 0.1.0 (git+https://github.com/GreptimeTeam/greptime-proto.git?rev=a11db14b8502f55ca5348917fd18e6fcf140f55e)", + "greptime-proto 0.1.0 (git+https://github.com/GreptimeTeam/greptime-proto.git?rev=902f75fdd170c572e90b1f640161d90995f20218)", "itertools 0.10.5", "lazy_static", "prometheus", @@ -7926,7 +7926,7 @@ dependencies = [ "format_num", "futures", "futures-util", - "greptime-proto 0.1.0 (git+https://github.com/GreptimeTeam/greptime-proto.git?rev=a11db14b8502f55ca5348917fd18e6fcf140f55e)", + "greptime-proto 0.1.0 (git+https://github.com/GreptimeTeam/greptime-proto.git?rev=902f75fdd170c572e90b1f640161d90995f20218)", "humantime", "lazy_static", "meter-core", @@ -10562,7 +10562,7 @@ dependencies = [ "datatypes", "derive_builder 0.12.0", "futures", - "greptime-proto", + "greptime-proto 0.1.0 (git+https://github.com/GreptimeTeam/greptime-proto.git?rev=902f75fdd170c572e90b1f640161d90995f20218)", "humantime", "humantime-serde", "parquet", diff --git a/src/frontend/src/instance/log_handler.rs b/src/frontend/src/instance/log_handler.rs new file mode 100644 index 000000000000..ffbf94f5c54e --- /dev/null +++ b/src/frontend/src/instance/log_handler.rs @@ -0,0 +1,57 @@ +// Copyright 2023 Greptime Team +// +// Licensed 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 api::v1::RowInsertRequests; +use async_trait::async_trait; +use auth::{PermissionChecker, PermissionCheckerRef, PermissionReq}; +use client::Output; +use common_error::ext::BoxedError; +use servers::error::{AuthSnafu, ExecuteGrpcRequestSnafu}; +use servers::query_handler::LogHandler; +use session::context::QueryContextRef; +use snafu::ResultExt; + +use super::Instance; + +#[async_trait] +impl LogHandler for Instance { + async fn insert_log( + &self, + log: RowInsertRequests, + ctx: QueryContextRef, + ) -> servers::error::Result { + self.plugins + .get::() + .as_ref() + // This is a bug, it should be PermissionReq::LogWrite + .check_permission(ctx.current_user(), PermissionReq::PromStoreWrite) + .context(AuthSnafu)?; + + self.handle_log_inserts(log, ctx).await + } +} + +impl Instance { + pub async fn handle_log_inserts( + &self, + log: RowInsertRequests, + ctx: QueryContextRef, + ) -> servers::error::Result { + self.inserter + .handle_log_inserts(log, ctx, self.statement_executor.as_ref()) + .await + .map_err(BoxedError::new) + .context(ExecuteGrpcRequestSnafu) + } +} From cd4d83d7cb79ab4123780f6b4d428a386846052e Mon Sep 17 00:00:00 2001 From: paomian Date: Mon, 27 May 2024 17:34:31 +0800 Subject: [PATCH 05/51] chore: add pipeline lib --- Cargo.lock | 31 ++++----------- src/servers/Cargo.toml | 3 +- src/servers/src/http/handler.rs | 70 +++++++++++++++------------------ 3 files changed, 41 insertions(+), 63 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 7db551ab64d8..3b0ee56188dd 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -222,7 +222,7 @@ dependencies = [ "common-macro", "common-time", "datatypes", - "greptime-proto 0.1.0 (git+https://github.com/GreptimeTeam/greptime-proto.git?rev=902f75fdd170c572e90b1f640161d90995f20218)", + "greptime-proto", "paste", "prost 0.12.4", "snafu 0.8.2", @@ -2113,7 +2113,7 @@ dependencies = [ "backtrace", "common-error", "console-subscriber", - "greptime-proto 0.1.0 (git+https://github.com/GreptimeTeam/greptime-proto.git?rev=902f75fdd170c572e90b1f640161d90995f20218)", + "greptime-proto", "lazy_static", "once_cell", "opentelemetry 0.21.0", @@ -3846,7 +3846,7 @@ dependencies = [ "enum-as-inner", "enum_dispatch", "futures", - "greptime-proto 0.1.0 (git+https://github.com/GreptimeTeam/greptime-proto.git?rev=902f75fdd170c572e90b1f640161d90995f20218)", + "greptime-proto", "hydroflow", "itertools 0.10.5", "minstant", @@ -4228,20 +4228,6 @@ dependencies = [ "tonic-build 0.11.0", ] -[[package]] -name = "greptime-proto" -version = "0.1.0" -source = "git+https://github.com/GreptimeTeam/greptime-proto.git#31dd060fc3c84836d0b4997b11c22da122084dae" -dependencies = [ - "prost 0.12.4", - "serde", - "serde_json", - "strum 0.25.0", - "strum_macros 0.25.3", - "tonic 0.11.0", - "tonic-build 0.11.0", -] - [[package]] name = "h2" version = "0.3.26" @@ -4733,7 +4719,7 @@ dependencies = [ "common-telemetry", "fst", "futures", - "greptime-proto 0.1.0 (git+https://github.com/GreptimeTeam/greptime-proto.git?rev=902f75fdd170c572e90b1f640161d90995f20218)", + "greptime-proto", "mockall", "pin-project", "prost 0.12.4", @@ -7181,12 +7167,11 @@ checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" [[package]] name = "pipeline" version = "0.1.0" -source = "git+ssh://git@github.com/GreptimeTeam/pipeline.git?rev=d3b700c226abc10b5030b38f2a20cc30c1983970#d3b700c226abc10b5030b38f2a20cc30c1983970" dependencies = [ "chrono", "chrono-tz 0.9.0", "csv", - "greptime-proto 0.1.0 (git+https://github.com/GreptimeTeam/greptime-proto.git)", + "greptime-proto", "itertools 0.12.1", "serde_json", "urlencoding", @@ -7585,7 +7570,7 @@ dependencies = [ "datafusion-functions 37.0.0", "datatypes", "futures", - "greptime-proto 0.1.0 (git+https://github.com/GreptimeTeam/greptime-proto.git?rev=902f75fdd170c572e90b1f640161d90995f20218)", + "greptime-proto", "itertools 0.10.5", "lazy_static", "prometheus", @@ -7926,7 +7911,7 @@ dependencies = [ "format_num", "futures", "futures-util", - "greptime-proto 0.1.0 (git+https://github.com/GreptimeTeam/greptime-proto.git?rev=902f75fdd170c572e90b1f640161d90995f20218)", + "greptime-proto", "humantime", "lazy_static", "meter-core", @@ -10562,7 +10547,7 @@ dependencies = [ "datatypes", "derive_builder 0.12.0", "futures", - "greptime-proto 0.1.0 (git+https://github.com/GreptimeTeam/greptime-proto.git?rev=902f75fdd170c572e90b1f640161d90995f20218)", + "greptime-proto", "humantime", "humantime-serde", "parquet", diff --git a/src/servers/Cargo.toml b/src/servers/Cargo.toml index 7421aa01f630..791eac7a1825 100644 --- a/src/servers/Cargo.toml +++ b/src/servers/Cargo.toml @@ -104,7 +104,8 @@ tower = { workspace = true, features = ["full"] } tower-http = { version = "0.4", features = ["full"] } urlencoding = "2.1" zstd.workspace = true -pipeline = { git = "ssh://git@github.com/GreptimeTeam/pipeline.git", rev = "d3b700c226abc10b5030b38f2a20cc30c1983970" } +#pipeline = { git = "ssh://git@github.com/GreptimeTeam/pipeline.git", rev = "6b88c3c627da9e20f8fd160071e9c69b3ebd4e6a" } +pipeline = { path = "../../../pipeline" } [target.'cfg(not(windows))'.dependencies] tikv-jemalloc-ctl = { version = "0.5", features = ["use_std"] } diff --git a/src/servers/src/http/handler.rs b/src/servers/src/http/handler.rs index 265e315fbc39..8849ae42372c 100644 --- a/src/servers/src/http/handler.rs +++ b/src/servers/src/http/handler.rs @@ -17,7 +17,7 @@ use std::env; use std::time::Instant; use aide::transform::TransformOperation; -use api::v1::{RowInsertRequest, RowInsertRequests}; +use api::v1::{RowInsertRequest, RowInsertRequests, Rows}; use axum::extract::{Json, Query, State}; use axum::response::{IntoResponse, Response}; use axum::{Extension, Form}; @@ -73,6 +73,7 @@ pub struct SqlQuery { #[derive(Debug, Default, Serialize, Deserialize, JsonSchema)] pub struct LogIngesterQueryParams { pub table_name: String, + pub db: Option, } fn validate_log_ingester_payload(payload: &Value) -> Result<(), String> { @@ -80,11 +81,11 @@ fn validate_log_ingester_payload(payload: &Value) -> Result<(), String> { return Err("payload must be an object".to_string()); } - if payload["processors"].as_str().is_none() { + if payload["pipeline_model"].as_str().is_none() { return Err("processors field is required".to_string()); } - if payload["data"].as_array().is_none() { + if payload["data"].as_array().is_none() && payload["data"].as_object().is_none() { return Err("data field is required".to_string()); } @@ -97,7 +98,7 @@ pub async fn log_ingester( State(_state): State, Query(_query_params): Query, Extension(_query_ctx): Extension, - Json(_payload): Json, + Json(mut _payload): Json, ) -> HttpResponse { // TODO (qtang): implement log ingester // transform payload to RowInsertRequest @@ -113,16 +114,17 @@ pub async fn log_ingester( )) } }; - let processors_ = _payload["processors"].as_str().unwrap(); - let data = _payload["data"]; - let yaml_content = Content::Yaml(processors_.into()); + let processors_ = _payload["pipeline_model"].take(); + let processors = processors_.as_str().unwrap(); + let data = _payload["data"].take(); + let yaml_content = Content::Yaml(processors.into()); let pipeline_: Result, String> = parse(&yaml_content); match pipeline_ { Ok(pipeline) => { let pipeline_data = PipelineValue::try_from(data); match pipeline_data { Ok(pipeline_data) => { - let transformed_data = pipeline.exec(pipeline_data); + let transformed_data: Result = pipeline.exec(pipeline_data); match transformed_data { Ok(rows) => { let insert_request = RowInsertRequest { @@ -135,43 +137,33 @@ pub async fn log_ingester( let insert_result = _state.insert_log(insert_requests, _query_ctx).await; match insert_result { - Ok(_) => { - return HttpResponse::GreptimedbV1(GreptimedbV1Response { - output: vec![], - execution_time_ms: 0, - resp_metrics: HashMap::new(), - }) - } - Err(e) => { - return HttpResponse::Error(ErrorResponse::from_error_message( - StatusCode::InvalidArguments, - e.to_string(), - )); - } + Ok(_) => HttpResponse::GreptimedbV1(GreptimedbV1Response { + output: vec![], + execution_time_ms: 0, + resp_metrics: HashMap::new(), + }), + Err(e) => HttpResponse::Error(ErrorResponse::from_error_message( + StatusCode::InvalidArguments, + e.to_string(), + )), } } - Err(e) => { - return HttpResponse::Error(ErrorResponse::from_error_message( - StatusCode::InvalidArguments, - e, - )) - } + Err(e) => HttpResponse::Error(ErrorResponse::from_error_message( + StatusCode::InvalidArguments, + e, + )), } } - Err(e) => { - return HttpResponse::Error(ErrorResponse::from_error_message( - StatusCode::InvalidArguments, - e, - )) - } + Err(e) => HttpResponse::Error(ErrorResponse::from_error_message( + StatusCode::InvalidArguments, + e, + )), } } - Err(e) => { - return HttpResponse::Error(ErrorResponse::from_error_message( - StatusCode::InvalidArguments, - e, - )) - } + Err(e) => HttpResponse::Error(ErrorResponse::from_error_message( + StatusCode::InvalidArguments, + e, + )), } } From 2bc19379fca707e72929939e754c189751577246 Mon Sep 17 00:00:00 2001 From: paomian Date: Wed, 29 May 2024 15:38:58 +0800 Subject: [PATCH 06/51] chore: import log handler --- Cargo.lock | 3 ++ src/servers/src/http.rs | 5 ++ src/servers/src/http/handler.rs | 89 ++++++++++++++------------------ src/servers/src/query_handler.rs | 1 - 4 files changed, 46 insertions(+), 52 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 3b0ee56188dd..a1197b15e82b 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7173,6 +7173,9 @@ dependencies = [ "csv", "greptime-proto", "itertools 0.12.1", + "lazy_static", + "log", + "regex", "serde_json", "urlencoding", "yaml-rust", diff --git a/src/servers/src/http.rs b/src/servers/src/http.rs index d112b63e4ec6..f3f734936fa7 100644 --- a/src/servers/src/http.rs +++ b/src/servers/src/http.rs @@ -712,6 +712,11 @@ impl HttpServer { fn route_log(log_handler: LogHandlerRef) -> Router { Router::new() .route("/logs", routing::post(handler::log_ingester)) + .layer( + ServiceBuilder::new() + .layer(HandleErrorLayer::new(handle_error)) + .layer(RequestDecompressionLayer::new()), + ) .with_state(log_handler) } diff --git a/src/servers/src/http/handler.rs b/src/servers/src/http/handler.rs index 8849ae42372c..b732e91aa3ea 100644 --- a/src/servers/src/http/handler.rs +++ b/src/servers/src/http/handler.rs @@ -92,6 +92,42 @@ fn validate_log_ingester_payload(payload: &Value) -> Result<(), String> { Ok(()) } +async fn log_ingester_inner( + state: LogHandlerRef, + query_params: LogIngesterQueryParams, + query_ctx: QueryContextRef, + mut payload: Value, +) -> Result { + let processors_ = payload["pipeline_model"].take(); + let processors = processors_.as_str().unwrap(); + let data = payload["data"].take(); + let yaml_content = Content::Yaml(processors.into()); + let pipeline: Pipeline = parse(&yaml_content)?; + + let pipeline_data = PipelineValue::try_from(data)?; + + let transformed_data: Rows = pipeline.exec(pipeline_data)?; + + let insert_request = RowInsertRequest { + rows: Some(transformed_data), + table_name: query_params.table_name.clone(), + }; + let insert_requests = RowInsertRequests { + inserts: vec![insert_request], + }; + state + .insert_log(insert_requests, query_ctx) + .await + .map(|_| { + HttpResponse::GreptimedbV1(GreptimedbV1Response { + output: vec![], + execution_time_ms: 0, + resp_metrics: HashMap::new(), + }) + }) + .map_err(|e| e.to_string()) +} + /// handler to log ingester #[axum_macros::debug_handler] pub async fn log_ingester( @@ -100,11 +136,6 @@ pub async fn log_ingester( Extension(_query_ctx): Extension, Json(mut _payload): Json, ) -> HttpResponse { - // TODO (qtang): implement log ingester - // transform payload to RowInsertRequest - // maybe we need a new trait for log ingester like `LogIngester` instead of `ServerGrpcQueryHandlerRef` - // let request = pileline::transform(payload); - // state.do_query(payload, query_ctx).await.to_string(); match validate_log_ingester_payload(&_payload) { Ok(_) => (), Err(e) => { @@ -114,52 +145,8 @@ pub async fn log_ingester( )) } }; - let processors_ = _payload["pipeline_model"].take(); - let processors = processors_.as_str().unwrap(); - let data = _payload["data"].take(); - let yaml_content = Content::Yaml(processors.into()); - let pipeline_: Result, String> = parse(&yaml_content); - match pipeline_ { - Ok(pipeline) => { - let pipeline_data = PipelineValue::try_from(data); - match pipeline_data { - Ok(pipeline_data) => { - let transformed_data: Result = pipeline.exec(pipeline_data); - match transformed_data { - Ok(rows) => { - let insert_request = RowInsertRequest { - rows: Some(rows), - table_name: _query_params.table_name.clone(), - }; - let insert_requests = RowInsertRequests { - inserts: vec![insert_request], - }; - let insert_result = - _state.insert_log(insert_requests, _query_ctx).await; - match insert_result { - Ok(_) => HttpResponse::GreptimedbV1(GreptimedbV1Response { - output: vec![], - execution_time_ms: 0, - resp_metrics: HashMap::new(), - }), - Err(e) => HttpResponse::Error(ErrorResponse::from_error_message( - StatusCode::InvalidArguments, - e.to_string(), - )), - } - } - Err(e) => HttpResponse::Error(ErrorResponse::from_error_message( - StatusCode::InvalidArguments, - e, - )), - } - } - Err(e) => HttpResponse::Error(ErrorResponse::from_error_message( - StatusCode::InvalidArguments, - e, - )), - } - } + match log_ingester_inner(_state, _query_params, _query_ctx, _payload).await { + Ok(resp) => resp, Err(e) => HttpResponse::Error(ErrorResponse::from_error_message( StatusCode::InvalidArguments, e, diff --git a/src/servers/src/query_handler.rs b/src/servers/src/query_handler.rs index f15f945df865..7664a77e1110 100644 --- a/src/servers/src/query_handler.rs +++ b/src/servers/src/query_handler.rs @@ -121,7 +121,6 @@ pub trait OpenTelemetryProtocolHandler { } #[async_trait] - pub trait LogHandler { async fn insert_log(&self, log: RowInsertRequests, ctx: QueryContextRef) -> Result; } From 2b16ef9c1fd76d543974f84d4e3e2a9c24ab427d Mon Sep 17 00:00:00 2001 From: paomian Date: Wed, 29 May 2024 21:27:09 +0800 Subject: [PATCH 07/51] chore: add pipelime http handler --- Cargo.lock | 1 + src/frontend/Cargo.toml | 2 + src/frontend/src/error.rs | 9 + src/frontend/src/lib.rs | 1 + src/frontend/src/pipeline.rs | 405 +++++++++++++++++++++++++++++++++++ 5 files changed, 418 insertions(+) create mode 100644 src/frontend/src/pipeline.rs diff --git a/Cargo.lock b/Cargo.lock index a1197b15e82b..e067e3bcd0e0 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3937,6 +3937,7 @@ dependencies = [ "opentelemetry-proto 0.5.0", "operator", "partition", + "pipeline", "prometheus", "prost 0.12.4", "query", diff --git a/src/frontend/Cargo.toml b/src/frontend/Cargo.toml index 2b8d5c746f60..1b5c4a957daa 100644 --- a/src/frontend/Cargo.toml +++ b/src/frontend/Cargo.toml @@ -60,6 +60,8 @@ table.workspace = true tokio.workspace = true toml.workspace = true tonic.workspace = true +#pipeline = { git = "ssh://git@github.com/GreptimeTeam/pipeline.git", rev = "6b88c3c627da9e20f8fd160071e9c69b3ebd4e6a" } +pipeline = { path = "../../../pipeline" } [dev-dependencies] catalog.workspace = true diff --git a/src/frontend/src/error.rs b/src/frontend/src/error.rs index 9b2a0faf6320..b75c871e7215 100644 --- a/src/frontend/src/error.rs +++ b/src/frontend/src/error.rs @@ -297,6 +297,14 @@ pub enum Error { source: table::error::Error, }, + #[snafu(display("Failed to insert pipeline to pipeline table, name: {}", name))] + InsertPipeline { + name: String, + #[snafu(implicit)] + location: Location, + source: BoxedError, + }, + #[snafu(display("Unsupported format: {:?}", format))] UnsupportedFormat { #[snafu(implicit)] @@ -383,6 +391,7 @@ impl ErrorExt for Error { | Error::EmptyData { .. } | Error::ColumnNoneDefaultValue { .. } | Error::IncompleteGrpcRequest { .. } + | Error::InsertPipeline { .. } | Error::InvalidTlsConfig { .. } => StatusCode::InvalidArguments, Error::NotSupported { .. } => StatusCode::Unsupported, diff --git a/src/frontend/src/lib.rs b/src/frontend/src/lib.rs index de800b0b41c6..90a01c19ddfb 100644 --- a/src/frontend/src/lib.rs +++ b/src/frontend/src/lib.rs @@ -22,3 +22,4 @@ pub(crate) mod metrics; mod script; pub mod server; pub mod service_config; +mod pipeline; diff --git a/src/frontend/src/pipeline.rs b/src/frontend/src/pipeline.rs new file mode 100644 index 000000000000..7ef8a5395983 --- /dev/null +++ b/src/frontend/src/pipeline.rs @@ -0,0 +1,405 @@ +// Copyright 2023 Greptime Team +// +// Licensed 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 std::collections::HashMap; +use std::result::Result as StdResult; +use std::sync::{Arc, RwLock}; + +use api::v1::ddl_request::Expr; +use api::v1::greptime_request::Request; +use api::v1::value::ValueData; +use api::v1::{ + ColumnDataType, ColumnDef, ColumnSchema as PbColumnSchema, CreateTableExpr, DdlRequest, Row, + RowInsertRequest, RowInsertRequests, Rows, SemanticType, +}; +use arc_swap::access::Access; +use arc_swap::ArcSwap; +use catalog::{CatalogManagerRef, RegisterSystemTableRequest}; +use client::DEFAULT_SCHEMA_NAME; +use common_catalog::consts::default_engine; +use common_error::ext::{BoxedError, ErrorExt}; +use common_meta::table_name::TableName; +use common_query::Output; +use common_telemetry::{error, info}; +use common_time::util; +use pipeline::transform::GreptimeTransformer; +use pipeline::{parse, Content, Pipeline}; +use servers::query_handler::grpc::GrpcQueryHandler; +use session::context::{QueryContext, QueryContextBuilder, QueryContextRef}; +use snafu::{OptionExt, ResultExt}; +use table::metadata::TableInfo; +use table::TableRef; + +use crate::error::{CatalogSnafu, Error, InsertPipelineSnafu, Result, TableNotFoundSnafu}; + +type FrontendGrpcQueryHandlerRef = Arc + Send + Sync>; +type PipelineTableRef = Arc; + +pub const PIPELINE_TABLE_NAME: &str = "pipelines"; + +struct DummyHandler; + +impl DummyHandler { + pub fn arc() -> Arc { + Arc::new(Self {}) + } +} + +#[async_trait::async_trait] +impl GrpcQueryHandler for DummyHandler { + type Error = Error; + + async fn do_query( + &self, + _query: Request, + _ctx: QueryContextRef, + ) -> std::result::Result { + unreachable!(); + } +} + +struct PipelineOperator { + grpc_handler: ArcSwap, + catalog_manager: CatalogManagerRef, + tables: RwLock>, + pipelines: RwLock>>, +} + +struct PipelineTable { + grpc_handler: FrontendGrpcQueryHandlerRef, + table: TableRef, +} + +impl PipelineTable { + pub fn new(table: TableRef, grpc_handler: FrontendGrpcQueryHandlerRef) -> Self { + let grpc_handler = DummyHandler::arc(); + Self { + grpc_handler, + table, + } + } + //ArcSwap::new(Arc::new(grpc_handler.clone() as _)) + + fn build_insert_column_schemas() -> Vec { + vec![ + PbColumnSchema { + column_name: "name".to_string(), + datatype: ColumnDataType::String.into(), + semantic_type: SemanticType::Tag.into(), + ..Default::default() + }, + PbColumnSchema { + column_name: "pipeline".to_string(), + datatype: ColumnDataType::String.into(), + semantic_type: SemanticType::Field.into(), + ..Default::default() + }, + PbColumnSchema { + column_name: "created_at".to_string(), + datatype: ColumnDataType::TimestampMillisecond.into(), + semantic_type: SemanticType::Timestamp.into(), + ..Default::default() + }, + PbColumnSchema { + column_name: "updated_at".to_string(), + datatype: ColumnDataType::TimestampMillisecond.into(), + semantic_type: SemanticType::Field.into(), + ..Default::default() + }, + ] + } + + fn query_ctx(table_info: &TableInfo) -> QueryContextRef { + QueryContextBuilder::default() + .current_catalog(table_info.catalog_name.to_string()) + .current_schema(table_info.schema_name.to_string()) + .build() + .into() + } + + pub async fn insert(&self, schema: &str, name: &str, pipeline: &str) -> Result<()> { + let now = util::current_time_millis(); + + let table_info = self.table.table_info(); + + let insert = RowInsertRequest { + table_name: PIPELINE_TABLE_NAME.to_string(), + rows: Some(Rows { + schema: Self::build_insert_column_schemas(), + rows: vec![Row { + values: vec![ + ValueData::StringValue(name.to_string()).into(), + ValueData::StringValue(pipeline.to_string()).into(), + ValueData::TimestampMillisecondValue(now).into(), + ValueData::TimestampMillisecondValue(now).into(), + ], + }], + }), + }; + + let requests = RowInsertRequests { + inserts: vec![insert], + }; + + let output = self + .grpc_handler + .do_query(Request::RowInserts(requests), Self::query_ctx(&table_info)) + .await + .map_err(BoxedError::new) + .context(InsertPipelineSnafu { name })?; + + info!( + "Inserted script: {} into scripts table: {}, output: {:?}.", + name, + table_info.full_table_name(), + output + ); + + Ok(()) + } +} + +impl PipelineOperator { + pub fn get_pipeline_table(&self, name: &str) -> Option { + self.tables.read().unwrap().get(name).cloned() + } + + pub fn build_pipeline_schema() -> (String, Vec, Vec) { + let created_at = "created_at"; + let updated_at = "updated_at"; + let pipeline_content = "pipeline"; + let pipeline_name = "name"; + + ( + created_at.to_string(), + vec![], + vec![ + ColumnDef { + name: created_at.to_string(), + data_type: ColumnDataType::TimestampMillisecond as i32, + is_nullable: false, + default_constraint: vec![], + semantic_type: SemanticType::Timestamp as i32, + comment: "".to_string(), + datatype_extension: None, + }, + ColumnDef { + name: updated_at.to_string(), + data_type: ColumnDataType::TimestampMillisecond as i32, + is_nullable: false, + default_constraint: vec![], + semantic_type: SemanticType::Field as i32, + comment: "".to_string(), + datatype_extension: None, + }, + ColumnDef { + name: pipeline_content.to_string(), + data_type: ColumnDataType::String as i32, + is_nullable: false, + default_constraint: vec![], + semantic_type: SemanticType::Tag as i32, + comment: "".to_string(), + datatype_extension: None, + }, + ColumnDef { + name: pipeline_name.to_string(), + data_type: ColumnDataType::String as i32, + is_nullable: false, + default_constraint: vec![], + semantic_type: SemanticType::Field as i32, + comment: "".to_string(), + datatype_extension: None, + }, + ], + ) + } + + pub fn create_table_request(&self, catalog: &str) -> RegisterSystemTableRequest { + let (time_index, primary_keys, column_defs) = Self::build_pipeline_schema(); + + let create_table_expr = CreateTableExpr { + catalog_name: catalog.to_string(), + // TODO(dennis): put the scripts table into `system` schema? + // We always put the scripts table into `public` schema right now. + schema_name: DEFAULT_SCHEMA_NAME.to_string(), + table_name: PIPELINE_TABLE_NAME.to_string(), + desc: "GreptimeDB scripts table for Python".to_string(), + column_defs, + time_index, + primary_keys, + create_if_not_exists: true, + table_options: Default::default(), + table_id: None, // Should and will be assigned by Meta. + engine: default_engine().to_string(), + }; + + RegisterSystemTableRequest { + create_table_expr, + open_hook: None, + } + } + + fn add_pipeline_table_to_cache(&self, catalog: &str, table: TableRef) { + let mut tables = self.tables.write().unwrap(); + if tables.contains_key(catalog) { + return; + } + // FIXME (qtang): we should add field to store in PipelineTable + tables.insert( + catalog.to_string(), + Arc::new(PipelineTable { + grpc_handler: self.grpc_handler.load().as_ref().clone(), + table, + }), + ); + } + + pub async fn create_pipeline_table_if_not_exists(&self, catalog: &str) -> Result<()> { + if let Some(_) = self.get_pipeline_table(catalog) { + return Ok(()); + } + + let RegisterSystemTableRequest { + create_table_expr: expr, + open_hook, + } = self.create_table_request(catalog); + + if let Some(table) = self + .catalog_manager + .table(&expr.catalog_name, &expr.schema_name, &expr.table_name) + .await + .context(CatalogSnafu)? + { + if let Some(open_hook) = open_hook { + (open_hook)(table.clone()).await.context(CatalogSnafu)?; + } + + self.add_pipeline_table_to_cache(catalog, table); + + return Ok(()); + } + + let table_name = TableName::new(&expr.catalog_name, &expr.schema_name, &expr.table_name); + + let _ = self + .grpc_handler + .load() + .do_query( + Request::Ddl(DdlRequest { + expr: Some(Expr::CreateTable(expr)), + }), + QueryContext::arc(), + ) + .await?; + + let table = self + .catalog_manager + .table( + &table_name.catalog_name, + &table_name.schema_name, + &table_name.table_name, + ) + .await + .context(CatalogSnafu)? + .with_context(|| TableNotFoundSnafu { + table_name: table_name.to_string(), + })?; + + if let Some(open_hook) = open_hook { + (open_hook)(table.clone()).await.context(CatalogSnafu)?; + } + + info!( + "Created scripts table {}.", + table.table_info().full_table_name() + ); + + self.add_pipeline_table_to_cache(catalog, table); + + Ok(()) + } + + pub async fn insert_and_compile( + &self, + catalog: &str, + schema: &str, + name: &str, + pipeline: &str, + ) -> Result>> { + let yaml_content = Content::Yaml(pipeline.into()); + let pipeline: StdResult, String> = parse(&yaml_content); + todo!() + } + + pub async fn insert_pipeline( + &self, + query_ctx: QueryContextRef, + name: &str, + pipeline: &str, + ) -> servers::error::Result<()> { + self.create_pipeline_table_if_not_exists(query_ctx.current_catalog()) + .await + .map_err(|e| { + if e.status_code().should_log_error() { + error!(e; "Failed to create pipeline table"); + } + + servers::error::InternalSnafu { + err_msg: e.to_string(), + } + .build() + })?; + + let _s = self + .insert_and_compile( + query_ctx.current_catalog(), + query_ctx.current_schema(), + name, + pipeline, + ) + .await + .map_err(|e| { + if e.status_code().should_log_error() { + error!(e; "Failed to insert script"); + } + + BoxedError::new(e) + }) + .context(servers::error::InsertScriptSnafu { name })?; + + Ok(()) + } +} + +impl PipelineOperator { + pub fn new(catalog_manager: CatalogManagerRef) -> Self { + let grpc_handler = ArcSwap::new(Arc::new(DummyHandler::arc() as _)); + Self { + grpc_handler: grpc_handler, + catalog_manager, + tables: RwLock::new(HashMap::new()), + pipelines: RwLock::new(HashMap::new()), + } + } + + // FIXME (qtang): we should impl this + pub async fn execute_script( + &self, + query_ctx: QueryContextRef, + name: &str, + params: HashMap, + ) -> servers::error::Result<()> { + todo!() + } +} From f1350cd2391ceaa2ef2f8f3ef33b45e96ff70052 Mon Sep 17 00:00:00 2001 From: paomian Date: Thu, 30 May 2024 17:29:01 +0800 Subject: [PATCH 08/51] chore: add pipeline private table --- Cargo.lock | 54 +++- Cargo.toml | 3 +- src/frontend/Cargo.toml | 2 +- src/frontend/src/error.rs | 9 + src/frontend/src/instance.rs | 3 + src/frontend/src/instance/builder.rs | 7 + src/frontend/src/instance/log_handler.rs | 27 ++ src/frontend/src/pipeline.rs | 314 +++++++---------------- src/pipeline/Cargo.toml | 61 +++++ src/pipeline/src/error.rs | 103 ++++++++ src/pipeline/src/lib.rs | 5 + src/pipeline/src/table.rs | 298 +++++++++++++++++++++ src/script/src/table.rs | 2 +- src/servers/src/error.rs | 9 + src/servers/src/http/handler.rs | 12 +- src/servers/src/query_handler.rs | 14 + 16 files changed, 695 insertions(+), 228 deletions(-) create mode 100644 src/pipeline/Cargo.toml create mode 100644 src/pipeline/src/error.rs create mode 100644 src/pipeline/src/lib.rs create mode 100644 src/pipeline/src/table.rs diff --git a/Cargo.lock b/Cargo.lock index e067e3bcd0e0..8bf0213120e1 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3937,7 +3937,7 @@ dependencies = [ "opentelemetry-proto 0.5.0", "operator", "partition", - "pipeline", + "pipeline 0.8.0", "prometheus", "prost 0.12.4", "query", @@ -7182,6 +7182,56 @@ dependencies = [ "yaml-rust", ] +[[package]] +name = "pipeline" +version = "0.8.0" +dependencies = [ + "api", + "arc-swap", + "arrow", + "async-trait", + "catalog", + "common-catalog", + "common-error", + "common-function", + "common-macro", + "common-meta", + "common-query", + "common-recordbatch", + "common-runtime", + "common-telemetry", + "common-test-util", + "common-time", + "console", + "criterion", + "crossbeam-utils", + "datafusion 37.0.0", + "datafusion-common 37.0.0", + "datafusion-expr 37.0.0", + "datafusion-functions 37.0.0", + "datafusion-physical-expr 37.0.0", + "datatypes", + "futures", + "lazy_static", + "log-store", + "once_cell", + "operator", + "paste", + "pipeline 0.1.0", + "prometheus", + "query", + "rayon", + "ron", + "serde", + "servers", + "session", + "snafu 0.8.2", + "sql", + "table", + "tokio", + "tokio-test", +] + [[package]] name = "pkcs1" version = "0.3.3" @@ -9576,7 +9626,7 @@ dependencies = [ "permutation", "pgwire", "pin-project", - "pipeline", + "pipeline 0.1.0", "postgres-types", "pprof", "prometheus", diff --git a/Cargo.toml b/Cargo.toml index a5b1f5aed9cc..2c3f370cc409 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -59,7 +59,7 @@ members = [ "src/index", "tests-fuzz", "tests-integration", - "tests/runner", + "tests/runner", "src/pipeline", ] resolver = "2" @@ -227,6 +227,7 @@ promql = { path = "src/promql" } puffin = { path = "src/puffin" } query = { path = "src/query" } script = { path = "src/script" } +pipeline = { path = "src/pipeline" } servers = { path = "src/servers" } session = { path = "src/session" } sql = { path = "src/sql" } diff --git a/src/frontend/Cargo.toml b/src/frontend/Cargo.toml index 1b5c4a957daa..095674f56d32 100644 --- a/src/frontend/Cargo.toml +++ b/src/frontend/Cargo.toml @@ -44,6 +44,7 @@ meta-client.workspace = true opentelemetry-proto.workspace = true operator.workspace = true partition.workspace = true +pipeline = { workspace = true } prometheus.workspace = true prost.workspace = true query.workspace = true @@ -61,7 +62,6 @@ tokio.workspace = true toml.workspace = true tonic.workspace = true #pipeline = { git = "ssh://git@github.com/GreptimeTeam/pipeline.git", rev = "6b88c3c627da9e20f8fd160071e9c69b3ebd4e6a" } -pipeline = { path = "../../../pipeline" } [dev-dependencies] catalog.workspace = true diff --git a/src/frontend/src/error.rs b/src/frontend/src/error.rs index b75c871e7215..40e4db6f1f54 100644 --- a/src/frontend/src/error.rs +++ b/src/frontend/src/error.rs @@ -305,6 +305,14 @@ pub enum Error { source: BoxedError, }, + #[snafu(display("Failed to get pipeline to pipeline table, name: {}", name))] + GetPipeline { + name: String, + #[snafu(implicit)] + location: Location, + source: BoxedError, + }, + #[snafu(display("Unsupported format: {:?}", format))] UnsupportedFormat { #[snafu(implicit)] @@ -392,6 +400,7 @@ impl ErrorExt for Error { | Error::ColumnNoneDefaultValue { .. } | Error::IncompleteGrpcRequest { .. } | Error::InsertPipeline { .. } + | Error::GetPipeline { .. } | Error::InvalidTlsConfig { .. } => StatusCode::InvalidArguments, Error::NotSupported { .. } => StatusCode::Unsupported, diff --git a/src/frontend/src/instance.rs b/src/frontend/src/instance.rs index 860ed700bb6c..dcab06fa68dd 100644 --- a/src/frontend/src/instance.rs +++ b/src/frontend/src/instance.rs @@ -89,6 +89,7 @@ use crate::error::{ }; use crate::frontend::FrontendOptions; use crate::heartbeat::HeartbeatTask; +use crate::pipeline::PipelineOperator; use crate::script::ScriptExecutor; #[async_trait] @@ -116,6 +117,7 @@ pub type StatementExecutorRef = Arc; pub struct Instance { catalog_manager: CatalogManagerRef, script_executor: Arc, + pipeline_operator: Arc, statement_executor: Arc, query_engine: QueryEngineRef, plugins: Plugins, @@ -265,6 +267,7 @@ impl FrontendInstance for Instance { } self.script_executor.start(self)?; + self.pipeline_operator.start(self); if let Some(t) = self.export_metrics_task.as_ref() { if t.send_by_handler { diff --git a/src/frontend/src/instance/builder.rs b/src/frontend/src/instance/builder.rs index f0993458a6aa..90634daf1523 100644 --- a/src/frontend/src/instance/builder.rs +++ b/src/frontend/src/instance/builder.rs @@ -38,6 +38,7 @@ use crate::error::{self, Result}; use crate::heartbeat::HeartbeatTask; use crate::instance::region_query::FrontendRegionQueryHandler; use crate::instance::{Instance, StatementExecutorRef}; +use crate::pipeline::PipelineOperator; use crate::script::ScriptExecutor; /// The frontend [`Instance`] builder. @@ -172,11 +173,17 @@ impl FrontendBuilder { table_route_cache, )); + let pipeline_operator = Arc::new(PipelineOperator::new( + self.catalog_manager.clone(), + query_engine.clone(), + )); + plugins.insert::(statement_executor.clone()); Ok(Instance { catalog_manager: self.catalog_manager, script_executor, + pipeline_operator, statement_executor, query_engine, plugins, diff --git a/src/frontend/src/instance/log_handler.rs b/src/frontend/src/instance/log_handler.rs index ffbf94f5c54e..c5edbce92de9 100644 --- a/src/frontend/src/instance/log_handler.rs +++ b/src/frontend/src/instance/log_handler.rs @@ -17,6 +17,7 @@ use async_trait::async_trait; use auth::{PermissionChecker, PermissionCheckerRef, PermissionReq}; use client::Output; use common_error::ext::BoxedError; +use pipeline::{GreptimeTransformer, Pipeline}; use servers::error::{AuthSnafu, ExecuteGrpcRequestSnafu}; use servers::query_handler::LogHandler; use session::context::QueryContextRef; @@ -40,6 +41,32 @@ impl LogHandler for Instance { self.handle_log_inserts(log, ctx).await } + + async fn get_pipeline( + &self, + query_ctx: QueryContextRef, + name: &str, + ) -> servers::error::Result> { + self.pipeline_operator + .get_pipeline(query_ctx, name) + .await + .map_err(BoxedError::new) + .context(servers::error::InsertPipelineSnafu { name }) + } + async fn insert_pipeline( + &self, + query_ctx: QueryContextRef, + name: &str, + content_type: &str, + pipeline: &str, + ) -> servers::error::Result<()> { + self.pipeline_operator + .insert_pipeline(query_ctx, name, content_type, pipeline) + .await + .map_err(BoxedError::new) + .context(servers::error::InsertPipelineSnafu { name })?; + Ok(()) + } } impl Instance { diff --git a/src/frontend/src/pipeline.rs b/src/frontend/src/pipeline.rs index 7ef8a5395983..ae0b5b52fe13 100644 --- a/src/frontend/src/pipeline.rs +++ b/src/frontend/src/pipeline.rs @@ -13,38 +13,32 @@ // limitations under the License. use std::collections::HashMap; -use std::result::Result as StdResult; use std::sync::{Arc, RwLock}; use api::v1::ddl_request::Expr; use api::v1::greptime_request::Request; -use api::v1::value::ValueData; -use api::v1::{ - ColumnDataType, ColumnDef, ColumnSchema as PbColumnSchema, CreateTableExpr, DdlRequest, Row, - RowInsertRequest, RowInsertRequests, Rows, SemanticType, -}; -use arc_swap::access::Access; +use api::v1::{CreateTableExpr, DdlRequest}; use arc_swap::ArcSwap; use catalog::{CatalogManagerRef, RegisterSystemTableRequest}; -use client::DEFAULT_SCHEMA_NAME; -use common_catalog::consts::default_engine; +use common_catalog::consts::{default_engine, DEFAULT_PRIVATE_SCHEMA_NAME}; use common_error::ext::{BoxedError, ErrorExt}; use common_meta::table_name::TableName; use common_query::Output; use common_telemetry::{error, info}; -use common_time::util; -use pipeline::transform::GreptimeTransformer; -use pipeline::{parse, Content, Pipeline}; +use pipeline::table::{PipelineTable, PipelineTableRef}; +use pipeline::{GreptimeTransformer, Pipeline}; +use query::QueryEngineRef; use servers::query_handler::grpc::GrpcQueryHandler; -use session::context::{QueryContext, QueryContextBuilder, QueryContextRef}; +use session::context::{QueryContext, QueryContextRef}; use snafu::{OptionExt, ResultExt}; -use table::metadata::TableInfo; use table::TableRef; -use crate::error::{CatalogSnafu, Error, InsertPipelineSnafu, Result, TableNotFoundSnafu}; +use crate::error::{ + CatalogSnafu, Error, GetPipelineSnafu, InsertPipelineSnafu, Result, TableNotFoundSnafu, +}; +use crate::instance::Instance; type FrontendGrpcQueryHandlerRef = Arc + Send + Sync>; -type PipelineTableRef = Arc; pub const PIPELINE_TABLE_NAME: &str = "pipelines"; @@ -69,170 +63,21 @@ impl GrpcQueryHandler for DummyHandler { } } -struct PipelineOperator { +pub struct PipelineOperator { grpc_handler: ArcSwap, catalog_manager: CatalogManagerRef, - tables: RwLock>, - pipelines: RwLock>>, -} - -struct PipelineTable { - grpc_handler: FrontendGrpcQueryHandlerRef, - table: TableRef, -} - -impl PipelineTable { - pub fn new(table: TableRef, grpc_handler: FrontendGrpcQueryHandlerRef) -> Self { - let grpc_handler = DummyHandler::arc(); - Self { - grpc_handler, - table, - } - } - //ArcSwap::new(Arc::new(grpc_handler.clone() as _)) - - fn build_insert_column_schemas() -> Vec { - vec![ - PbColumnSchema { - column_name: "name".to_string(), - datatype: ColumnDataType::String.into(), - semantic_type: SemanticType::Tag.into(), - ..Default::default() - }, - PbColumnSchema { - column_name: "pipeline".to_string(), - datatype: ColumnDataType::String.into(), - semantic_type: SemanticType::Field.into(), - ..Default::default() - }, - PbColumnSchema { - column_name: "created_at".to_string(), - datatype: ColumnDataType::TimestampMillisecond.into(), - semantic_type: SemanticType::Timestamp.into(), - ..Default::default() - }, - PbColumnSchema { - column_name: "updated_at".to_string(), - datatype: ColumnDataType::TimestampMillisecond.into(), - semantic_type: SemanticType::Field.into(), - ..Default::default() - }, - ] - } - - fn query_ctx(table_info: &TableInfo) -> QueryContextRef { - QueryContextBuilder::default() - .current_catalog(table_info.catalog_name.to_string()) - .current_schema(table_info.schema_name.to_string()) - .build() - .into() - } - - pub async fn insert(&self, schema: &str, name: &str, pipeline: &str) -> Result<()> { - let now = util::current_time_millis(); - - let table_info = self.table.table_info(); - - let insert = RowInsertRequest { - table_name: PIPELINE_TABLE_NAME.to_string(), - rows: Some(Rows { - schema: Self::build_insert_column_schemas(), - rows: vec![Row { - values: vec![ - ValueData::StringValue(name.to_string()).into(), - ValueData::StringValue(pipeline.to_string()).into(), - ValueData::TimestampMillisecondValue(now).into(), - ValueData::TimestampMillisecondValue(now).into(), - ], - }], - }), - }; - - let requests = RowInsertRequests { - inserts: vec![insert], - }; - - let output = self - .grpc_handler - .do_query(Request::RowInserts(requests), Self::query_ctx(&table_info)) - .await - .map_err(BoxedError::new) - .context(InsertPipelineSnafu { name })?; - - info!( - "Inserted script: {} into scripts table: {}, output: {:?}.", - name, - table_info.full_table_name(), - output - ); - - Ok(()) - } + query_engine: QueryEngineRef, + tables: RwLock>>, } impl PipelineOperator { - pub fn get_pipeline_table(&self, name: &str) -> Option { - self.tables.read().unwrap().get(name).cloned() - } - - pub fn build_pipeline_schema() -> (String, Vec, Vec) { - let created_at = "created_at"; - let updated_at = "updated_at"; - let pipeline_content = "pipeline"; - let pipeline_name = "name"; - - ( - created_at.to_string(), - vec![], - vec![ - ColumnDef { - name: created_at.to_string(), - data_type: ColumnDataType::TimestampMillisecond as i32, - is_nullable: false, - default_constraint: vec![], - semantic_type: SemanticType::Timestamp as i32, - comment: "".to_string(), - datatype_extension: None, - }, - ColumnDef { - name: updated_at.to_string(), - data_type: ColumnDataType::TimestampMillisecond as i32, - is_nullable: false, - default_constraint: vec![], - semantic_type: SemanticType::Field as i32, - comment: "".to_string(), - datatype_extension: None, - }, - ColumnDef { - name: pipeline_content.to_string(), - data_type: ColumnDataType::String as i32, - is_nullable: false, - default_constraint: vec![], - semantic_type: SemanticType::Tag as i32, - comment: "".to_string(), - datatype_extension: None, - }, - ColumnDef { - name: pipeline_name.to_string(), - data_type: ColumnDataType::String as i32, - is_nullable: false, - default_constraint: vec![], - semantic_type: SemanticType::Field as i32, - comment: "".to_string(), - datatype_extension: None, - }, - ], - ) - } - pub fn create_table_request(&self, catalog: &str) -> RegisterSystemTableRequest { - let (time_index, primary_keys, column_defs) = Self::build_pipeline_schema(); + let (time_index, primary_keys, column_defs) = + PipelineTable::::build_pipeline_schema(); let create_table_expr = CreateTableExpr { catalog_name: catalog.to_string(), - // TODO(dennis): put the scripts table into `system` schema? - // We always put the scripts table into `public` schema right now. - schema_name: DEFAULT_SCHEMA_NAME.to_string(), + schema_name: DEFAULT_PRIVATE_SCHEMA_NAME.to_string(), table_name: PIPELINE_TABLE_NAME.to_string(), desc: "GreptimeDB scripts table for Python".to_string(), column_defs, @@ -255,18 +100,18 @@ impl PipelineOperator { if tables.contains_key(catalog) { return; } - // FIXME (qtang): we should add field to store in PipelineTable tables.insert( catalog.to_string(), - Arc::new(PipelineTable { - grpc_handler: self.grpc_handler.load().as_ref().clone(), + Arc::new(PipelineTable::new( table, - }), + self.grpc_handler.load().as_ref().clone(), + self.query_engine.clone(), + )), ); } pub async fn create_pipeline_table_if_not_exists(&self, catalog: &str) -> Result<()> { - if let Some(_) = self.get_pipeline_table(catalog) { + if self.get_pipeline_table_from_cache(catalog).is_some() { return Ok(()); } @@ -330,22 +175,79 @@ impl PipelineOperator { Ok(()) } + pub fn get_pipeline_table_from_cache(&self, catalog: &str) -> Option> { + // FIXME (qtang): we should impl this + self.tables.read().unwrap().get(catalog).cloned() + } + pub async fn insert_and_compile( &self, catalog: &str, schema: &str, name: &str, + content_type: &str, pipeline: &str, - ) -> Result>> { - let yaml_content = Content::Yaml(pipeline.into()); - let pipeline: StdResult, String> = parse(&yaml_content); - todo!() + ) -> Result<()> { + let _compiled_pipeline = PipelineTable::::compile_pipeline(name, pipeline) + .map_err(BoxedError::new) + .context(InsertPipelineSnafu { name })?; + self.get_pipeline_table_from_cache(catalog) + // FIXME (qtang): we should add error handling here + .unwrap() + .insert_pipeline_to_pipeline_table(schema, name, content_type, pipeline) + .await + .map_err(|e| { + if e.status_code().should_log_error() { + error!(e; "Failed to insert pipeline"); + } + BoxedError::new(e) + }) + .context(InsertPipelineSnafu { name })?; + Ok(()) + } +} + +impl PipelineOperator { + pub fn new(catalog_manager: CatalogManagerRef, query_engine: QueryEngineRef) -> Self { + let grpc_handler = ArcSwap::new(Arc::new(DummyHandler::arc() as _)); + Self { + grpc_handler, + catalog_manager, + tables: RwLock::new(HashMap::new()), + query_engine, + } + } + + pub fn start(&self, instance: &Instance) { + self.grpc_handler + .store(Arc::new(Arc::new(instance.clone()) as _)); + } + + pub async fn get_pipeline( + &self, + query_ctx: QueryContextRef, + name: &str, + ) -> Result> { + let _s = self + .create_pipeline_table_if_not_exists(query_ctx.current_catalog()) + .await; + let table = self + .get_pipeline_table_from_cache(query_ctx.current_catalog()) + .unwrap() + .find_pipeline_by_name(query_ctx.current_schema(), name) + .await + .map_err(BoxedError::new) + .context(GetPipelineSnafu { name })?; + PipelineTable::::compile_pipeline(name, &table) + .map_err(BoxedError::new) + .context(InsertPipelineSnafu { name }) } pub async fn insert_pipeline( &self, query_ctx: QueryContextRef, name: &str, + content_type: &str, pipeline: &str, ) -> servers::error::Result<()> { self.create_pipeline_table_if_not_exists(query_ctx.current_catalog()) @@ -361,45 +263,23 @@ impl PipelineOperator { .build() })?; - let _s = self - .insert_and_compile( - query_ctx.current_catalog(), - query_ctx.current_schema(), - name, - pipeline, - ) - .await - .map_err(|e| { - if e.status_code().should_log_error() { - error!(e; "Failed to insert script"); - } + self.insert_and_compile( + query_ctx.current_catalog(), + query_ctx.current_schema(), + name, + content_type, + pipeline, + ) + .await + .map_err(|e| { + if e.status_code().should_log_error() { + error!(e; "Failed to insert pipeline"); + } - BoxedError::new(e) - }) - .context(servers::error::InsertScriptSnafu { name })?; + BoxedError::new(e) + }) + .context(servers::error::InsertPipelineSnafu { name })?; Ok(()) } } - -impl PipelineOperator { - pub fn new(catalog_manager: CatalogManagerRef) -> Self { - let grpc_handler = ArcSwap::new(Arc::new(DummyHandler::arc() as _)); - Self { - grpc_handler: grpc_handler, - catalog_manager, - tables: RwLock::new(HashMap::new()), - pipelines: RwLock::new(HashMap::new()), - } - } - - // FIXME (qtang): we should impl this - pub async fn execute_script( - &self, - query_ctx: QueryContextRef, - name: &str, - params: HashMap, - ) -> servers::error::Result<()> { - todo!() - } -} diff --git a/src/pipeline/Cargo.toml b/src/pipeline/Cargo.toml new file mode 100644 index 000000000000..0998d19aacc5 --- /dev/null +++ b/src/pipeline/Cargo.toml @@ -0,0 +1,61 @@ +[package] +name = "pipeline" +edition.workspace = true +version.workspace = true +license.workspace = true + +[features] + + +[lints] +workspace = true + +[dependencies] +api.workspace = true +arc-swap = "1.0" +arrow.workspace = true +async-trait.workspace = true +catalog.workspace = true +common-catalog.workspace = true +common-error.workspace = true +common-function.workspace = true +common-macro.workspace = true +common-query.workspace = true +common-recordbatch.workspace = true +common-runtime.workspace = true +common-telemetry.workspace = true +common-time.workspace = true +common-meta.workspace = true +console = "0.15" +crossbeam-utils.workspace = true +datafusion = { workspace = true} +datafusion-common = { workspace = true} +datafusion-expr = { workspace = true} +datafusion-functions = { workspace = true} +datafusion-physical-expr = { workspace = true} +datatypes.workspace = true +futures.workspace = true +lazy_static.workspace = true +once_cell.workspace = true +paste = { workspace = true} +prometheus.workspace = true +query.workspace = true +servers.workspace = true +session.workspace = true +snafu.workspace = true +sql.workspace = true +table.workspace = true +tokio.workspace = true +pipeline = { path = "../../../pipeline" } + +[dev-dependencies] +catalog = { workspace = true, features = ["testing"] } +common-test-util.workspace = true +criterion = { version = "0.4", features = ["html_reports", "async_tokio"] } +log-store.workspace = true +operator.workspace = true +rayon = "1.0" +ron = "0.7" +serde = { version = "1.0", features = ["derive"] } +session = { workspace = true, features = ["testing"] } +tokio-test = "0.4" diff --git a/src/pipeline/src/error.rs b/src/pipeline/src/error.rs new file mode 100644 index 000000000000..652ae9a0484a --- /dev/null +++ b/src/pipeline/src/error.rs @@ -0,0 +1,103 @@ +// Copyright 2023 Greptime Team +// +// Licensed 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 std::any::Any; + +use common_error::ext::{BoxedError, ErrorExt}; +use common_error::status_code::StatusCode; +use common_macro::stack_trace_debug; +use snafu::{Location, Snafu}; + +#[derive(Snafu)] +#[snafu(visibility(pub))] +#[stack_trace_debug] +pub enum Error { + #[snafu(display("Failed to find column in pipeline table, name: {}", name))] + FindColumnInPipelineTable { + name: String, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("Pipeline table not found"))] + PipelineTableNotFound { + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("Failed to insert pipeline to pipelines table, name: {}", name))] + InsertPipeline { + name: String, + #[snafu(implicit)] + location: Location, + source: BoxedError, + }, + + #[snafu(display("Pipeline not found, name: {}", name))] + PipelineNotFound { + #[snafu(implicit)] + location: Location, + name: String, + }, + + #[snafu(display("Failed to collect record batch"))] + CollectRecords { + #[snafu(implicit)] + location: Location, + source: common_recordbatch::error::Error, + }, + + #[snafu(display("Failed to cast type, msg: {}", msg))] + CastType { + msg: String, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("Failed to build DataFusion logical plan"))] + BuildDfLogicalPlan { + #[snafu(source)] + error: datafusion_common::DataFusionError, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("Failed to execute internal statement"))] + ExecuteInternalStatement { + source: query::error::Error, + #[snafu(implicit)] + location: Location, + }, +} + +pub type Result = std::result::Result; + +impl ErrorExt for Error { + fn status_code(&self) -> StatusCode { + use Error::*; + match self { + FindColumnInPipelineTable { .. } | CastType { .. } => StatusCode::Unexpected, + PipelineTableNotFound { .. } => StatusCode::TableNotFound, + InsertPipeline { source, .. } => source.status_code(), + CollectRecords { source, .. } => source.status_code(), + PipelineNotFound { .. } => StatusCode::InvalidArguments, + BuildDfLogicalPlan { .. } => StatusCode::Internal, + ExecuteInternalStatement { source, .. } => source.status_code(), + } + } + + fn as_any(&self) -> &dyn Any { + self + } +} diff --git a/src/pipeline/src/lib.rs b/src/pipeline/src/lib.rs new file mode 100644 index 000000000000..e102ce489a3e --- /dev/null +++ b/src/pipeline/src/lib.rs @@ -0,0 +1,5 @@ +pub mod error; +pub mod table; + +pub use pipeline::transform::GreptimeTransformer; +pub use pipeline::Pipeline; diff --git a/src/pipeline/src/table.rs b/src/pipeline/src/table.rs new file mode 100644 index 000000000000..6e20c475177f --- /dev/null +++ b/src/pipeline/src/table.rs @@ -0,0 +1,298 @@ +use std::sync::Arc; + +use api::v1::greptime_request::Request; +use api::v1::value::ValueData; +use api::v1::{ + ColumnDataType, ColumnDef, ColumnSchema as PbColumnSchema, Row, RowInsertRequest, + RowInsertRequests, Rows, SemanticType, +}; +use common_error::ext::{BoxedError, ErrorExt, PlainError}; +use common_error::status_code::StatusCode; +use common_query::OutputData; +use common_recordbatch::util as record_util; +use common_telemetry::info; +use common_time::util; +use datafusion::datasource::DefaultTableSource; +use datafusion::logical_expr::{and, col, lit}; +use datafusion_common::TableReference; +use datafusion_expr::LogicalPlanBuilder; +use datatypes::prelude::ScalarVector; +use datatypes::vectors::{StringVector, Vector}; +use pipeline::transform::GreptimeTransformer; +use pipeline::{parse, Content, Pipeline}; +use query::plan::LogicalPlan; +use query::QueryEngineRef; +use servers::query_handler::grpc::GrpcQueryHandlerRef; +use session::context::{QueryContextBuilder, QueryContextRef}; +use snafu::{ensure, OptionExt, ResultExt}; +use table::metadata::TableInfo; +use table::table::adapter::DfTableProviderAdapter; +use table::TableRef; + +use crate::error::{ + BuildDfLogicalPlanSnafu, CastTypeSnafu, CollectRecordsSnafu, ExecuteInternalStatementSnafu, + InsertPipelineSnafu, PipelineNotFoundSnafu, Result, +}; + +pub type PipelineTableRef = Arc>; + +pub const PIPELINE_TABLE_NAME: &str = "pipelines"; +pub struct PipelineTable { + grpc_handler: GrpcQueryHandlerRef, + table: TableRef, + query_engine: QueryEngineRef, +} + +impl PipelineTable { + pub fn new( + table: TableRef, + grpc_handler: GrpcQueryHandlerRef, + query_engine: QueryEngineRef, + ) -> Self { + Self { + grpc_handler, + table, + query_engine, + } + } + + pub fn build_pipeline_schema() -> (String, Vec, Vec) { + let pipeline_name = "name"; + let schema_name = "schema"; + let content_type = "content_type"; + let pipeline_content = "pipeline"; + let created_at = "created_at"; + let updated_at = "updated_at"; + + ( + created_at.to_string(), + vec![ + schema_name.to_string(), + pipeline_name.to_string(), + content_type.to_string(), + ], + vec![ + ColumnDef { + name: pipeline_name.to_string(), + data_type: ColumnDataType::String as i32, + is_nullable: false, + default_constraint: vec![], + semantic_type: SemanticType::Tag as i32, + comment: "".to_string(), + datatype_extension: None, + }, + ColumnDef { + name: schema_name.to_string(), + data_type: ColumnDataType::String as i32, + is_nullable: false, + default_constraint: vec![], + semantic_type: SemanticType::Tag as i32, + comment: "".to_string(), + datatype_extension: None, + }, + ColumnDef { + name: content_type.to_string(), + data_type: ColumnDataType::String as i32, + is_nullable: false, + default_constraint: vec![], + semantic_type: SemanticType::Tag as i32, + comment: "".to_string(), + datatype_extension: None, + }, + ColumnDef { + name: pipeline_content.to_string(), + data_type: ColumnDataType::String as i32, + is_nullable: false, + default_constraint: vec![], + semantic_type: SemanticType::Field as i32, + comment: "".to_string(), + datatype_extension: None, + }, + ColumnDef { + name: created_at.to_string(), + data_type: ColumnDataType::TimestampMillisecond as i32, + is_nullable: false, + default_constraint: vec![], + semantic_type: SemanticType::Timestamp as i32, + comment: "".to_string(), + datatype_extension: None, + }, + ColumnDef { + name: updated_at.to_string(), + data_type: ColumnDataType::TimestampMillisecond as i32, + is_nullable: false, + default_constraint: vec![], + semantic_type: SemanticType::Field as i32, + comment: "".to_string(), + datatype_extension: None, + }, + ], + ) + } + + fn build_insert_column_schemas() -> Vec { + vec![ + PbColumnSchema { + column_name: "name".to_string(), + datatype: ColumnDataType::String.into(), + semantic_type: SemanticType::Tag.into(), + ..Default::default() + }, + PbColumnSchema { + column_name: "schema".to_string(), + datatype: ColumnDataType::String.into(), + semantic_type: SemanticType::Tag.into(), + ..Default::default() + }, + PbColumnSchema { + column_name: "content_type".to_string(), + datatype: ColumnDataType::String.into(), + semantic_type: SemanticType::Tag.into(), + ..Default::default() + }, + PbColumnSchema { + column_name: "pipeline".to_string(), + datatype: ColumnDataType::String.into(), + semantic_type: SemanticType::Field.into(), + ..Default::default() + }, + PbColumnSchema { + column_name: "created_at".to_string(), + datatype: ColumnDataType::TimestampMillisecond.into(), + semantic_type: SemanticType::Timestamp.into(), + ..Default::default() + }, + PbColumnSchema { + column_name: "updated_at".to_string(), + datatype: ColumnDataType::TimestampMillisecond.into(), + semantic_type: SemanticType::Field.into(), + ..Default::default() + }, + ] + } + + fn query_ctx(table_info: &TableInfo) -> QueryContextRef { + QueryContextBuilder::default() + .current_catalog(table_info.catalog_name.to_string()) + .current_schema(table_info.schema_name.to_string()) + .build() + .into() + } + + pub fn compile_pipeline(name: &str, pipeline: &str) -> Result> { + let yaml_content = Content::Yaml(pipeline.into()); + parse::(&yaml_content) + .map_err(|e| BoxedError::new(PlainError::new(e, StatusCode::InvalidArguments))) + .context(InsertPipelineSnafu { name }) + } + + pub async fn insert_pipeline_to_pipeline_table( + &self, + schema: &str, + name: &str, + content_type: &str, + pipeline: &str, + ) -> Result<()> { + let now = util::current_time_millis(); + + let table_info = self.table.table_info(); + + let insert = RowInsertRequest { + table_name: PIPELINE_TABLE_NAME.to_string(), + rows: Some(Rows { + schema: Self::build_insert_column_schemas(), + rows: vec![Row { + values: vec![ + ValueData::StringValue(name.to_string()).into(), + ValueData::StringValue(schema.to_string()).into(), + ValueData::StringValue(content_type.to_string()).into(), + ValueData::StringValue(pipeline.to_string()).into(), + ValueData::TimestampMillisecondValue(now).into(), + ValueData::TimestampMillisecondValue(now).into(), + ], + }], + }), + }; + + let requests = RowInsertRequests { + inserts: vec![insert], + }; + + let output = self + .grpc_handler + .do_query(Request::RowInserts(requests), Self::query_ctx(&table_info)) + .await + .map_err(BoxedError::new) + .context(InsertPipelineSnafu { name })?; + + info!( + "Inserted pipeline: {} into {} table: {}, output: {:?}.", + name, + PIPELINE_TABLE_NAME, + table_info.full_table_name(), + output + ); + + Ok(()) + } + pub async fn find_pipeline_by_name(&self, schema: &str, name: &str) -> Result { + let table_info = self.table.table_info(); + + let table_name = TableReference::full( + table_info.catalog_name.clone(), + table_info.schema_name.clone(), + table_info.name.clone(), + ); + + let table_provider = Arc::new(DfTableProviderAdapter::new(self.table.clone())); + let table_source = Arc::new(DefaultTableSource::new(table_provider)); + + let plan = LogicalPlanBuilder::scan(table_name, table_source, None) + .context(BuildDfLogicalPlanSnafu)? + .filter(and( + col("schema").eq(lit(schema)), + col("name").eq(lit(name)), + )) + .context(BuildDfLogicalPlanSnafu)? + .project(vec![col("pipeline")]) + .context(BuildDfLogicalPlanSnafu)? + .build() + .context(BuildDfLogicalPlanSnafu)?; + + let output = self + .query_engine + .execute(LogicalPlan::DfPlan(plan), Self::query_ctx(&table_info)) + .await + .context(ExecuteInternalStatementSnafu)?; + let stream = match output.data { + OutputData::Stream(stream) => stream, + OutputData::RecordBatches(record_batches) => record_batches.as_stream(), + _ => unreachable!(), + }; + + let records = record_util::collect(stream) + .await + .context(CollectRecordsSnafu)?; + + ensure!(!records.is_empty(), PipelineNotFoundSnafu { name }); + + assert_eq!(records.len(), 1); + assert_eq!(records[0].num_columns(), 1); + + let script_column = records[0].column(0); + let script_column = script_column + .as_any() + .downcast_ref::() + .with_context(|| CastTypeSnafu { + msg: format!( + "can't downcast {:?} array into string vector", + script_column.data_type() + ), + })?; + + assert_eq!(script_column.len(), 1); + + // Safety: asserted above + Ok(script_column.get_data(0).unwrap().to_string()) + } +} diff --git a/src/script/src/table.rs b/src/script/src/table.rs index 5eacf1ff7ce9..1ba160028d1d 100644 --- a/src/script/src/table.rs +++ b/src/script/src/table.rs @@ -292,7 +292,7 @@ impl ScriptsTable { } /// Build the inserted column schemas -fn build_insert_column_schemas() -> Vec { +fn build_insert_column_schemas() -> Vec { vec![ // The schema that script belongs to. PbColumnSchema { diff --git a/src/servers/src/error.rs b/src/servers/src/error.rs index 7515b767e235..1b26ac541370 100644 --- a/src/servers/src/error.rs +++ b/src/servers/src/error.rs @@ -148,6 +148,14 @@ pub enum Error { source: BoxedError, }, + #[snafu(display("Failed to insert pipeline with name: {}", name))] + InsertPipeline { + name: String, + #[snafu(implicit)] + location: Location, + source: BoxedError, + }, + #[snafu(display("Failed to execute script by name: {}", name))] ExecuteScript { name: String, @@ -607,6 +615,7 @@ impl ErrorExt for Error { CollectRecordbatch { .. } => StatusCode::EngineExecuteQuery, InsertScript { source, .. } + | InsertPipeline { source, .. } | ExecuteScript { source, .. } | ExecuteQuery { source, .. } | ExecutePlan { source, .. } diff --git a/src/servers/src/http/handler.rs b/src/servers/src/http/handler.rs index b732e91aa3ea..ad0dc9767aa5 100644 --- a/src/servers/src/http/handler.rs +++ b/src/servers/src/http/handler.rs @@ -27,9 +27,7 @@ use common_plugins::GREPTIME_EXEC_WRITE_COST; use common_query::{Output, OutputData}; use common_recordbatch::util; use common_telemetry::tracing; -use pipeline::transform::GreptimeTransformer; use pipeline::value::Value as PipelineValue; -use pipeline::{parse, Content, Pipeline}; use query::parser::{PromQuery, DEFAULT_LOOKBACK_STRING}; use schemars::JsonSchema; use serde::{Deserialize, Serialize}; @@ -98,14 +96,16 @@ async fn log_ingester_inner( query_ctx: QueryContextRef, mut payload: Value, ) -> Result { - let processors_ = payload["pipeline_model"].take(); - let processors = processors_.as_str().unwrap(); + let pipeline_id = payload["pipeline_id"].take(); + let pipeline_id = pipeline_id.as_str().unwrap(); let data = payload["data"].take(); - let yaml_content = Content::Yaml(processors.into()); - let pipeline: Pipeline = parse(&yaml_content)?; let pipeline_data = PipelineValue::try_from(data)?; + let pipeline = state + .get_pipeline(query_ctx.clone(), pipeline_id) + .await + .map_err(|e| e.to_string())?; let transformed_data: Rows = pipeline.exec(pipeline_data)?; let insert_request = RowInsertRequest { diff --git a/src/servers/src/query_handler.rs b/src/servers/src/query_handler.rs index 7664a77e1110..e138c8aff0ca 100644 --- a/src/servers/src/query_handler.rs +++ b/src/servers/src/query_handler.rs @@ -35,6 +35,8 @@ use common_query::Output; use headers::HeaderValue; use opentelemetry_proto::tonic::collector::metrics::v1::ExportMetricsServiceRequest; use opentelemetry_proto::tonic::collector::trace::v1::ExportTraceServiceRequest; +use pipeline::transform::GreptimeTransformer; +use pipeline::Pipeline; use serde_json::Value; use session::context::QueryContextRef; @@ -123,4 +125,16 @@ pub trait OpenTelemetryProtocolHandler { #[async_trait] pub trait LogHandler { async fn insert_log(&self, log: RowInsertRequests, ctx: QueryContextRef) -> Result; + async fn get_pipeline( + &self, + query_ctx: QueryContextRef, + name: &str, + ) -> Result>; + async fn insert_pipeline( + &self, + query_ctx: QueryContextRef, + name: &str, + content_type: &str, + pipeline: &str, + ) -> Result<()>; } From 1d52cad254e46596afbc5889a5ba17606e042896 Mon Sep 17 00:00:00 2001 From: paomian Date: Fri, 31 May 2024 12:10:03 +0800 Subject: [PATCH 09/51] chore: add pipeline API --- src/servers/src/http.rs | 1 + src/servers/src/http/handler.rs | 36 +++++++++++++++++++++++++++++---- 2 files changed, 33 insertions(+), 4 deletions(-) diff --git a/src/servers/src/http.rs b/src/servers/src/http.rs index f3f734936fa7..c54decff4160 100644 --- a/src/servers/src/http.rs +++ b/src/servers/src/http.rs @@ -712,6 +712,7 @@ impl HttpServer { fn route_log(log_handler: LogHandlerRef) -> Router { Router::new() .route("/logs", routing::post(handler::log_ingester)) + .route("/pipelines", routing::post(handler::add_pipeline)) .layer( ServiceBuilder::new() .layer(HandleErrorLayer::new(handle_error)) diff --git a/src/servers/src/http/handler.rs b/src/servers/src/http/handler.rs index ad0dc9767aa5..684cd8dbb7bf 100644 --- a/src/servers/src/http/handler.rs +++ b/src/servers/src/http/handler.rs @@ -18,7 +18,8 @@ use std::time::Instant; use aide::transform::TransformOperation; use api::v1::{RowInsertRequest, RowInsertRequests, Rows}; -use axum::extract::{Json, Query, State}; +use axum::extract::{Json, Query, State, TypedHeader}; +use axum::headers::ContentType; use axum::response::{IntoResponse, Response}; use axum::{Extension, Form}; use common_error::ext::ErrorExt; @@ -70,7 +71,7 @@ pub struct SqlQuery { #[derive(Debug, Default, Serialize, Deserialize, JsonSchema)] pub struct LogIngesterQueryParams { - pub table_name: String, + pub table_name: Option, pub db: Option, } @@ -79,7 +80,7 @@ fn validate_log_ingester_payload(payload: &Value) -> Result<(), String> { return Err("payload must be an object".to_string()); } - if payload["pipeline_model"].as_str().is_none() { + if payload["pipeline_id"].as_str().is_none() { return Err("processors field is required".to_string()); } @@ -108,9 +109,13 @@ async fn log_ingester_inner( .map_err(|e| e.to_string())?; let transformed_data: Rows = pipeline.exec(pipeline_data)?; + let table_name = query_params + .table_name + .ok_or("table_name is required".to_string())?; + let insert_request = RowInsertRequest { rows: Some(transformed_data), - table_name: query_params.table_name.clone(), + table_name: table_name.clone(), }; let insert_requests = RowInsertRequests { inserts: vec![insert_request], @@ -154,6 +159,29 @@ pub async fn log_ingester( } } +#[axum_macros::debug_handler] +pub async fn add_pipeline( + State(_state): State, + Query(_query_params): Query, + Extension(_query_ctx): Extension, + TypedHeader(_content_type): TypedHeader, + Json(paylod): Json, +) -> String { + let name = paylod["name"].as_str().unwrap(); + let pipeline = paylod["pipeline"].as_str().unwrap(); + let content_type = "yaml"; + let result = _state + .insert_pipeline(_query_ctx, name, content_type, pipeline) + .await; + match result { + Ok(_) => String::from("ok"), + Err(e) => { + common_telemetry::error!("failed to insert pipeline.{e:?}"); + e.to_string() + } + } +} + /// Handler to execute sql #[axum_macros::debug_handler] #[tracing::instrument(skip_all, fields(protocol = "http", request_type = "sql"))] From 8c69abb9c572253964274e4fd16e2b3ba5dfe49f Mon Sep 17 00:00:00 2001 From: paomian Date: Fri, 31 May 2024 18:00:26 +0800 Subject: [PATCH 10/51] chore: improve error handling --- src/frontend/src/error.rs | 8 +++ src/frontend/src/instance/log_handler.rs | 11 +++- src/frontend/src/pipeline.rs | 23 ++++---- src/pipeline/src/error.rs | 8 +++ src/pipeline/src/table.rs | 70 ++++++++++++++++++++++-- src/servers/src/error.rs | 9 +++ src/servers/src/query_handler.rs | 1 + 7 files changed, 110 insertions(+), 20 deletions(-) diff --git a/src/frontend/src/error.rs b/src/frontend/src/error.rs index 40e4db6f1f54..a98bbe271115 100644 --- a/src/frontend/src/error.rs +++ b/src/frontend/src/error.rs @@ -305,6 +305,13 @@ pub enum Error { source: BoxedError, }, + #[snafu(display("Failed to parse pipeline"))] + ParsePipeline { + #[snafu(implicit)] + location: Location, + source: BoxedError, + }, + #[snafu(display("Failed to get pipeline to pipeline table, name: {}", name))] GetPipeline { name: String, @@ -400,6 +407,7 @@ impl ErrorExt for Error { | Error::ColumnNoneDefaultValue { .. } | Error::IncompleteGrpcRequest { .. } | Error::InsertPipeline { .. } + | Error::ParsePipeline { .. } | Error::GetPipeline { .. } | Error::InvalidTlsConfig { .. } => StatusCode::InvalidArguments, diff --git a/src/frontend/src/instance/log_handler.rs b/src/frontend/src/instance/log_handler.rs index c5edbce92de9..a6bf57a38c2e 100644 --- a/src/frontend/src/instance/log_handler.rs +++ b/src/frontend/src/instance/log_handler.rs @@ -51,8 +51,9 @@ impl LogHandler for Instance { .get_pipeline(query_ctx, name) .await .map_err(BoxedError::new) - .context(servers::error::InsertPipelineSnafu { name }) + .context(servers::error::GetPipelineSnafu { name }) } + async fn insert_pipeline( &self, query_ctx: QueryContextRef, @@ -67,6 +68,14 @@ impl LogHandler for Instance { .context(servers::error::InsertPipelineSnafu { name })?; Ok(()) } + + async fn delete_pipeline( + &self, + _query_ctx: QueryContextRef, + _name: &str, + ) -> servers::error::Result<()> { + todo!("delete_pipeline") + } } impl Instance { diff --git a/src/frontend/src/pipeline.rs b/src/frontend/src/pipeline.rs index ae0b5b52fe13..95c2d967fd0b 100644 --- a/src/frontend/src/pipeline.rs +++ b/src/frontend/src/pipeline.rs @@ -188,13 +188,13 @@ impl PipelineOperator { content_type: &str, pipeline: &str, ) -> Result<()> { - let _compiled_pipeline = PipelineTable::::compile_pipeline(name, pipeline) + let _compiled_pipeline = PipelineTable::::compile_pipeline(pipeline) .map_err(BoxedError::new) .context(InsertPipelineSnafu { name })?; self.get_pipeline_table_from_cache(catalog) // FIXME (qtang): we should add error handling here .unwrap() - .insert_pipeline_to_pipeline_table(schema, name, content_type, pipeline) + .insert_and_compile(schema, name, content_type, pipeline) .await .map_err(|e| { if e.status_code().should_log_error() { @@ -228,19 +228,16 @@ impl PipelineOperator { query_ctx: QueryContextRef, name: &str, ) -> Result> { - let _s = self - .create_pipeline_table_if_not_exists(query_ctx.current_catalog()) - .await; - let table = self - .get_pipeline_table_from_cache(query_ctx.current_catalog()) - .unwrap() - .find_pipeline_by_name(query_ctx.current_schema(), name) + self.create_pipeline_table_if_not_exists(query_ctx.current_catalog()) + .await?; + self.get_pipeline_table_from_cache(query_ctx.current_catalog()) + .context(TableNotFoundSnafu { + table_name: PIPELINE_TABLE_NAME, + })? + .get_pipeline(query_ctx.current_schema(), name) .await .map_err(BoxedError::new) - .context(GetPipelineSnafu { name })?; - PipelineTable::::compile_pipeline(name, &table) - .map_err(BoxedError::new) - .context(InsertPipelineSnafu { name }) + .context(GetPipelineSnafu { name }) } pub async fn insert_pipeline( diff --git a/src/pipeline/src/error.rs b/src/pipeline/src/error.rs index 652ae9a0484a..a6182282dad8 100644 --- a/src/pipeline/src/error.rs +++ b/src/pipeline/src/error.rs @@ -44,6 +44,13 @@ pub enum Error { source: BoxedError, }, + #[snafu(display("Failed to parse pipeline"))] + ParsePipeline { + #[snafu(implicit)] + location: Location, + source: BoxedError, + }, + #[snafu(display("Pipeline not found, name: {}", name))] PipelineNotFound { #[snafu(implicit)] @@ -92,6 +99,7 @@ impl ErrorExt for Error { InsertPipeline { source, .. } => source.status_code(), CollectRecords { source, .. } => source.status_code(), PipelineNotFound { .. } => StatusCode::InvalidArguments, + ParsePipeline { .. } => StatusCode::InvalidArguments, BuildDfLogicalPlan { .. } => StatusCode::Internal, ExecuteInternalStatement { source, .. } => source.status_code(), } diff --git a/src/pipeline/src/table.rs b/src/pipeline/src/table.rs index 6e20c475177f..b6d0a31d5a39 100644 --- a/src/pipeline/src/table.rs +++ b/src/pipeline/src/table.rs @@ -1,4 +1,5 @@ -use std::sync::Arc; +use std::collections::HashMap; +use std::sync::{Arc, RwLock}; use api::v1::greptime_request::Request; use api::v1::value::ValueData; @@ -31,7 +32,7 @@ use table::TableRef; use crate::error::{ BuildDfLogicalPlanSnafu, CastTypeSnafu, CollectRecordsSnafu, ExecuteInternalStatementSnafu, - InsertPipelineSnafu, PipelineNotFoundSnafu, Result, + InsertPipelineSnafu, ParsePipelineSnafu, PipelineNotFoundSnafu, Result, }; pub type PipelineTableRef = Arc>; @@ -41,6 +42,7 @@ pub struct PipelineTable { grpc_handler: GrpcQueryHandlerRef, table: TableRef, query_engine: QueryEngineRef, + pipelines: RwLock>>, } impl PipelineTable { @@ -53,6 +55,7 @@ impl PipelineTable { grpc_handler, table, query_engine, + pipelines: RwLock::new(HashMap::default()), } } @@ -179,14 +182,30 @@ impl PipelineTable { .into() } - pub fn compile_pipeline(name: &str, pipeline: &str) -> Result> { + pub fn compile_pipeline(pipeline: &str) -> Result> { let yaml_content = Content::Yaml(pipeline.into()); parse::(&yaml_content) .map_err(|e| BoxedError::new(PlainError::new(e, StatusCode::InvalidArguments))) - .context(InsertPipelineSnafu { name }) + .context(ParsePipelineSnafu) } - pub async fn insert_pipeline_to_pipeline_table( + fn generate_pipeline_cache_key(schema: &str, name: &str) -> String { + format!("{}.{}", schema, name) + } + + fn get_compiled_pipeline_from_cache( + &self, + schema: &str, + name: &str, + ) -> Option> { + self.pipelines + .read() + .unwrap() + .get(&Self::generate_pipeline_cache_key(schema, name)) + .cloned() + } + + async fn insert_pipeline_to_pipeline_table( &self, schema: &str, name: &str, @@ -235,7 +254,46 @@ impl PipelineTable { Ok(()) } - pub async fn find_pipeline_by_name(&self, schema: &str, name: &str) -> Result { + + pub async fn get_pipeline( + &self, + schema: &str, + name: &str, + ) -> Result> { + if let Some(pipeline) = self.get_compiled_pipeline_from_cache(schema, name) { + return Ok(pipeline); + } + + let pipeline = self.find_pipeline_by_name(schema, name).await?; + let compiled_pipeline = Self::compile_pipeline(&pipeline)?; + self.pipelines.write().unwrap().insert( + Self::generate_pipeline_cache_key(schema, name), + compiled_pipeline.clone(), + ); + Ok(compiled_pipeline) + } + + pub async fn insert_and_compile( + &self, + schema: &str, + name: &str, + content_type: &str, + pipeline: &str, + ) -> Result<()> { + let compiled_pipeline = Self::compile_pipeline(pipeline)?; + + self.insert_pipeline_to_pipeline_table(schema, name, content_type, pipeline) + .await?; + + self.pipelines.write().unwrap().insert( + Self::generate_pipeline_cache_key(schema, name), + compiled_pipeline, + ); + + Ok(()) + } + + async fn find_pipeline_by_name(&self, schema: &str, name: &str) -> Result { let table_info = self.table.table_info(); let table_name = TableReference::full( diff --git a/src/servers/src/error.rs b/src/servers/src/error.rs index 1b26ac541370..c40ccca7528b 100644 --- a/src/servers/src/error.rs +++ b/src/servers/src/error.rs @@ -156,6 +156,14 @@ pub enum Error { source: BoxedError, }, + #[snafu(display("Failed to parse pipeline with name: {}", name))] + GetPipeline { + name: String, + #[snafu(implicit)] + location: Location, + source: BoxedError, + }, + #[snafu(display("Failed to execute script by name: {}", name))] ExecuteScript { name: String, @@ -616,6 +624,7 @@ impl ErrorExt for Error { InsertScript { source, .. } | InsertPipeline { source, .. } + | GetPipeline { source, .. } | ExecuteScript { source, .. } | ExecuteQuery { source, .. } | ExecutePlan { source, .. } diff --git a/src/servers/src/query_handler.rs b/src/servers/src/query_handler.rs index e138c8aff0ca..d7facac82e82 100644 --- a/src/servers/src/query_handler.rs +++ b/src/servers/src/query_handler.rs @@ -137,4 +137,5 @@ pub trait LogHandler { content_type: &str, pipeline: &str, ) -> Result<()>; + async fn delete_pipeline(&self, query_ctx: QueryContextRef, name: &str) -> Result<()>; } From 73432dcaa921afa456fb5e9df91b7b3e48aa4fc6 Mon Sep 17 00:00:00 2001 From: shuiyisong Date: Mon, 3 Jun 2024 16:02:55 +0800 Subject: [PATCH 11/51] chore: merge main --- Cargo.lock | 17 +++++++++-------- Cargo.toml | 5 +++-- src/frontend/Cargo.toml | 3 +-- src/frontend/src/pipeline.rs | 13 +++++-------- src/pipeline/Cargo.toml | 17 ++++++++--------- src/servers/Cargo.toml | 3 +-- 6 files changed, 27 insertions(+), 31 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index a41d2994cb84..f9186ec6b204 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3932,7 +3932,7 @@ dependencies = [ "opentelemetry-proto 0.5.0", "operator", "partition", - "pipeline 0.8.0", + "pipeline 0.8.1", "prometheus", "prost 0.12.6", "query", @@ -7240,6 +7240,7 @@ checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" [[package]] name = "pipeline" version = "0.1.0" +source = "git+ssh://git@github.com/GreptimeTeam/pipeline.git?rev=3eb890c551b8d7f60c4491fcfec18966e2b210a4#3eb890c551b8d7f60c4491fcfec18966e2b210a4" dependencies = [ "chrono", "chrono-tz 0.9.0", @@ -7256,7 +7257,7 @@ dependencies = [ [[package]] name = "pipeline" -version = "0.8.0" +version = "0.8.1" dependencies = [ "api", "arc-swap", @@ -7277,11 +7278,11 @@ dependencies = [ "console", "criterion", "crossbeam-utils", - "datafusion 37.0.0", - "datafusion-common 37.0.0", - "datafusion-expr 37.0.0", - "datafusion-functions 37.0.0", - "datafusion-physical-expr 37.0.0", + "datafusion 38.0.0", + "datafusion-common 38.0.0", + "datafusion-expr 38.0.0", + "datafusion-functions 38.0.0", + "datafusion-physical-expr 38.0.0", "datatypes", "futures", "lazy_static", @@ -7297,7 +7298,7 @@ dependencies = [ "serde", "servers", "session", - "snafu 0.8.2", + "snafu 0.8.3", "sql", "table", "tokio", diff --git a/Cargo.toml b/Cargo.toml index 26508efd90b1..124715d7883e 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -46,6 +46,7 @@ members = [ "src/object-store", "src/operator", "src/partition", + "src/pipeline", "src/plugins", "src/promql", "src/puffin", @@ -59,7 +60,7 @@ members = [ "src/index", "tests-fuzz", "tests-integration", - "tests/runner", "src/pipeline", + "tests/runner", ] resolver = "2" @@ -222,12 +223,12 @@ mito2 = { path = "src/mito2" } object-store = { path = "src/object-store" } operator = { path = "src/operator" } partition = { path = "src/partition" } +pipeline = { path = "src/pipeline" } plugins = { path = "src/plugins" } promql = { path = "src/promql" } puffin = { path = "src/puffin" } query = { path = "src/query" } script = { path = "src/script" } -pipeline = { path = "src/pipeline" } servers = { path = "src/servers" } session = { path = "src/session" } sql = { path = "src/sql" } diff --git a/src/frontend/Cargo.toml b/src/frontend/Cargo.toml index 095674f56d32..a359a56702b5 100644 --- a/src/frontend/Cargo.toml +++ b/src/frontend/Cargo.toml @@ -44,7 +44,7 @@ meta-client.workspace = true opentelemetry-proto.workspace = true operator.workspace = true partition.workspace = true -pipeline = { workspace = true } +pipeline.workspace = true prometheus.workspace = true prost.workspace = true query.workspace = true @@ -61,7 +61,6 @@ table.workspace = true tokio.workspace = true toml.workspace = true tonic.workspace = true -#pipeline = { git = "ssh://git@github.com/GreptimeTeam/pipeline.git", rev = "6b88c3c627da9e20f8fd160071e9c69b3ebd4e6a" } [dev-dependencies] catalog.workspace = true diff --git a/src/frontend/src/pipeline.rs b/src/frontend/src/pipeline.rs index 95c2d967fd0b..75f4e2c1c703 100644 --- a/src/frontend/src/pipeline.rs +++ b/src/frontend/src/pipeline.rs @@ -21,8 +21,8 @@ use api::v1::{CreateTableExpr, DdlRequest}; use arc_swap::ArcSwap; use catalog::{CatalogManagerRef, RegisterSystemTableRequest}; use common_catalog::consts::{default_engine, DEFAULT_PRIVATE_SCHEMA_NAME}; +use common_catalog::format_full_table_name; use common_error::ext::{BoxedError, ErrorExt}; -use common_meta::table_name::TableName; use common_query::Output; use common_telemetry::{error, info}; use pipeline::table::{PipelineTable, PipelineTableRef}; @@ -135,7 +135,8 @@ impl PipelineOperator { return Ok(()); } - let table_name = TableName::new(&expr.catalog_name, &expr.schema_name, &expr.table_name); + let schema = expr.schema_name.clone(); + let table_name = expr.table_name.clone(); let _ = self .grpc_handler @@ -150,15 +151,11 @@ impl PipelineOperator { let table = self .catalog_manager - .table( - &table_name.catalog_name, - &table_name.schema_name, - &table_name.table_name, - ) + .table(catalog, &schema, &table_name) .await .context(CatalogSnafu)? .with_context(|| TableNotFoundSnafu { - table_name: table_name.to_string(), + table_name: format_full_table_name(catalog, &schema, &table_name), })?; if let Some(open_hook) = open_hook { diff --git a/src/pipeline/Cargo.toml b/src/pipeline/Cargo.toml index 0998d19aacc5..af282eb139d3 100644 --- a/src/pipeline/Cargo.toml +++ b/src/pipeline/Cargo.toml @@ -6,7 +6,6 @@ license.workspace = true [features] - [lints] workspace = true @@ -20,24 +19,25 @@ common-catalog.workspace = true common-error.workspace = true common-function.workspace = true common-macro.workspace = true +common-meta.workspace = true common-query.workspace = true common-recordbatch.workspace = true common-runtime.workspace = true common-telemetry.workspace = true common-time.workspace = true -common-meta.workspace = true console = "0.15" crossbeam-utils.workspace = true -datafusion = { workspace = true} -datafusion-common = { workspace = true} -datafusion-expr = { workspace = true} -datafusion-functions = { workspace = true} -datafusion-physical-expr = { workspace = true} +datafusion = { workspace = true } +datafusion-common = { workspace = true } +datafusion-expr = { workspace = true } +datafusion-functions = { workspace = true } +datafusion-physical-expr = { workspace = true } datatypes.workspace = true futures.workspace = true lazy_static.workspace = true once_cell.workspace = true -paste = { workspace = true} +paste = { workspace = true } +pipeline = { git = "ssh://git@github.com/GreptimeTeam/pipeline.git", rev = "3eb890c551b8d7f60c4491fcfec18966e2b210a4" } prometheus.workspace = true query.workspace = true servers.workspace = true @@ -46,7 +46,6 @@ snafu.workspace = true sql.workspace = true table.workspace = true tokio.workspace = true -pipeline = { path = "../../../pipeline" } [dev-dependencies] catalog = { workspace = true, features = ["testing"] } diff --git a/src/servers/Cargo.toml b/src/servers/Cargo.toml index e2f4aea2b9c6..a686ea29ebea 100644 --- a/src/servers/Cargo.toml +++ b/src/servers/Cargo.toml @@ -69,6 +69,7 @@ opentelemetry-proto.workspace = true parking_lot = "0.12" pgwire = "0.20" pin-project = "1.0" +pipeline = { git = "ssh://git@github.com/GreptimeTeam/pipeline.git", rev = "3eb890c551b8d7f60c4491fcfec18966e2b210a4" } postgres-types = { version = "0.2", features = ["with-chrono-0_4"] } pprof = { version = "0.13", features = [ "flamegraph", @@ -104,8 +105,6 @@ tower = { workspace = true, features = ["full"] } tower-http = { version = "0.4", features = ["full"] } urlencoding = "2.1" zstd.workspace = true -#pipeline = { git = "ssh://git@github.com/GreptimeTeam/pipeline.git", rev = "6b88c3c627da9e20f8fd160071e9c69b3ebd4e6a" } -pipeline = { path = "../../../pipeline" } [target.'cfg(not(windows))'.dependencies] tikv-jemalloc-ctl = { version = "0.5", features = ["use_std"] } From 1a03b7e37a38ca6db4a38b4680fca1935a1b2d5b Mon Sep 17 00:00:00 2001 From: paomian Date: Mon, 3 Jun 2024 20:59:46 +0800 Subject: [PATCH 12/51] chore: add multi content type support for log handler --- src/auth/src/permission.rs | 1 + src/frontend/src/instance/log_handler.rs | 3 +- src/servers/src/error.rs | 33 +++++++++++ src/servers/src/http/handler.rs | 75 +++++++++++------------- 4 files changed, 70 insertions(+), 42 deletions(-) diff --git a/src/auth/src/permission.rs b/src/auth/src/permission.rs index 9a8c2a243de7..272d59192fc5 100644 --- a/src/auth/src/permission.rs +++ b/src/auth/src/permission.rs @@ -30,6 +30,7 @@ pub enum PermissionReq<'a> { PromStoreWrite, PromStoreRead, Otlp, + LogWrite } #[derive(Debug)] diff --git a/src/frontend/src/instance/log_handler.rs b/src/frontend/src/instance/log_handler.rs index a6bf57a38c2e..35541314c2ef 100644 --- a/src/frontend/src/instance/log_handler.rs +++ b/src/frontend/src/instance/log_handler.rs @@ -35,8 +35,7 @@ impl LogHandler for Instance { self.plugins .get::() .as_ref() - // This is a bug, it should be PermissionReq::LogWrite - .check_permission(ctx.current_user(), PermissionReq::PromStoreWrite) + .check_permission(ctx.current_user(), PermissionReq::LogWrite) .context(AuthSnafu)?; self.handle_log_inserts(log, ctx).await diff --git a/src/servers/src/error.rs b/src/servers/src/error.rs index c40ccca7528b..335dc7373984 100644 --- a/src/servers/src/error.rs +++ b/src/servers/src/error.rs @@ -26,6 +26,7 @@ use common_error::status_code::StatusCode; use common_macro::stack_trace_debug; use common_telemetry::{debug, error}; use datatypes::prelude::ConcreteDataType; +use headers::ContentType; use query::parser::PromQuery; use serde_json::json; use snafu::{Location, Snafu}; @@ -549,6 +550,34 @@ pub enum Error { location: Location, }, + #[snafu(display("Failed to parse payload as json"))] + ParseJson { + #[snafu(source)] + error: serde_json::error::Error, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("Failed to conver to structed log"))] + ToStructedLog { + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("Unsupport content type: {:?}", content_type))] + UnsupportedContentType { + content_type: ContentType, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("Failed to insert log. msg: {}", msg))] + InsertLog { + msg: String, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("Failed to decode url"))] UrlDecode { #[snafu(source)] @@ -655,6 +684,10 @@ impl ErrorExt for Error { | MissingQueryContext { .. } | MysqlValueConversion { .. } | UnexpectedPhysicalTable { .. } + | ParseJson { .. } + | ToStructedLog { .. } + | UnsupportedContentType { .. } + | InsertLog { .. } | TimestampOverflow { .. } => StatusCode::InvalidArguments, RowWriter { source, .. } diff --git a/src/servers/src/http/handler.rs b/src/servers/src/http/handler.rs index 37fe4f3f75a4..09c19c42260a 100644 --- a/src/servers/src/http/handler.rs +++ b/src/servers/src/http/handler.rs @@ -34,8 +34,12 @@ use schemars::JsonSchema; use serde::{Deserialize, Serialize}; use serde_json::Value; use session::context::QueryContextRef; +use snafu::ResultExt; use super::header::collect_plan_metrics; +use crate::error::{ + Error, InsertLogSnafu, ParseJsonSnafu, UnsupportedContentTypeSnafu, +}; use crate::http::arrow_result::ArrowResponse; use crate::http::csv_result::CsvResponse; use crate::http::error_result::ErrorResponse; @@ -73,38 +77,30 @@ pub struct SqlQuery { pub struct LogIngesterQueryParams { pub table_name: Option, pub db: Option, + pub pipeline_name: Option, } -fn validate_log_ingester_payload(payload: &Value) -> Result<(), String> { - if !payload.is_object() { - return Err("payload must be an object".to_string()); - } - - if payload["pipeline_id"].as_str().is_none() { - return Err("processors field is required".to_string()); - } - - if payload["data"].as_array().is_none() && payload["data"].as_object().is_none() { - return Err("data field is required".to_string()); - } - - Ok(()) +fn parse_space_separated_log(payload: String) -> Result { + // ToStructedLogSnafu + let _log = payload.split_whitespace().collect::>(); + // TODO (qtang): implement this + todo!() } async fn log_ingester_inner( state: LogHandlerRef, query_params: LogIngesterQueryParams, query_ctx: QueryContextRef, - mut payload: Value, + payload: Value, ) -> Result { - let pipeline_id = payload["pipeline_id"].take(); - let pipeline_id = pipeline_id.as_str().unwrap(); - let data = payload["data"].take(); + let pipeline_id = query_params + .pipeline_name + .ok_or("pipeline_name is required".to_string())?; - let pipeline_data = PipelineValue::try_from(data)?; + let pipeline_data = PipelineValue::try_from(payload)?; let pipeline = state - .get_pipeline(query_ctx.clone(), pipeline_id) + .get_pipeline(query_ctx.clone(), &pipeline_id) .await .map_err(|e| e.to_string())?; let transformed_data: Rows = pipeline.exec(pipeline_data)?; @@ -136,27 +132,26 @@ async fn log_ingester_inner( /// handler to log ingester #[axum_macros::debug_handler] pub async fn log_ingester( - State(_state): State, - Query(_query_params): Query, - Extension(_query_ctx): Extension, - Json(mut _payload): Json, -) -> HttpResponse { - match validate_log_ingester_payload(&_payload) { - Ok(_) => (), - Err(e) => { - return HttpResponse::Error(ErrorResponse::from_error_message( - StatusCode::InvalidArguments, - e, - )) - } - }; - match log_ingester_inner(_state, _query_params, _query_ctx, _payload).await { - Ok(resp) => resp, - Err(e) => HttpResponse::Error(ErrorResponse::from_error_message( - StatusCode::InvalidArguments, - e, - )), + State(state): State, + Query(query_params): Query, + Extension(query_ctx): Extension, + TypedHeader(content_type): TypedHeader, + payload: String, +) -> Result { + let value; + // TODO (qtang): we should decide json or jsonl + if content_type == ContentType::json() { + value = serde_json::from_str(&payload).context(ParseJsonSnafu)?; + // TODO (qtang): we should decide which content type to support + // form_url_cncoded type is only placeholder + } else if content_type == ContentType::form_url_encoded() { + value = parse_space_separated_log(payload)?; + } else { + return UnsupportedContentTypeSnafu { content_type }.fail(); } + log_ingester_inner(state, query_params, query_ctx, value) + .await + .or_else(|e| InsertLogSnafu { msg: e }.fail()) } #[axum_macros::debug_handler] From 6a0998dfecbfbf7cbe345b728e42a8f17214c665 Mon Sep 17 00:00:00 2001 From: shuiyisong Date: Mon, 3 Jun 2024 17:18:56 +0800 Subject: [PATCH 13/51] refactor: remove servers dep on pipeline --- Cargo.lock | 1 - src/frontend/src/instance.rs | 2 - src/frontend/src/instance/builder.rs | 6 ++- src/frontend/src/pipeline.rs | 81 +++++++++------------------- src/operator/src/statement.rs | 2 + src/pipeline/Cargo.toml | 3 +- src/pipeline/src/table.rs | 29 ++++++---- 7 files changed, 51 insertions(+), 73 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 9d53fa4ea3a7..f4256a6c40cd 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7296,7 +7296,6 @@ dependencies = [ "rayon", "ron", "serde", - "servers", "session", "snafu 0.8.3", "sql", diff --git a/src/frontend/src/instance.rs b/src/frontend/src/instance.rs index ed6f52cb6387..47eabbe551fe 100644 --- a/src/frontend/src/instance.rs +++ b/src/frontend/src/instance.rs @@ -111,7 +111,6 @@ pub trait FrontendInstance: } pub type FrontendInstanceRef = Arc; -pub type StatementExecutorRef = Arc; #[derive(Clone)] pub struct Instance { @@ -267,7 +266,6 @@ impl FrontendInstance for Instance { } self.script_executor.start(self)?; - self.pipeline_operator.start(self); if let Some(t) = self.export_metrics_task.as_ref() { if t.send_by_handler { diff --git a/src/frontend/src/instance/builder.rs b/src/frontend/src/instance/builder.rs index 90634daf1523..2ad668ef1f19 100644 --- a/src/frontend/src/instance/builder.rs +++ b/src/frontend/src/instance/builder.rs @@ -27,7 +27,7 @@ use operator::delete::Deleter; use operator::insert::Inserter; use operator::procedure::ProcedureServiceOperator; use operator::request::Requester; -use operator::statement::StatementExecutor; +use operator::statement::{StatementExecutor, StatementExecutorRef}; use operator::table::TableMutationOperator; use partition::manager::PartitionRuleManager; use query::QueryEngineFactory; @@ -37,7 +37,7 @@ use snafu::OptionExt; use crate::error::{self, Result}; use crate::heartbeat::HeartbeatTask; use crate::instance::region_query::FrontendRegionQueryHandler; -use crate::instance::{Instance, StatementExecutorRef}; +use crate::instance::Instance; use crate::pipeline::PipelineOperator; use crate::script::ScriptExecutor; @@ -174,6 +174,8 @@ impl FrontendBuilder { )); let pipeline_operator = Arc::new(PipelineOperator::new( + inserter.clone(), + statement_executor.clone(), self.catalog_manager.clone(), query_engine.clone(), )); diff --git a/src/frontend/src/pipeline.rs b/src/frontend/src/pipeline.rs index 75f4e2c1c703..3dd107f7dcac 100644 --- a/src/frontend/src/pipeline.rs +++ b/src/frontend/src/pipeline.rs @@ -15,65 +15,38 @@ use std::collections::HashMap; use std::sync::{Arc, RwLock}; -use api::v1::ddl_request::Expr; -use api::v1::greptime_request::Request; -use api::v1::{CreateTableExpr, DdlRequest}; -use arc_swap::ArcSwap; +use api::v1::CreateTableExpr; use catalog::{CatalogManagerRef, RegisterSystemTableRequest}; use common_catalog::consts::{default_engine, DEFAULT_PRIVATE_SCHEMA_NAME}; use common_catalog::format_full_table_name; use common_error::ext::{BoxedError, ErrorExt}; -use common_query::Output; use common_telemetry::{error, info}; +use operator::insert::InserterRef; +use operator::statement::StatementExecutorRef; use pipeline::table::{PipelineTable, PipelineTableRef}; use pipeline::{GreptimeTransformer, Pipeline}; use query::QueryEngineRef; -use servers::query_handler::grpc::GrpcQueryHandler; use session::context::{QueryContext, QueryContextRef}; use snafu::{OptionExt, ResultExt}; use table::TableRef; use crate::error::{ - CatalogSnafu, Error, GetPipelineSnafu, InsertPipelineSnafu, Result, TableNotFoundSnafu, + CatalogSnafu, GetPipelineSnafu, InsertPipelineSnafu, Result, TableNotFoundSnafu, }; -use crate::instance::Instance; - -type FrontendGrpcQueryHandlerRef = Arc + Send + Sync>; pub const PIPELINE_TABLE_NAME: &str = "pipelines"; -struct DummyHandler; - -impl DummyHandler { - pub fn arc() -> Arc { - Arc::new(Self {}) - } -} - -#[async_trait::async_trait] -impl GrpcQueryHandler for DummyHandler { - type Error = Error; - - async fn do_query( - &self, - _query: Request, - _ctx: QueryContextRef, - ) -> std::result::Result { - unreachable!(); - } -} - pub struct PipelineOperator { - grpc_handler: ArcSwap, + inserter: InserterRef, + statement_executor: StatementExecutorRef, catalog_manager: CatalogManagerRef, query_engine: QueryEngineRef, - tables: RwLock>>, + tables: RwLock>, } impl PipelineOperator { pub fn create_table_request(&self, catalog: &str) -> RegisterSystemTableRequest { - let (time_index, primary_keys, column_defs) = - PipelineTable::::build_pipeline_schema(); + let (time_index, primary_keys, column_defs) = PipelineTable::build_pipeline_schema(); let create_table_expr = CreateTableExpr { catalog_name: catalog.to_string(), @@ -103,8 +76,9 @@ impl PipelineOperator { tables.insert( catalog.to_string(), Arc::new(PipelineTable::new( + self.inserter.clone(), + self.statement_executor.clone(), table, - self.grpc_handler.load().as_ref().clone(), self.query_engine.clone(), )), ); @@ -116,7 +90,7 @@ impl PipelineOperator { } let RegisterSystemTableRequest { - create_table_expr: expr, + create_table_expr: mut expr, open_hook, } = self.create_table_request(catalog); @@ -138,14 +112,11 @@ impl PipelineOperator { let schema = expr.schema_name.clone(); let table_name = expr.table_name.clone(); - let _ = self - .grpc_handler - .load() - .do_query( - Request::Ddl(DdlRequest { - expr: Some(Expr::CreateTable(expr)), - }), - QueryContext::arc(), + self.statement_executor + .create_table_inner( + &mut expr, + None, + Arc::new(QueryContext::with(catalog, &schema)), ) .await?; @@ -172,7 +143,7 @@ impl PipelineOperator { Ok(()) } - pub fn get_pipeline_table_from_cache(&self, catalog: &str) -> Option> { + pub fn get_pipeline_table_from_cache(&self, catalog: &str) -> Option { // FIXME (qtang): we should impl this self.tables.read().unwrap().get(catalog).cloned() } @@ -185,7 +156,7 @@ impl PipelineOperator { content_type: &str, pipeline: &str, ) -> Result<()> { - let _compiled_pipeline = PipelineTable::::compile_pipeline(pipeline) + let _compiled_pipeline = PipelineTable::compile_pipeline(pipeline) .map_err(BoxedError::new) .context(InsertPipelineSnafu { name })?; self.get_pipeline_table_from_cache(catalog) @@ -205,21 +176,21 @@ impl PipelineOperator { } impl PipelineOperator { - pub fn new(catalog_manager: CatalogManagerRef, query_engine: QueryEngineRef) -> Self { - let grpc_handler = ArcSwap::new(Arc::new(DummyHandler::arc() as _)); + pub fn new( + inserter: InserterRef, + statement_executor: StatementExecutorRef, + catalog_manager: CatalogManagerRef, + query_engine: QueryEngineRef, + ) -> Self { Self { - grpc_handler, + inserter, + statement_executor, catalog_manager, tables: RwLock::new(HashMap::new()), query_engine, } } - pub fn start(&self, instance: &Instance) { - self.grpc_handler - .store(Arc::new(Arc::new(instance.clone()) as _)); - } - pub async fn get_pipeline( &self, query_ctx: QueryContextRef, diff --git a/src/operator/src/statement.rs b/src/operator/src/statement.rs index 649af286a4bb..a7d170d75fd6 100644 --- a/src/operator/src/statement.rs +++ b/src/operator/src/statement.rs @@ -73,6 +73,8 @@ pub struct StatementExecutor { inserter: InserterRef, } +pub type StatementExecutorRef = Arc; + impl StatementExecutor { pub fn new( catalog_manager: CatalogManagerRef, diff --git a/src/pipeline/Cargo.toml b/src/pipeline/Cargo.toml index af282eb139d3..538428711a2c 100644 --- a/src/pipeline/Cargo.toml +++ b/src/pipeline/Cargo.toml @@ -36,11 +36,11 @@ datatypes.workspace = true futures.workspace = true lazy_static.workspace = true once_cell.workspace = true +operator.workspace = true paste = { workspace = true } pipeline = { git = "ssh://git@github.com/GreptimeTeam/pipeline.git", rev = "3eb890c551b8d7f60c4491fcfec18966e2b210a4" } prometheus.workspace = true query.workspace = true -servers.workspace = true session.workspace = true snafu.workspace = true sql.workspace = true @@ -52,7 +52,6 @@ catalog = { workspace = true, features = ["testing"] } common-test-util.workspace = true criterion = { version = "0.4", features = ["html_reports", "async_tokio"] } log-store.workspace = true -operator.workspace = true rayon = "1.0" ron = "0.7" serde = { version = "1.0", features = ["derive"] } diff --git a/src/pipeline/src/table.rs b/src/pipeline/src/table.rs index b6d0a31d5a39..681301c13650 100644 --- a/src/pipeline/src/table.rs +++ b/src/pipeline/src/table.rs @@ -1,13 +1,12 @@ use std::collections::HashMap; use std::sync::{Arc, RwLock}; -use api::v1::greptime_request::Request; use api::v1::value::ValueData; use api::v1::{ ColumnDataType, ColumnDef, ColumnSchema as PbColumnSchema, Row, RowInsertRequest, RowInsertRequests, Rows, SemanticType, }; -use common_error::ext::{BoxedError, ErrorExt, PlainError}; +use common_error::ext::{BoxedError, PlainError}; use common_error::status_code::StatusCode; use common_query::OutputData; use common_recordbatch::util as record_util; @@ -19,11 +18,12 @@ use datafusion_common::TableReference; use datafusion_expr::LogicalPlanBuilder; use datatypes::prelude::ScalarVector; use datatypes::vectors::{StringVector, Vector}; +use operator::insert::InserterRef; +use operator::statement::StatementExecutorRef; use pipeline::transform::GreptimeTransformer; use pipeline::{parse, Content, Pipeline}; use query::plan::LogicalPlan; use query::QueryEngineRef; -use servers::query_handler::grpc::GrpcQueryHandlerRef; use session::context::{QueryContextBuilder, QueryContextRef}; use snafu::{ensure, OptionExt, ResultExt}; use table::metadata::TableInfo; @@ -35,24 +35,27 @@ use crate::error::{ InsertPipelineSnafu, ParsePipelineSnafu, PipelineNotFoundSnafu, Result, }; -pub type PipelineTableRef = Arc>; +pub type PipelineTableRef = Arc; pub const PIPELINE_TABLE_NAME: &str = "pipelines"; -pub struct PipelineTable { - grpc_handler: GrpcQueryHandlerRef, +pub struct PipelineTable { + inserter: InserterRef, + statement_executor: StatementExecutorRef, table: TableRef, query_engine: QueryEngineRef, pipelines: RwLock>>, } -impl PipelineTable { +impl PipelineTable { pub fn new( + inserter: InserterRef, + statement_executor: StatementExecutorRef, table: TableRef, - grpc_handler: GrpcQueryHandlerRef, query_engine: QueryEngineRef, ) -> Self { Self { - grpc_handler, + inserter, + statement_executor, table, query_engine, pipelines: RwLock::new(HashMap::default()), @@ -238,8 +241,12 @@ impl PipelineTable { }; let output = self - .grpc_handler - .do_query(Request::RowInserts(requests), Self::query_ctx(&table_info)) + .inserter + .handle_row_inserts( + requests, + Self::query_ctx(&table_info), + &self.statement_executor, + ) .await .map_err(BoxedError::new) .context(InsertPipelineSnafu { name })?; From 443eaf9c831bf7c4a82c3f2c3e5258defb042f39 Mon Sep 17 00:00:00 2001 From: shuiyisong Date: Mon, 3 Jun 2024 17:54:09 +0800 Subject: [PATCH 14/51] refactor: move define_into_tonic_status to common-error --- Cargo.lock | 2 +- src/pipeline/src/lib.rs | 1 + src/servers/Cargo.toml | 2 +- src/servers/src/http/handler.rs | 2 +- src/servers/src/query_handler.rs | 3 +-- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index f4256a6c40cd..09ed63c2c989 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -9703,7 +9703,7 @@ dependencies = [ "permutation", "pgwire", "pin-project", - "pipeline 0.1.0", + "pipeline 0.8.1", "postgres-types", "pprof", "prometheus", diff --git a/src/pipeline/src/lib.rs b/src/pipeline/src/lib.rs index e102ce489a3e..f5acf8d86af1 100644 --- a/src/pipeline/src/lib.rs +++ b/src/pipeline/src/lib.rs @@ -2,4 +2,5 @@ pub mod error; pub mod table; pub use pipeline::transform::GreptimeTransformer; +pub use pipeline::value::Value; pub use pipeline::Pipeline; diff --git a/src/servers/Cargo.toml b/src/servers/Cargo.toml index a686ea29ebea..31321a84010c 100644 --- a/src/servers/Cargo.toml +++ b/src/servers/Cargo.toml @@ -69,7 +69,7 @@ opentelemetry-proto.workspace = true parking_lot = "0.12" pgwire = "0.20" pin-project = "1.0" -pipeline = { git = "ssh://git@github.com/GreptimeTeam/pipeline.git", rev = "3eb890c551b8d7f60c4491fcfec18966e2b210a4" } +pipeline.workspace = true postgres-types = { version = "0.2", features = ["with-chrono-0_4"] } pprof = { version = "0.13", features = [ "flamegraph", diff --git a/src/servers/src/http/handler.rs b/src/servers/src/http/handler.rs index 09c19c42260a..302c2186eb89 100644 --- a/src/servers/src/http/handler.rs +++ b/src/servers/src/http/handler.rs @@ -28,7 +28,7 @@ use common_plugins::GREPTIME_EXEC_WRITE_COST; use common_query::{Output, OutputData}; use common_recordbatch::util; use common_telemetry::tracing; -use pipeline::value::Value as PipelineValue; +use pipeline::Value as PipelineValue; use query::parser::{PromQuery, DEFAULT_LOOKBACK_STRING}; use schemars::JsonSchema; use serde::{Deserialize, Serialize}; diff --git a/src/servers/src/query_handler.rs b/src/servers/src/query_handler.rs index d7facac82e82..c2ef268d494f 100644 --- a/src/servers/src/query_handler.rs +++ b/src/servers/src/query_handler.rs @@ -35,8 +35,7 @@ use common_query::Output; use headers::HeaderValue; use opentelemetry_proto::tonic::collector::metrics::v1::ExportMetricsServiceRequest; use opentelemetry_proto::tonic::collector::trace::v1::ExportTraceServiceRequest; -use pipeline::transform::GreptimeTransformer; -use pipeline::Pipeline; +use pipeline::{GreptimeTransformer, Pipeline}; use serde_json::Value; use session::context::QueryContextRef; From c8ce4ee5bfdb7cce6b7bf3736360d5aad474595c Mon Sep 17 00:00:00 2001 From: shuiyisong Date: Tue, 4 Jun 2024 15:34:21 +0800 Subject: [PATCH 15/51] refactor: bring in pipeline 3eb890c551b8d7f60c4491fcfec18966e2b210a4 --- Cargo.lock | 38 +- src/auth/src/permission.rs | 2 +- src/frontend/src/lib.rs | 2 +- src/pipeline/Cargo.toml | 28 +- src/pipeline/src/etl/field.rs | 195 ++++++++ src/pipeline/src/etl/mod.rs | 195 ++++++++ src/pipeline/src/etl/processor/cmcd.rs | 361 ++++++++++++++ src/pipeline/src/etl/processor/csv.rs | 327 +++++++++++++ src/pipeline/src/etl/processor/date.rs | 345 +++++++++++++ src/pipeline/src/etl/processor/epoch.rs | 205 ++++++++ src/pipeline/src/etl/processor/letter.rs | 188 +++++++ src/pipeline/src/etl/processor/mod.rs | 198 ++++++++ src/pipeline/src/etl/processor/regex.rs | 315 ++++++++++++ src/pipeline/src/etl/processor/urlencoding.rs | 177 +++++++ src/pipeline/src/etl/transform/index.rs | 57 +++ src/pipeline/src/etl/transform/mod.rs | 205 ++++++++ .../transform/transformer/greptime/coerce.rs | 310 ++++++++++++ .../etl/transform/transformer/greptime/mod.rs | 172 +++++++ .../src/etl/transform/transformer/mod.rs | 16 + .../src/etl/transform/transformer/noop.rs | 36 ++ src/pipeline/src/etl/value/array.rs | 56 +++ src/pipeline/src/etl/value/map.rs | 58 +++ src/pipeline/src/etl/value/mod.rs | 303 ++++++++++++ src/pipeline/src/etl/value/time.rs | 187 +++++++ src/pipeline/src/lib.rs | 11 +- src/pipeline/src/{ => mng}/error.rs | 0 src/pipeline/src/mng/mod.rs | 2 + src/pipeline/src/{ => mng}/table.rs | 4 +- src/pipeline/tests/pipeline.rs | 461 ++++++++++++++++++ src/script/src/table.rs | 2 +- src/servers/src/http/handler.rs | 4 +- 31 files changed, 4407 insertions(+), 53 deletions(-) create mode 100644 src/pipeline/src/etl/field.rs create mode 100644 src/pipeline/src/etl/mod.rs create mode 100644 src/pipeline/src/etl/processor/cmcd.rs create mode 100644 src/pipeline/src/etl/processor/csv.rs create mode 100644 src/pipeline/src/etl/processor/date.rs create mode 100644 src/pipeline/src/etl/processor/epoch.rs create mode 100644 src/pipeline/src/etl/processor/letter.rs create mode 100644 src/pipeline/src/etl/processor/mod.rs create mode 100644 src/pipeline/src/etl/processor/regex.rs create mode 100644 src/pipeline/src/etl/processor/urlencoding.rs create mode 100644 src/pipeline/src/etl/transform/index.rs create mode 100644 src/pipeline/src/etl/transform/mod.rs create mode 100644 src/pipeline/src/etl/transform/transformer/greptime/coerce.rs create mode 100644 src/pipeline/src/etl/transform/transformer/greptime/mod.rs create mode 100644 src/pipeline/src/etl/transform/transformer/mod.rs create mode 100644 src/pipeline/src/etl/transform/transformer/noop.rs create mode 100644 src/pipeline/src/etl/value/array.rs create mode 100644 src/pipeline/src/etl/value/map.rs create mode 100644 src/pipeline/src/etl/value/mod.rs create mode 100644 src/pipeline/src/etl/value/time.rs rename src/pipeline/src/{ => mng}/error.rs (100%) create mode 100644 src/pipeline/src/mng/mod.rs rename src/pipeline/src/{ => mng}/table.rs (99%) create mode 100644 src/pipeline/tests/pipeline.rs diff --git a/Cargo.lock b/Cargo.lock index 09ed63c2c989..6961827e690f 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3933,7 +3933,7 @@ dependencies = [ "opentelemetry-proto 0.5.0", "operator", "partition", - "pipeline 0.8.1", + "pipeline", "prometheus", "prost 0.12.6", "query", @@ -7237,33 +7237,16 @@ version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" -[[package]] -name = "pipeline" -version = "0.1.0" -source = "git+ssh://git@github.com/GreptimeTeam/pipeline.git?rev=3eb890c551b8d7f60c4491fcfec18966e2b210a4#3eb890c551b8d7f60c4491fcfec18966e2b210a4" -dependencies = [ - "chrono", - "chrono-tz 0.9.0", - "csv", - "greptime-proto", - "itertools 0.12.1", - "lazy_static", - "log", - "regex", - "serde_json", - "urlencoding", - "yaml-rust", -] - [[package]] name = "pipeline" version = "0.8.1" dependencies = [ "api", - "arc-swap", "arrow", "async-trait", "catalog", + "chrono", + "chrono-tz 0.9.0", "common-catalog", "common-error", "common-function", @@ -7273,11 +7256,9 @@ dependencies = [ "common-recordbatch", "common-runtime", "common-telemetry", - "common-test-util", "common-time", - "console", - "criterion", "crossbeam-utils", + "csv", "datafusion 38.0.0", "datafusion-common 38.0.0", "datafusion-expr 38.0.0", @@ -7285,23 +7266,26 @@ dependencies = [ "datafusion-physical-expr 38.0.0", "datatypes", "futures", + "greptime-proto", + "itertools 0.10.5", "lazy_static", - "log-store", "once_cell", "operator", "paste", - "pipeline 0.1.0", "prometheus", "query", "rayon", + "regex", "ron", "serde", + "serde_json", "session", "snafu 0.8.3", "sql", "table", "tokio", - "tokio-test", + "urlencoding", + "yaml-rust", ] [[package]] @@ -9703,7 +9687,7 @@ dependencies = [ "permutation", "pgwire", "pin-project", - "pipeline 0.8.1", + "pipeline", "postgres-types", "pprof", "prometheus", diff --git a/src/auth/src/permission.rs b/src/auth/src/permission.rs index 272d59192fc5..57afda471c8b 100644 --- a/src/auth/src/permission.rs +++ b/src/auth/src/permission.rs @@ -30,7 +30,7 @@ pub enum PermissionReq<'a> { PromStoreWrite, PromStoreRead, Otlp, - LogWrite + LogWrite, } #[derive(Debug)] diff --git a/src/frontend/src/lib.rs b/src/frontend/src/lib.rs index 90a01c19ddfb..83fcbb3d09d2 100644 --- a/src/frontend/src/lib.rs +++ b/src/frontend/src/lib.rs @@ -19,7 +19,7 @@ pub mod frontend; pub mod heartbeat; pub mod instance; pub(crate) mod metrics; +mod pipeline; mod script; pub mod server; pub mod service_config; -mod pipeline; diff --git a/src/pipeline/Cargo.toml b/src/pipeline/Cargo.toml index 538428711a2c..f44c9825e63f 100644 --- a/src/pipeline/Cargo.toml +++ b/src/pipeline/Cargo.toml @@ -11,10 +11,11 @@ workspace = true [dependencies] api.workspace = true -arc-swap = "1.0" arrow.workspace = true async-trait.workspace = true catalog.workspace = true +chrono.workspace = true +chrono-tz = "0.9.0" common-catalog.workspace = true common-error.workspace = true common-function.workspace = true @@ -25,35 +26,36 @@ common-recordbatch.workspace = true common-runtime.workspace = true common-telemetry.workspace = true common-time.workspace = true -console = "0.15" crossbeam-utils.workspace = true -datafusion = { workspace = true } -datafusion-common = { workspace = true } -datafusion-expr = { workspace = true } -datafusion-functions = { workspace = true } -datafusion-physical-expr = { workspace = true } +csv = "1.3.0" +datafusion.workspace = true +datafusion-common.workspace = true +datafusion-expr.workspace = true +datafusion-functions.workspace = true +datafusion-physical-expr.workspace = true datatypes.workspace = true futures.workspace = true +greptime-proto.workspace = true +itertools.workspace = true lazy_static.workspace = true once_cell.workspace = true operator.workspace = true -paste = { workspace = true } -pipeline = { git = "ssh://git@github.com/GreptimeTeam/pipeline.git", rev = "3eb890c551b8d7f60c4491fcfec18966e2b210a4" } +paste.workspace = true prometheus.workspace = true query.workspace = true +regex.workspace = true +serde_json.workspace = true session.workspace = true snafu.workspace = true sql.workspace = true table.workspace = true tokio.workspace = true +urlencoding = "2.1" +yaml-rust = "0.4" [dev-dependencies] catalog = { workspace = true, features = ["testing"] } -common-test-util.workspace = true -criterion = { version = "0.4", features = ["html_reports", "async_tokio"] } -log-store.workspace = true rayon = "1.0" ron = "0.7" serde = { version = "1.0", features = ["derive"] } session = { workspace = true, features = ["testing"] } -tokio-test = "0.4" diff --git a/src/pipeline/src/etl/field.rs b/src/pipeline/src/etl/field.rs new file mode 100644 index 000000000000..9d76b540953f --- /dev/null +++ b/src/pipeline/src/etl/field.rs @@ -0,0 +1,195 @@ +// Copyright 2024 Greptime Team +// +// Licensed 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 itertools::Itertools; + +#[derive(Debug, Default, Clone)] +pub struct Fields(Vec); + +impl Fields { + pub(crate) fn new(fields: Vec) -> Result { + let ff = Fields(fields); + ff.check() + } + + pub(crate) fn one(field: Field) -> Self { + Fields(vec![field]) + } + + pub(crate) fn get_target_fields(&self) -> Vec<&str> { + self.0.iter().map(|f| f.get_target_field()).collect() + } + + fn check(self) -> Result { + if self.0.is_empty() { + return Err("fields must not be empty".to_string()); + } + + let mut set = std::collections::HashSet::new(); + for f in self.0.iter() { + if set.contains(&f.field) { + return Err(format!( + "field name must be unique, but got duplicated: {}", + f.field + )); + } + set.insert(&f.field); + } + + Ok(self) + } +} + +impl std::fmt::Display for Fields { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + let s = self.0.iter().map(|f| f.to_string()).join(";"); + write!(f, "{s}") + } +} + +impl std::ops::Deref for Fields { + type Target = Vec; + + fn deref(&self) -> &Self::Target { + &self.0 + } +} + +#[derive(Debug, Default, Clone)] +pub struct Field { + pub field: String, + + // rename + pub target_field: Option, + + // 1-to-many mapping + // processors: + // - csv + pub target_fields: Option>, +} + +impl Field { + pub(crate) fn new(field: impl Into) -> Self { + Field { + field: field.into(), + target_field: None, + target_fields: None, + } + } + + // column_name in transform + pub(crate) fn get_target_field(&self) -> &str { + self.target_field.as_deref().unwrap_or(&self.field) + } + + pub(crate) fn get_field(&self) -> &str { + &self.field + } +} + +impl std::str::FromStr for Field { + type Err = String; + + fn from_str(s: &str) -> Result { + let mut parts = s.split(','); + let field = parts.next().ok_or("field is missing")?.trim().to_string(); + + if field.is_empty() { + return Err("field is empty".to_string()); + } + + let target_field = match parts.next() { + Some(s) if !s.trim().is_empty() => Some(s.trim().to_string()), + _ => None, + }; + + let fields: Vec<_> = parts + .filter(|s| !s.trim().is_empty()) + .map(|s| s.trim().to_string()) + .collect(); + let target_fields = if fields.is_empty() { + None + } else { + Some(fields) + }; + + Ok(Field { + field, + target_field, + target_fields, + }) + } +} + +impl std::fmt::Display for Field { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + match (&self.target_field, &self.target_fields) { + (Some(target_field), None) => write!(f, "{}, {target_field}", self.field), + (None, Some(target_fields)) => { + write!(f, "{}, {}", self.field, target_fields.iter().join(",")) + } + _ => write!(f, "{}", self.field), + } + } +} + +#[cfg(test)] +mod tests { + use crate::etl::field::Field; + + #[test] + fn test_parse_field() { + let field: Result = " ".parse(); + assert!(field.is_err()); + + let field: Result = ",".parse(); + assert!(field.is_err()); + + let field: Result = ",field".parse(); + assert!(field.is_err()); + + let cases = [ + // ("field", "field", None, None), + ( + "field, target_field", + "field", + Some("target_field".into()), + None, + ), + ( + "field, target_field1, target_field2, target_field3", + "field", + Some("target_field1".into()), + Some(vec!["target_field2".into(), "target_field3".into()]), + ), + ( + "field,, target_field1, target_field2, target_field3", + "field", + None, + Some(vec![ + "target_field1".into(), + "target_field2".into(), + "target_field3".into(), + ]), + ), + ]; + + for (s, field, target_field, target_fields) in cases.into_iter() { + let f: Field = s.parse().unwrap(); + assert_eq!(f.get_field(), field, "{s}"); + assert_eq!(f.target_field, target_field, "{s}"); + assert_eq!(f.target_fields, target_fields, "{s}"); + } + } +} diff --git a/src/pipeline/src/etl/mod.rs b/src/pipeline/src/etl/mod.rs new file mode 100644 index 000000000000..74c6cd96c547 --- /dev/null +++ b/src/pipeline/src/etl/mod.rs @@ -0,0 +1,195 @@ +// Copyright 2024 Greptime Team +// +// Licensed 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. + +#![allow(dead_code)] + +pub mod field; +pub mod processor; +pub mod transform; +pub mod value; + +use itertools::Itertools; +use transform::{Transformer, Transforms}; +use yaml_rust::YamlLoader; + +const DESCRIPTION: &str = "description"; +const PROCESSORS: &str = "processors"; +const TRANSFORM: &str = "transform"; + +pub enum Content { + Json(String), + Yaml(String), +} + +pub fn parse(input: &Content) -> Result, String> +where + T: Transformer, +{ + match input { + Content::Yaml(str) => { + let docs = YamlLoader::load_from_str(str).map_err(|e| e.to_string())?; + + let doc = &docs[0]; + + let description = doc[DESCRIPTION].as_str().map(|s| s.to_string()); + + let processors = if let Some(v) = doc[PROCESSORS].as_vec() { + v.try_into()? + } else { + processor::Processors::default() + }; + + let transforms = if let Some(v) = doc[TRANSFORM].as_vec() { + v.try_into()? + } else { + Transforms::default() + }; + + Ok(Pipeline { + description, + processors, + transformer: T::new(transforms)?, + }) + } + Content::Json(_) => unimplemented!(), + } +} + +#[derive(Debug, Clone)] +pub struct Pipeline +where + T: Transformer, +{ + description: Option, + processors: processor::Processors, + transformer: T, + // pub on_failure: processor::Processors, +} + +impl std::fmt::Display for Pipeline +where + T: Transformer, +{ + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + if let Some(description) = &self.description { + writeln!(f, "description: {description}")?; + } + + let processors = self.processors.iter().map(|p| p.kind()).join(","); + writeln!(f, "processors: {processors}")?; + + writeln!(f, "transformer: {}", self.transformer) + } +} + +impl Pipeline +where + T: Transformer, +{ + pub fn exec(&self, val: value::Value) -> Result { + let mut val = val; + for processor in self.processors.iter() { + val = processor.exec(val)?; + } + + self.transformer.transform(val) + } +} + +#[cfg(test)] +mod tests { + + use greptime_proto::v1::{self, ColumnDataType, SemanticType}; + + use crate::etl::transform::GreptimeTransformer; + use crate::etl::{parse, Content, Pipeline}; + + #[test] + fn test_csv_pipeline() { + let input_value_str = r#" + { + "my_field": "1,2", + "foo": "bar" + } + "#; + let input_value: serde_json::Value = serde_json::from_str(input_value_str).unwrap(); + + let pipeline_yaml = r#" +--- +description: Pipeline for Apache Tomcat + +processors: + - csv: + field: my_field, field1, field2 + +transform: + - field: field1 + type: uint32 + - field: field2 + type: uint32 +"#; + + let pipeline: Pipeline = + parse(&Content::Yaml(pipeline_yaml.into())).unwrap(); + let output = pipeline.exec(input_value.try_into().unwrap()); + assert!(output.is_ok()); + } + + #[test] + fn test_date_pipeline() { + let input_value_str = r#" + { + "my_field": "1,2", + "foo": "bar", + "test_time": "2014-5-17T04:34:56+00:00" + } + "#; + let input_value: serde_json::Value = serde_json::from_str(input_value_str).unwrap(); + + let pipeline_yaml = r#" +--- +description: Pipeline for Apache Tomcat + +processors: + - date: + field: test_time + +transform: + - field: test_time + type: time + index: timestamp +"#; + + let pipeline: Pipeline = + parse(&Content::Yaml(pipeline_yaml.into())).unwrap(); + let output = pipeline.exec(input_value.try_into().unwrap()).unwrap(); + let schemas = output.schema; + + assert_eq!(schemas.len(), 1); + let schema = schemas[0].clone(); + assert_eq!("test_time", schema.column_name); + assert_eq!(ColumnDataType::TimestampNanosecond as i32, schema.datatype); + assert_eq!(SemanticType::Timestamp as i32, schema.semantic_type); + + let row = output.rows[0].clone(); + assert_eq!(1, row.values.len()); + let value_data = row.values[0].clone().value_data; + assert_eq!( + Some(v1::value::ValueData::TimestampNanosecondValue( + 1400301296000000000 + )), + value_data + ); + } +} diff --git a/src/pipeline/src/etl/processor/cmcd.rs b/src/pipeline/src/etl/processor/cmcd.rs new file mode 100644 index 000000000000..7001ddb5b493 --- /dev/null +++ b/src/pipeline/src/etl/processor/cmcd.rs @@ -0,0 +1,361 @@ +// Copyright 2024 Greptime Team +// +// Licensed 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 urlencoding::decode; + +use crate::etl::field::{Field, Fields}; +use crate::etl::processor::{ + yaml_bool, yaml_field, yaml_fields, FIELDS_NAME, FIELD_NAME, IGNORE_MISSING_NAME, +}; +use crate::etl::value::{Map, Value}; + +pub(crate) const PROCESSOR_CMCD: &str = "cmcd"; + +const CMCD_KEY_BR: &str = "br"; // Encoded bitrate, Integer kbps +const CMCD_KEY_BL: &str = "bl"; // Buffer length, Integer milliseconds +const CMCD_KEY_BS: &str = "bs"; // Buffer starvation, Boolean +const CMCD_KEY_CID: &str = "cid"; // Content ID, String +const CMCD_KEY_D: &str = "d"; // Object duration, Integer milliseconds +const CMCD_KEY_DL: &str = "dl"; // Deadline, Integer milliseconds +const CMCD_KEY_MTP: &str = "mtp"; // Measured throughput, Integer kbps +const CMCD_KEY_NOR: &str = "nor"; // Next object request, String +const CMCD_KEY_NRR: &str = "nrr"; // Next request range, String, "-" +const CMCD_KEY_OT: &str = "ot"; // Object type, Token - one of [m,a,v,av,i,c,tt,k,o] +const CMCD_KEY_PR: &str = "pr"; // Playback rate, Decimal +const CMCD_KEY_RTP: &str = "rtp"; // Requested maximum throughput, Integer kbps +const CMCD_KEY_SF: &str = "sf"; // Stall frequency, Token - one of [d,h,s,o] +const CMCD_KEY_SID: &str = "sid"; // Session ID, String +const CMCD_KEY_ST: &str = "st"; // Stream type, Token - one of [v,l] +const CMCD_KEY_SU: &str = "su"; // Startup, Boolean +const CMCD_KEY_TB: &str = "tb"; // Top bitrate, Integer kbps +const CMCD_KEY_V: &str = "v"; // Version + +/// Common Media Client Data Specification: +/// https://cdn.cta.tech/cta/media/media/resources/standards/pdfs/cta-5004-final.pdf +/// +/// +/// The data payload for Header and Query Argument transmission consists of a series of +/// key/value pairs constructed according to the following rules: +/// 1. All information in the payload MUST be represented as = pairs. +/// 2. The key and value MUST be separated by an equals sign Unicode 0x3D. If the +/// value type is BOOLEAN and the value is TRUE, then the equals sign and the value +/// MUST be omitted. +/// 3. Successive key/value pairs MUST be delimited by a comma Unicode 0x2C. +/// 4. The key names described in this specification are reserved. Custom key names +/// may be used, but they MUST carry a hyphenated prefix to ensure that there will +/// not be a namespace collision with future revisions to this specification. Clients +/// SHOULD use a reverse-DNS syntax when defining their own prefix. +/// 5. If headers are used for data transmission, then custom keys SHOULD be +/// allocated to one of the four defined header names based upon their expected +/// level of variability: +/// a. CMCD-Request: keys whose values vary with each request. +/// b. CMCD-Object: keys whose values vary with the object being requested. +/// c. CMCD-Status: keys whose values do not vary with every request or object. +/// d. CMCD-Session: keys whose values are expected to be invariant over the life of the session. +/// 6. All key names are case-sensitive. +/// 7. Any value of type String MUST be enclosed by opening and closing double +/// quotes Unicode 0x22. Double quotes and backslashes MUST be escaped using a +/// backslash "\" Unicode 0x5C character. Any value of type Token does not require +/// quoting. +/// 8. All keys are OPTIONAL. +/// 9. Key-value pairs SHOULD be sequenced in alphabetical order of the key name in +/// order to reduce the fingerprinting surface exposed by the player. +/// 10. If the data payload is transmitted as a query argument, then the entire payload +/// string MUST be URLEncoded per [5]. Data payloads transmitted via headers +/// MUST NOT be URLEncoded. +/// 11. The data payload syntax is intended to be compliant with Structured Field Values for HTTP [6]. +/// 12. Transport Layer Security SHOULD be used to protect all transmission of CMCD data. +#[derive(Debug, Default)] +pub struct CMCDProcessor { + fields: Fields, + + ignore_missing: bool, +} + +impl CMCDProcessor { + fn with_fields(&mut self, fields: Fields) { + self.fields = fields; + } + + fn with_ignore_missing(&mut self, ignore_missing: bool) { + self.ignore_missing = ignore_missing; + } + + fn parse(prefix: &str, s: &str) -> Result { + let mut map = Map::default(); + let parts = s.split(','); + for part in parts { + let mut kv = part.split('='); + let k = kv.next().ok_or(format!("{part} missing key in {s}"))?; + let v = kv.next(); + + let key = format!("{prefix}_{k}"); + match k { + CMCD_KEY_BS | CMCD_KEY_SU => { + map.insert(key, Value::Boolean(true)); + } + CMCD_KEY_BR | CMCD_KEY_BL | CMCD_KEY_D | CMCD_KEY_DL | CMCD_KEY_MTP + | CMCD_KEY_RTP | CMCD_KEY_TB => { + let v = v.ok_or(format!("{k} missing value in {s}"))?; + let val: i64 = v + .parse() + .map_err(|_| format!("failed to parse {v} as i64"))?; + map.insert(key, Value::Int64(val)); + } + CMCD_KEY_CID | CMCD_KEY_NRR | CMCD_KEY_OT | CMCD_KEY_SF | CMCD_KEY_SID + | CMCD_KEY_ST | CMCD_KEY_V => { + let v = v.ok_or(format!("{k} missing value in {s}"))?; + map.insert(key, Value::String(v.to_string())); + } + CMCD_KEY_NOR => { + let v = v.ok_or(format!("{k} missing value in {s}"))?; + let val = match decode(v) { + Ok(val) => val.to_string(), + Err(_) => v.to_string(), + }; + map.insert(key, Value::String(val)); + } + CMCD_KEY_PR => { + let v = v.ok_or(format!("{k} missing value in {s}"))?; + let val: f64 = v + .parse() + .map_err(|_| format!("failed to parse {v} as f64"))?; + map.insert(key, Value::Float64(val)); + } + _ => match v { + Some(v) => map.insert(key, Value::String(v.to_string())), + None => map.insert(k, Value::Boolean(true)), + }, + } + } + + Ok(map) + } + + fn process_field(&self, val: &str, field: &Field) -> Result { + let prefix = match field.target_field { + Some(ref target_field) => target_field, + None => field.get_field(), + }; + + Self::parse(prefix, val) + } +} + +impl TryFrom<&yaml_rust::yaml::Hash> for CMCDProcessor { + type Error = String; + + fn try_from(value: &yaml_rust::yaml::Hash) -> Result { + let mut processor = CMCDProcessor::default(); + + for (k, v) in value.iter() { + let key = k + .as_str() + .ok_or(format!("key must be a string, but got {k:?}"))?; + match key { + FIELD_NAME => { + processor.with_fields(Fields::one(yaml_field(v, FIELD_NAME)?)); + } + FIELDS_NAME => { + processor.with_fields(yaml_fields(v, FIELDS_NAME)?); + } + + IGNORE_MISSING_NAME => { + processor.with_ignore_missing(yaml_bool(v, IGNORE_MISSING_NAME)?); + } + + _ => {} + } + } + + Ok(processor) + } +} + +impl crate::etl::processor::Processor for CMCDProcessor { + fn kind(&self) -> &str { + PROCESSOR_CMCD + } + + fn ignore_missing(&self) -> bool { + self.ignore_missing + } + + fn fields(&self) -> &Fields { + &self.fields + } + + fn exec_field(&self, val: &Value, field: &Field) -> Result { + match val { + Value::String(val) => self.process_field(val, field), + _ => Err(format!( + "{} processor: expect string value, but got {val:?}", + self.kind() + )), + } + } +} + +#[cfg(test)] +mod tests { + use std::collections::HashMap; + + use urlencoding::decode; + + use super::CMCDProcessor; + use crate::etl::value::{Map, Value}; + + #[test] + fn test_cmcd() { + let ss = [ + ( + "sid%3D%226e2fb550-c457-11e9-bb97-0800200c9a66%22", + vec![( + "prefix_sid", + Value::String("\"6e2fb550-c457-11e9-bb97-0800200c9a66\"".into()), + )], + ), + ( + "br%3D3200%2Cbs%2Cd%3D4004%2Cmtp%3D25400%2Cot%3Dv%2Crtp%3D15000%2Csid%3D%226e2fb550-c457-11e9-bb97-0800200c9a66%22%2Ctb%3D6000", + vec![ + ("prefix_bs", Value::Boolean(true)), + ("prefix_ot", Value::String("v".into())), + ("prefix_rtp", Value::Int64(15000)), + ("prefix_br", Value::Int64(3200)), + ("prefix_tb", Value::Int64(6000)), + ("prefix_d", Value::Int64(4004)), + ( + "prefix_sid", + Value::String("\"6e2fb550-c457-11e9-bb97-0800200c9a66\"".into()), + ), + ("prefix_mtp", Value::Int64(25400)), + ], + ), + ( + "b%2Crtp%3D15000%2Csid%3D%226e2fb550-c457-11e9-bb97-0800200c9a66%22", + vec![ + ( + "prefix_sid", + Value::String("\"6e2fb550-c457-11e9-bb97-0800200c9a66\"".into()), + ), + ("prefix_rtp", Value::Int64(15000)), + ("b", Value::Boolean(true)), + ], + ), + ( + "bs%2Csu", + vec![ + ("prefix_su", Value::Boolean(true)), + ("prefix_bs", Value::Boolean(true)), + ], + ), + ( + "d%3D4004%2Ccom.example-myNumericKey%3D500%2Ccom.examplemyStringKey%3D%22myStringValue%22", + vec![ + ( + "prefix_com.example-myNumericKey", + Value::String("500".into()), + ), + ( + "prefix_com.examplemyStringKey", + Value::String("\"myStringValue\"".into()), + ), + ("prefix_d", Value::Int64(4004)), + ], + ), + ( + "nor%3D%22..%252F300kbps%252Fsegment35.m4v%22%2Csid%3D%226e2fb550-c457-11e9-bb97-0800200c9a66%22", + vec![ + ( + "prefix_sid", + Value::String("\"6e2fb550-c457-11e9-bb97-0800200c9a66\"".into()), + ), + ( + "prefix_nor", + Value::String("\"../300kbps/segment35.m4v\"".into()), + + ), + ], + ), + ( + "nrr%3D%2212323-48763%22%2Csid%3D%226e2fb550-c457-11e9-bb97-0800200c9a66%22", + vec![ + ("prefix_nrr", Value::String("\"12323-48763\"".into())), + ( + "prefix_sid", + Value::String("\"6e2fb550-c457-11e9-bb97-0800200c9a66\"".into()), + ), + ], + ), + ( + "nor%3D%22..%252F300kbps%252Ftrack.m4v%22%2Cnrr%3D%2212323-48763%22%2Csid%3D%226e2fb550-c457-11e9-bb97-0800200c9a66%22", + vec![ + ("prefix_nrr", Value::String("\"12323-48763\"".into())), + ( + "prefix_sid", + Value::String("\"6e2fb550-c457-11e9-bb97-0800200c9a66\"".into()), + ), + ( + "prefix_nor", + Value::String("\"../300kbps/track.m4v\"".into()), + ), + ], + ), + ( + "bl%3D21300%2Cbr%3D3200%2Cbs%2Ccid%3D%22faec5fc2-ac30-11eabb37-0242ac130002%22%2Cd%3D4004%2Cdl%3D18500%2Cmtp%3D48100%2Cnor%3D%22..%252F300kbps%252Ftrack.m4v%22%2Cnrr%3D%2212323-48763%22%2Cot%3Dv%2Cpr%3D1.08%2Crtp%3D12000%2Csf%3Dd%2Csid%3D%226e2fb550-c457-11e9-bb97-0800200c9a66%22%2Cst%3Dv%2Csu%2Ctb%3D6000", + vec![ + ("prefix_bl", Value::Int64(21300)), + ("prefix_bs", Value::Boolean(true)), + ("prefix_st", Value::String("v".into())), + ("prefix_ot", Value::String("v".into())), + ( + "prefix_sid", + Value::String("\"6e2fb550-c457-11e9-bb97-0800200c9a66\"".into()), + ), + ("prefix_tb", Value::Int64(6000)), + ("prefix_d", Value::Int64(4004)), + ( + "prefix_cid", + Value::String("\"faec5fc2-ac30-11eabb37-0242ac130002\"".into()), + ), + ("prefix_mtp", Value::Int64(48100)), + ("prefix_rtp", Value::Int64(12000)), + ( + "prefix_nor", + Value::String("\"../300kbps/track.m4v\"".into()), + ), + ("prefix_sf", Value::String("d".into())), + ("prefix_br", Value::Int64(3200)), + ("prefix_nrr", Value::String("\"12323-48763\"".into())), + ("prefix_pr", Value::Float64(1.08)), + ("prefix_su", Value::Boolean(true)), + ("prefix_dl", Value::Int64(18500)), + ], + ), + ]; + + for (s, vec) in ss.into_iter() { + let decoded = decode(s).unwrap().to_string(); + + let values = vec + .into_iter() + .map(|(k, v)| (k.to_string(), v)) + .collect::>(); + let expected = Map { values }; + + let actual = CMCDProcessor::parse("prefix", &decoded).unwrap(); + assert_eq!(actual, expected); + } + } +} diff --git a/src/pipeline/src/etl/processor/csv.rs b/src/pipeline/src/etl/processor/csv.rs new file mode 100644 index 000000000000..1cd110922892 --- /dev/null +++ b/src/pipeline/src/etl/processor/csv.rs @@ -0,0 +1,327 @@ +// Copyright 2024 Greptime Team +// +// Licensed 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. + +// Reference: https://www.elastic.co/guide/en/elasticsearch/reference/current/csv-processor.html + +use std::collections::HashMap; + +use csv::{ReaderBuilder, Trim}; +use itertools::EitherOrBoth::{Both, Left, Right}; +use itertools::Itertools; + +use crate::etl::field::{Field, Fields}; +use crate::etl::processor::{ + yaml_bool, yaml_field, yaml_fields, yaml_string, Processor, FIELDS_NAME, FIELD_NAME, + IGNORE_MISSING_NAME, +}; +use crate::etl::value::{Map, Value}; + +pub(crate) const PROCESSOR_CSV: &str = "csv"; + +const SEPARATOR_NAME: &str = "separator"; +const QUOTE_NAME: &str = "quote"; +const TRIM_NAME: &str = "trim"; +const EMPTY_VALUE_NAME: &str = "empty_value"; + +/// only support string value +#[derive(Debug)] +pub struct CsvProcessor { + reader: ReaderBuilder, + + fields: Fields, + + ignore_missing: bool, + + // Value used to fill empty fields, empty fields will be skipped if this is not provided. + empty_value: Option, + // description + // if + // ignore_failure + // on_failure + // tag +} + +impl CsvProcessor { + fn new() -> Self { + let mut reader = ReaderBuilder::new(); + reader.has_headers(false); + + Self { + reader, + fields: Fields::default(), + ignore_missing: false, + empty_value: None, + } + } + + fn with_fields(&mut self, fields: Fields) { + self.fields = fields; + } + + fn try_separator(&mut self, separator: String) -> Result<(), String> { + if separator.len() != 1 { + Err(format!( + "'{}' must be a single character, but got '{}'", + SEPARATOR_NAME, separator + )) + } else { + self.reader.delimiter(separator.as_bytes()[0]); + Ok(()) + } + } + + fn try_quote(&mut self, quote: String) -> Result<(), String> { + if quote.len() != 1 { + Err(format!( + "'{}' must be a single character, but got '{}'", + QUOTE_NAME, quote + )) + } else { + self.reader.quote(quote.as_bytes()[0]); + Ok(()) + } + } + + fn with_trim(&mut self, trim: bool) { + if trim { + self.reader.trim(Trim::All); + } else { + self.reader.trim(Trim::None); + } + } + + fn with_ignore_missing(&mut self, ignore_missing: bool) { + self.ignore_missing = ignore_missing; + } + + fn with_empty_value(&mut self, empty_value: String) { + self.empty_value = Some(empty_value); + } + + // process the csv format string to a map with target_fields as keys + fn process_field(&self, val: &str, field: &Field) -> Result { + let mut reader = self.reader.from_reader(val.as_bytes()); + + if let Some(result) = reader.records().next() { + let record: csv::StringRecord = result.map_err(|e| e.to_string())?; + + let values: HashMap = field + .target_fields + .as_ref() + .ok_or(format!( + "target fields must be set after '{}'", + field.get_field() + ))? + .iter() + .map(|f| f.to_string()) + .zip_longest(record.iter()) + .filter_map(|zipped| match zipped { + Both(target_field, val) => Some((target_field, Value::String(val.into()))), + // if target fields are more than extracted fields, fill the rest with empty value + Left(target_field) => { + let value = self + .empty_value + .as_ref() + .map(|s| Value::String(s.clone())) + .unwrap_or(Value::Null); + Some((target_field, value)) + } + // if extracted fields are more than target fields, ignore the rest + Right(_) => None, + }) + .collect(); + + Ok(Map { values }) + } else { + Err("expected at least one record from csv format, but got none".into()) + } + } +} + +impl TryFrom<&yaml_rust::yaml::Hash> for CsvProcessor { + type Error = String; + + fn try_from(hash: &yaml_rust::yaml::Hash) -> Result { + let mut processor = CsvProcessor::new(); + for (k, v) in hash { + let key = k + .as_str() + .ok_or(format!("key must be a string, but got {k:?}"))?; + match key { + FIELD_NAME => { + processor.with_fields(Fields::one(yaml_field(v, FIELD_NAME)?)); + } + FIELDS_NAME => { + processor.with_fields(yaml_fields(v, FIELDS_NAME)?); + } + SEPARATOR_NAME => { + processor.try_separator(yaml_string(v, SEPARATOR_NAME)?)?; + } + QUOTE_NAME => { + processor.try_quote(yaml_string(v, QUOTE_NAME)?)?; + } + TRIM_NAME => { + processor.with_trim(yaml_bool(v, TRIM_NAME)?); + } + IGNORE_MISSING_NAME => { + processor.with_ignore_missing(yaml_bool(v, IGNORE_MISSING_NAME)?); + } + EMPTY_VALUE_NAME => { + processor.with_empty_value(yaml_string(v, EMPTY_VALUE_NAME)?); + } + + _ => {} + } + } + + Ok(processor) + } +} + +impl Processor for CsvProcessor { + fn kind(&self) -> &str { + PROCESSOR_CSV + } + + fn ignore_missing(&self) -> bool { + self.ignore_missing + } + + fn fields(&self) -> &Fields { + &self.fields + } + + fn exec_field(&self, val: &Value, field: &Field) -> Result { + match val { + Value::String(val) => self.process_field(val, field), + _ => Err(format!( + "{} processor: expect string value, but got {val:?}", + self.kind() + )), + } + } +} + +// TODO(yuanbohan): more test cases +#[cfg(test)] +mod tests { + use std::collections::HashMap; + + use super::{CsvProcessor, Value}; + use crate::etl::field::Fields; + use crate::etl::processor::Processor; + use crate::etl::value::Map; + + #[test] + fn test_equal_length() { + let mut processor = CsvProcessor::new(); + let field = "data,, a, b".parse().unwrap(); + processor.with_fields(Fields::one(field)); + + let values: HashMap = [("data".into(), Value::String("1,2".into()))] + .into_iter() + .collect(); + + let result = processor.exec(Value::Map(Map { values })).unwrap(); + + let values = [ + ("data".into(), Value::String("1,2".into())), + ("a".into(), Value::String("1".into())), + ("b".into(), Value::String("2".into())), + ] + .into_iter() + .collect(); + let expected = Value::Map(Map { values }); + + assert_eq!(expected, result); + } + + // test target_fields length larger than the record length + #[test] + fn test_target_fields_has_more_length() { + let values = [("data".into(), Value::String("1,2".into()))] + .into_iter() + .collect(); + let input = Value::Map(Map { values }); + + // with no empty value + { + let mut processor = CsvProcessor::new(); + let field = "data,, a,b,c".parse().unwrap(); + processor.with_fields(Fields::one(field)); + + let result = processor.exec(input.clone()).unwrap(); + + let values = [ + ("data".into(), Value::String("1,2".into())), + ("a".into(), Value::String("1".into())), + ("b".into(), Value::String("2".into())), + ("c".into(), Value::Null), + ] + .into_iter() + .collect(); + let expected = Value::Map(Map { values }); + + assert_eq!(expected, result); + } + + // with empty value + { + let mut processor = CsvProcessor::new(); + let field = "data,, a,b,c".parse().unwrap(); + processor.with_fields(Fields::one(field)); + processor.with_empty_value("default".into()); + + let result = processor.exec(input).unwrap(); + + let values = [ + ("data".into(), Value::String("1,2".into())), + ("a".into(), Value::String("1".into())), + ("b".into(), Value::String("2".into())), + ("c".into(), Value::String("default".into())), + ] + .into_iter() + .collect(); + let expected = Value::Map(Map { values }); + + assert_eq!(expected, result); + } + } + + // test record has larger length + #[test] + fn test_target_fields_has_less_length() { + let values = [("data".into(), Value::String("1,2,3".into()))] + .into_iter() + .collect(); + let input = Value::Map(Map { values }); + + let mut processor = CsvProcessor::new(); + let field = "data,,a,b".parse().unwrap(); + processor.with_fields(Fields::one(field)); + + let result = processor.exec(input).unwrap(); + + let values = [ + ("data".into(), Value::String("1,2,3".into())), + ("a".into(), Value::String("1".into())), + ("b".into(), Value::String("2".into())), + ] + .into_iter() + .collect(); + let expected = Value::Map(Map { values }); + + assert_eq!(expected, result); + } +} diff --git a/src/pipeline/src/etl/processor/date.rs b/src/pipeline/src/etl/processor/date.rs new file mode 100644 index 000000000000..6715522793a7 --- /dev/null +++ b/src/pipeline/src/etl/processor/date.rs @@ -0,0 +1,345 @@ +// Copyright 2024 Greptime Team +// +// Licensed 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 chrono::{DateTime, NaiveDateTime}; +use chrono_tz::Tz; +use lazy_static::lazy_static; + +use crate::etl::field::{Field, Fields}; +use crate::etl::processor::{ + yaml_bool, yaml_field, yaml_fields, yaml_string, yaml_strings, Processor, FIELDS_NAME, + FIELD_NAME, IGNORE_MISSING_NAME, +}; +use crate::etl::value::{Map, Time, Value}; + +pub(crate) const PROCESSOR_DATE: &str = "date"; + +const FORMATS_NAME: &str = "formats"; // default RFC3339 +const TIMEZONE_NAME: &str = "timezone"; // default UTC +const LOCALE_NAME: &str = "locale"; +const OUTPUT_FORMAT_NAME: &str = "output_format"; // default with input format + +lazy_static! { + static ref DEFAULT_FORMATS: Vec = vec![ + // timezone with colon + "%Y-%m-%dT%H:%M:%S%:z", + "%Y-%m-%dT%H:%M:%S%.3f%:z", + "%Y-%m-%dT%H:%M:%S%.6f%:z", + "%Y-%m-%dT%H:%M:%S%.9f%:z", + // timezone without colon + "%Y-%m-%dT%H:%M:%S%z", + "%Y-%m-%dT%H:%M:%S%.3f%z", + "%Y-%m-%dT%H:%M:%S%.6f%z", + "%Y-%m-%dT%H:%M:%S%.9f%z", + // without timezone + "%Y-%m-%dT%H:%M:%SZ", + "%Y-%m-%dT%H:%M:%S", + "%Y-%m-%dT%H:%M:%S%.3f", + "%Y-%m-%dT%H:%M:%S%.6f", + "%Y-%m-%dT%H:%M:%S%.9f", + ] + .iter() + .map(|s| s.to_string()) + .collect(); +} + +#[derive(Debug, Default)] +struct Formats(Vec); + +impl Formats { + fn new(mut formats: Vec) -> Self { + formats.sort(); + formats.dedup(); + Formats(formats) + } +} + +impl std::ops::Deref for Formats { + type Target = Vec; + + fn deref(&self) -> &Self::Target { + &self.0 + } +} + +#[derive(Debug, Default)] +pub struct DateProcessor { + fields: Fields, + + formats: Formats, + timezone: Option, + locale: Option, // to support locale + output_format: Option, + + ignore_missing: bool, + // description + // if + // ignore_failure + // on_failure + // tag +} + +impl DateProcessor { + fn with_fields(&mut self, fields: Fields) { + self.fields = fields + } + + fn with_formats(&mut self, v: Option>) { + let v = match v { + Some(v) if !v.is_empty() => v, + _ => DEFAULT_FORMATS.clone(), + }; + + let formats = Formats::new(v); + self.formats = formats; + } + + fn with_timezone(&mut self, timezone: String) { + if !timezone.is_empty() { + self.timezone = Some(timezone); + } + } + + fn with_locale(&mut self, locale: String) { + if !locale.is_empty() { + self.locale = Some(locale); + } + } + + fn with_output_format(&mut self, output_format: String) { + if !output_format.is_empty() { + self.output_format = Some(output_format); + } + } + + fn with_ignore_missing(&mut self, ignore_missing: bool) { + self.ignore_missing = ignore_missing; + } + + fn parse(&self, val: &str) -> Result { + let mut tz = Tz::UTC; + if let Some(timezone) = &self.timezone { + tz = timezone.parse::().map_err(|e| e.to_string())?; + } + + for fmt in self.formats.iter() { + if let Ok(ns) = try_parse(val, fmt, tz) { + let mut t = Time::new(val, ns); + t.with_format(fmt); + t.with_timezone(self.timezone.clone()); + return Ok(t); + } + } + + Err(format!("{} processor: failed to parse {val}", self.kind(),)) + } + + fn process_field(&self, val: &str, field: &Field) -> Result { + let key = match field.target_field { + Some(ref target_field) => target_field, + None => field.get_field(), + }; + + Ok(Map::one(key, Value::Time(self.parse(val)?))) + } +} + +impl TryFrom<&yaml_rust::yaml::Hash> for DateProcessor { + type Error = String; + + fn try_from(hash: &yaml_rust::yaml::Hash) -> Result { + let mut processor = DateProcessor::default(); + + let mut formats_opt = None; + + for (k, v) in hash { + let key = k + .as_str() + .ok_or(format!("key must be a string, but got {k:?}"))?; + + match key { + FIELD_NAME => { + processor.with_fields(Fields::one(yaml_field(v, FIELD_NAME)?)); + } + FIELDS_NAME => { + processor.with_fields(yaml_fields(v, FIELDS_NAME)?); + } + + FORMATS_NAME => { + let formats = yaml_strings(v, FORMATS_NAME)?; + formats_opt = Some(formats); + } + TIMEZONE_NAME => { + processor.with_timezone(yaml_string(v, TIMEZONE_NAME)?); + } + LOCALE_NAME => { + processor.with_locale(yaml_string(v, LOCALE_NAME)?); + } + OUTPUT_FORMAT_NAME => { + processor.with_output_format(yaml_string(v, OUTPUT_FORMAT_NAME)?); + } + + IGNORE_MISSING_NAME => { + processor.with_ignore_missing(yaml_bool(v, IGNORE_MISSING_NAME)?); + } + + _ => {} + } + } + + processor.with_formats(formats_opt); + + Ok(processor) + } +} + +impl Processor for DateProcessor { + fn kind(&self) -> &str { + PROCESSOR_DATE + } + + fn ignore_missing(&self) -> bool { + self.ignore_missing + } + + fn fields(&self) -> &Fields { + &self.fields + } + + fn exec_field(&self, val: &Value, field: &Field) -> Result { + match val { + Value::String(s) => self.process_field(s, field), + _ => Err(format!( + "{} processor: expect string value, but got {val:?}", + self.kind() + )), + } + } +} + +/// try to parse val with timezone first, if failed, parse without timezone +fn try_parse(val: &str, fmt: &str, tz: Tz) -> Result { + if let Ok(dt) = DateTime::parse_from_str(val, fmt) { + Ok(dt.timestamp_nanos_opt().ok_or("failed to get timestamp")?) + } else { + let dt = NaiveDateTime::parse_from_str(val, fmt) + .map_err(|e| e.to_string())? + .and_local_timezone(tz) + .single() + .ok_or("failed to get local timezone")?; + Ok(dt.timestamp_nanos_opt().ok_or("failed to get timestamp")?) + } +} + +#[cfg(test)] +mod tests { + use chrono_tz::Asia::Tokyo; + + use crate::etl::processor::date::{try_parse, DateProcessor}; + + #[test] + fn test_try_parse() { + let time_with_tz = "2014-5-17T04:34:56+00:00"; + let fmt_with_tz = "%Y-%m-%dT%H:%M:%S%:z"; + + let time_without_tz = "2014-5-17T13:34:56"; + let fmt_without_tz = "%Y-%m-%dT%H:%M:%S"; + + let tz = Tokyo; + + let parsed_with_tz = try_parse(time_with_tz, fmt_with_tz, tz); + assert!(parsed_with_tz.is_ok()); + + let parsed_without_tz = try_parse(time_without_tz, fmt_without_tz, tz); + assert!(parsed_without_tz.is_ok()); + + assert_eq!(parsed_with_tz.unwrap(), parsed_without_tz.unwrap()); + } + + #[test] + fn test_parse() { + let mut processor = DateProcessor::default(); + processor.with_formats(None); + + let values: Vec<&str> = vec![ + "2014-5-17T12:34:56", + "2014-5-17T12:34:56Z", + "2014-5-17T12:34:56+09:30", + "2014-5-17T12:34:56.000+09:30", + "2014-5-17T12:34:56-0930", + "2014-5-17T12:34:56.000-0930", + ] + .into_iter() + .collect(); + + for value in values { + let parsed = processor.parse(value); + assert!(parsed.is_ok()); + } + } + + #[test] + fn test_parse_with_formats() { + let mut processor = DateProcessor::default(); + let formats = vec![ + "%Y-%m-%dT%H:%M:%S%:z", + "%Y-%m-%dT%H:%M:%S%.3f%:z", + "%Y-%m-%dT%H:%M:%S", + "%Y-%m-%dT%H:%M:%SZ", + ] + .into_iter() + .map(|s| s.to_string()) + .collect(); + processor.with_formats(Some(formats)); + + let values: Vec<&str> = vec![ + "2014-5-17T12:34:56", + "2014-5-17T12:34:56Z", + "2014-5-17T12:34:56+09:30", + "2014-5-17T12:34:56.000+09:30", + "2014-5-17T12:34:56-0930", + "2014-5-17T12:34:56.000-0930", + ] + .into_iter() + .collect(); + + for value in values { + let parsed = processor.parse(value); + assert!(parsed.is_ok()); + } + } + + #[test] + fn test_parse_with_timezone() { + let mut processor = DateProcessor::default(); + processor.with_formats(None); + processor.with_timezone("Asia/Tokyo".to_string()); + + let values: Vec<&str> = vec![ + "2014-5-17T12:34:56", + "2014-5-17T12:34:56Z", + "2014-5-17T12:34:56+09:30", + "2014-5-17T12:34:56.000+09:30", + "2014-5-17T12:34:56-0930", + "2014-5-17T12:34:56.000-0930", + ] + .into_iter() + .collect(); + + for value in values { + let parsed = processor.parse(value); + assert!(parsed.is_ok()); + } + } +} diff --git a/src/pipeline/src/etl/processor/epoch.rs b/src/pipeline/src/etl/processor/epoch.rs new file mode 100644 index 000000000000..feee2fa8d717 --- /dev/null +++ b/src/pipeline/src/etl/processor/epoch.rs @@ -0,0 +1,205 @@ +// Copyright 2024 Greptime Team +// +// Licensed 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 crate::etl::field::{Field, Fields}; +use crate::etl::processor::{ + yaml_bool, yaml_field, yaml_fields, yaml_string, Processor, FIELDS_NAME, FIELD_NAME, + IGNORE_MISSING_NAME, +}; +use crate::etl::value::time::{ + MICROSECOND_RESOLUTION, MICRO_RESOLUTION, MILLISECOND_RESOLUTION, MILLI_RESOLUTION, + MS_RESOLUTION, NANOSECOND_RESOLUTION, NANO_RESOLUTION, NS_RESOLUTION, SECOND_RESOLUTION, + SEC_RESOLUTION, S_RESOLUTION, US_RESOLUTION, +}; +use crate::etl::value::{Epoch, Map, Value}; + +pub(crate) const PROCESSOR_EPOCH: &str = "epoch"; +const RESOLUTION_NAME: &str = "resolution"; + +#[derive(Debug, Default)] +enum Resolution { + Second, + #[default] + Milli, + Micro, + Nano, +} + +impl TryFrom<&str> for Resolution { + type Error = String; + + fn try_from(s: &str) -> Result { + match s { + SECOND_RESOLUTION | SEC_RESOLUTION | S_RESOLUTION => Ok(Resolution::Second), + MILLISECOND_RESOLUTION | MILLI_RESOLUTION | MS_RESOLUTION => Ok(Resolution::Milli), + MICROSECOND_RESOLUTION | MICRO_RESOLUTION | US_RESOLUTION => Ok(Resolution::Micro), + NANOSECOND_RESOLUTION | NANO_RESOLUTION | NS_RESOLUTION => Ok(Resolution::Nano), + _ => Err(format!("invalid resolution: {s}")), + } + } +} + +/// support string, integer, float, time, epoch +#[derive(Debug, Default)] +pub struct EpochProcessor { + fields: Fields, + resolution: Resolution, + ignore_missing: bool, + // description + // if + // ignore_failure + // on_failure + // tag +} + +impl EpochProcessor { + fn with_fields(&mut self, fields: Fields) { + self.fields = fields + } + + fn with_resolution(&mut self, resolution: Resolution) { + self.resolution = resolution; + } + + fn with_ignore_missing(&mut self, ignore_missing: bool) { + self.ignore_missing = ignore_missing; + } + + fn parse(&self, val: &Value) -> Result { + let t: i64 = match val { + Value::String(s) => s.parse::().map_err(|e| e.to_string())?, + Value::Int16(i) => *i as i64, + Value::Int32(i) => *i as i64, + Value::Int64(i) => *i, + Value::Uint8(i) => *i as i64, + Value::Uint16(i) => *i as i64, + Value::Uint32(i) => *i as i64, + Value::Uint64(i) => *i as i64, + Value::Float32(f) => *f as i64, + Value::Float64(f) => *f as i64, + + Value::Time(t) => match self.resolution { + Resolution::Second => t.timestamp(), + Resolution::Milli => t.timestamp_millis(), + Resolution::Micro => t.timestamp_micros(), + Resolution::Nano => t.timestamp_nanos(), + }, + + Value::Epoch(e) => match self.resolution { + Resolution::Second => e.timestamp(), + Resolution::Milli => e.timestamp_millis(), + Resolution::Micro => e.timestamp_micros(), + Resolution::Nano => e.timestamp_nanos(), + }, + + _ => { + return Err(format!( + "{PROCESSOR_EPOCH} processor: unsupported value {val}" + )) + } + }; + + match self.resolution { + Resolution::Second => Ok(Epoch::Second(t)), + Resolution::Milli => Ok(Epoch::Millisecond(t)), + Resolution::Micro => Ok(Epoch::Microsecond(t)), + Resolution::Nano => Ok(Epoch::Nanosecond(t)), + } + } + + fn process_field(&self, val: &Value, field: &Field) -> Result { + let key = match field.target_field { + Some(ref target_field) => target_field, + None => field.get_field(), + }; + + Ok(Map::one(key, Value::Epoch(self.parse(val)?))) + } +} + +impl TryFrom<&yaml_rust::yaml::Hash> for EpochProcessor { + type Error = String; + + fn try_from(hash: &yaml_rust::yaml::Hash) -> Result { + let mut processor = EpochProcessor::default(); + + for (k, v) in hash { + let key = k + .as_str() + .ok_or(format!("key must be a string, but got {k:?}"))?; + + match key { + FIELD_NAME => { + processor.with_fields(Fields::one(yaml_field(v, FIELD_NAME)?)); + } + FIELDS_NAME => { + processor.with_fields(yaml_fields(v, FIELDS_NAME)?); + } + RESOLUTION_NAME => { + let s = yaml_string(v, RESOLUTION_NAME)?.as_str().try_into()?; + processor.with_resolution(s); + } + IGNORE_MISSING_NAME => { + processor.with_ignore_missing(yaml_bool(v, IGNORE_MISSING_NAME)?); + } + + _ => {} + } + } + + Ok(processor) + } +} + +impl Processor for EpochProcessor { + fn kind(&self) -> &str { + PROCESSOR_EPOCH + } + + fn ignore_missing(&self) -> bool { + self.ignore_missing + } + + fn fields(&self) -> &Fields { + &self.fields + } + + fn exec_field(&self, val: &Value, field: &Field) -> Result { + self.process_field(val, field) + } +} + +#[cfg(test)] +mod tests { + use super::EpochProcessor; + use crate::etl::value::Value; + + #[test] + fn test_parse_epoch() { + let mut processor = EpochProcessor::default(); + processor.with_resolution(super::Resolution::Second); + + let values = [ + Value::String("1573840000".into()), + Value::Int32(1573840000), + Value::Uint64(1573840000), + Value::Float32(1573840000.0), + ]; + + for value in values { + let parsed = processor.parse(&value).unwrap(); + assert_eq!(parsed, super::Epoch::Second(1573840000)); + } + } +} diff --git a/src/pipeline/src/etl/processor/letter.rs b/src/pipeline/src/etl/processor/letter.rs new file mode 100644 index 000000000000..1c2fcf9eacfc --- /dev/null +++ b/src/pipeline/src/etl/processor/letter.rs @@ -0,0 +1,188 @@ +// Copyright 2024 Greptime Team +// +// Licensed 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 crate::etl::field::{Field, Fields}; +use crate::etl::processor::{ + yaml_bool, yaml_field, yaml_fields, yaml_string, Processor, FIELDS_NAME, FIELD_NAME, + IGNORE_MISSING_NAME, METHOD_NAME, +}; +use crate::etl::value::{Map, Value}; + +pub(crate) const PROCESSOR_LETTER: &str = "letter"; + +#[derive(Debug, Default)] +enum Method { + Upper, + #[default] + Lower, + Capital, +} + +impl std::fmt::Display for Method { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + match self { + Method::Upper => write!(f, "upper"), + Method::Lower => write!(f, "lower"), + Method::Capital => write!(f, "capital"), + } + } +} + +impl std::str::FromStr for Method { + type Err = String; + + fn from_str(s: &str) -> Result { + match s.to_lowercase().as_str() { + "upper" => Ok(Method::Upper), + "lower" => Ok(Method::Lower), + "capital" => Ok(Method::Capital), + _ => Err(format!("invalid method: {s}")), + } + } +} + +/// only support string value +#[derive(Debug, Default)] +pub struct LetterProcessor { + fields: Fields, + method: Method, + ignore_missing: bool, +} + +impl LetterProcessor { + fn with_fields(&mut self, fields: Fields) { + self.fields = fields; + } + + fn with_method(&mut self, method: Method) { + self.method = method; + } + + fn with_ignore_missing(&mut self, ignore_missing: bool) { + self.ignore_missing = ignore_missing; + } + + fn process_field(&self, val: &str, field: &Field) -> Result { + let processed = match self.method { + Method::Upper => val.to_uppercase(), + Method::Lower => val.to_lowercase(), + Method::Capital => capitalize(val), + }; + let val = Value::String(processed); + + let key = match field.target_field { + Some(ref target_field) => target_field, + None => field.get_field(), + }; + + Ok(Map::one(key, val)) + } +} + +impl TryFrom<&yaml_rust::yaml::Hash> for LetterProcessor { + type Error = String; + + fn try_from(value: &yaml_rust::yaml::Hash) -> Result { + let mut processor = LetterProcessor::default(); + + for (k, v) in value.iter() { + let key = k + .as_str() + .ok_or(format!("key must be a string, but got {k:?}"))?; + match key { + FIELD_NAME => { + processor.with_fields(Fields::one(yaml_field(v, FIELD_NAME)?)); + } + FIELDS_NAME => { + processor.with_fields(yaml_fields(v, FIELDS_NAME)?); + } + METHOD_NAME => { + let method = yaml_string(v, METHOD_NAME)?; + processor.with_method(method.parse()?); + } + IGNORE_MISSING_NAME => { + processor.with_ignore_missing(yaml_bool(v, IGNORE_MISSING_NAME)?); + } + _ => {} + } + } + + Ok(processor) + } +} + +impl Processor for LetterProcessor { + fn kind(&self) -> &str { + PROCESSOR_LETTER + } + + fn ignore_missing(&self) -> bool { + self.ignore_missing + } + + fn fields(&self) -> &Fields { + &self.fields + } + + fn exec_field(&self, val: &Value, field: &Field) -> Result { + match val { + Value::String(val) => self.process_field(val, field), + _ => Err(format!( + "{} processor: expect string value, but got {val:?}", + self.kind() + )), + } + } +} + +fn capitalize(s: &str) -> String { + let mut c = s.chars(); + match c.next() { + None => String::new(), + Some(f) => f.to_uppercase().collect::() + c.as_str(), + } +} + +#[cfg(test)] +mod tests { + use crate::etl::field::Fields; + use crate::etl::processor::letter::{LetterProcessor, Method}; + use crate::etl::value::{Map, Value}; + + #[test] + fn test_process() { + let field = "letter"; + let ff: crate::etl::processor::Field = field.parse().unwrap(); + let mut processor = LetterProcessor::default(); + processor.with_fields(Fields::one(ff.clone())); + + { + processor.with_method(Method::Upper); + let processed = processor.process_field("pipeline", &ff).unwrap(); + assert_eq!(Map::one(field, Value::String("PIPELINE".into())), processed) + } + + { + processor.with_method(Method::Lower); + let processed = processor.process_field("Pipeline", &ff).unwrap(); + assert_eq!(Map::one(field, Value::String("pipeline".into())), processed) + } + + { + processor.with_method(Method::Capital); + let processed = processor.process_field("pipeline", &ff).unwrap(); + assert_eq!(Map::one(field, Value::String("Pipeline".into())), processed) + } + } +} diff --git a/src/pipeline/src/etl/processor/mod.rs b/src/pipeline/src/etl/processor/mod.rs new file mode 100644 index 000000000000..c04414b87cbd --- /dev/null +++ b/src/pipeline/src/etl/processor/mod.rs @@ -0,0 +1,198 @@ +// Copyright 2024 Greptime Team +// +// Licensed 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. + +pub mod cmcd; +pub mod csv; +pub mod date; +pub mod epoch; +pub mod letter; +pub mod regex; +pub mod urlencoding; + +use std::sync::Arc; + +use cmcd::CMCDProcessor; +use common_telemetry::warn; +use csv::CsvProcessor; +use date::DateProcessor; +use epoch::EpochProcessor; +use letter::LetterProcessor; +use regex::RegexProcessor; +use urlencoding::UrlEncodingProcessor; + +use crate::etl::field::{Field, Fields}; +use crate::etl::value::{Array, Map, Value}; + +const FIELD_NAME: &str = "field"; +const FIELDS_NAME: &str = "fields"; +const IGNORE_MISSING_NAME: &str = "ignore_missing"; +const METHOD_NAME: &str = "method"; + +// const IF_NAME: &str = "if"; +// const IGNORE_FAILURE_NAME: &str = "ignore_failure"; +// const ON_FAILURE_NAME: &str = "on_failure"; +// const TAG_NAME: &str = "tag"; + +pub trait Processor: std::fmt::Debug + Send + Sync + 'static { + fn fields(&self) -> &Fields; + fn kind(&self) -> &str; + fn ignore_missing(&self) -> bool; + + fn ignore_processor_array_failure(&self) -> bool { + true + } + + fn exec_field(&self, _val: &Value, _field: &Field) -> Result { + Ok(Map::default()) + } + + fn exec_map(&self, mut map: Map) -> Result { + for ff @ Field { field, .. } in self.fields().iter() { + let val = map.get(field); + match val { + Some(v) => { + map.extend(self.exec_field(v, ff)?); + } + None if self.ignore_missing() => {} + None => { + return Err(format!( + "{} processor: field '{field}' is required but missing in {map}", + self.kind(), + )) + } + } + } + + Ok(Value::Map(map)) + } + + fn exec_array(&self, arr: Array) -> Result { + let mut values = vec![]; + for val in arr.into_iter() { + match val { + Value::Map(map) => { + values.push(self.exec_map(map)?); + } + _ if self.ignore_processor_array_failure() => { + warn!("expected a map, but got {val}") + } + _ => return Err(format!("expected a map, but got {}", val)), + } + } + + Ok(Value::Array(Array { values })) + } + + fn exec(&self, val: Value) -> Result { + match val { + Value::Map(map) => self.exec_map(map), + Value::Array(arr) => self.exec_array(arr), + _ => Err(format!("expected a map or array, but got {}", val)), + } + } +} + +#[derive(Debug, Default, Clone)] +pub struct Processors { + pub processors: Vec>, +} + +impl Processors { + pub fn new() -> Self { + Processors { processors: vec![] } + } +} + +impl std::ops::Deref for Processors { + type Target = Vec>; + + fn deref(&self) -> &Self::Target { + &self.processors + } +} + +impl TryFrom<&Vec> for Processors { + type Error = String; + + fn try_from(vec: &Vec) -> Result { + let mut processors = vec![]; + + for doc in vec { + processors.push(parse_processor(doc)?); + } + + Ok(Processors { processors }) + } +} + +fn parse_processor(doc: &yaml_rust::Yaml) -> Result, String> { + let map = doc.as_hash().ok_or("processor must be a map".to_string())?; + + let key = map + .keys() + .next() + .ok_or("processor must have a string key".to_string())?; + + let value = map + .get(key) + .unwrap() + .as_hash() + .expect("processor value must be a map"); + + let str_key = key + .as_str() + .ok_or("processor key must be a string".to_string())?; + + let processor: Arc = match str_key { + cmcd::PROCESSOR_CMCD => Arc::new(CMCDProcessor::try_from(value)?), + csv::PROCESSOR_CSV => Arc::new(CsvProcessor::try_from(value)?), + date::PROCESSOR_DATE => Arc::new(DateProcessor::try_from(value)?), + epoch::PROCESSOR_EPOCH => Arc::new(EpochProcessor::try_from(value)?), + letter::PROCESSOR_LETTER => Arc::new(LetterProcessor::try_from(value)?), + regex::PROCESSOR_REGEX => Arc::new(RegexProcessor::try_from(value)?), + urlencoding::PROCESSOR_URL_ENCODING => Arc::new(UrlEncodingProcessor::try_from(value)?), + _ => return Err(format!("unsupported {} processor", str_key)), + }; + + Ok(processor) +} + +pub(crate) fn yaml_string(v: &yaml_rust::Yaml, field: &str) -> Result { + v.as_str() + .map(|s| s.trim().to_string()) + .ok_or(format!("'{field}' must be a string")) +} + +pub(crate) fn yaml_strings(v: &yaml_rust::Yaml, field: &str) -> Result, String> { + let vec = v + .as_vec() + .ok_or(format!("'{field}' must be a list of strings",))? + .iter() + .map(|v| v.as_str().unwrap_or_default().into()) + .collect(); + Ok(vec) +} + +pub(crate) fn yaml_bool(v: &yaml_rust::Yaml, field: &str) -> Result { + v.as_bool().ok_or(format!("'{field}' must be a boolean")) +} + +pub(crate) fn yaml_field(v: &yaml_rust::Yaml, field: &str) -> Result { + yaml_string(v, field)?.parse() +} + +pub(crate) fn yaml_fields(v: &yaml_rust::Yaml, field: &str) -> Result { + let ff = yaml_strings(v, field).and_then(|v| v.into_iter().map(|s| s.parse()).collect())?; + Fields::new(ff) +} diff --git a/src/pipeline/src/etl/processor/regex.rs b/src/pipeline/src/etl/processor/regex.rs new file mode 100644 index 000000000000..078deef603b5 --- /dev/null +++ b/src/pipeline/src/etl/processor/regex.rs @@ -0,0 +1,315 @@ +// Copyright 2024 Greptime Team +// +// Licensed 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. + +// field_name and prefix with comma separated, like: +// name, new_name +const PATTERNS_NAME: &str = "patterns"; + +pub(crate) const PROCESSOR_REGEX: &str = "regex"; + +use lazy_static::lazy_static; +use regex::Regex; + +use crate::etl::field::Fields; +use crate::etl::processor::{ + yaml_bool, yaml_field, yaml_fields, yaml_strings, Field, Processor, FIELDS_NAME, FIELD_NAME, + IGNORE_MISSING_NAME, +}; +use crate::etl::value::{Map, Value}; + +lazy_static! { + static ref GROUPS_NAME_REGEX: Regex = Regex::new(r"\(\?P?<([[:word:]]+)>.+?\)").unwrap(); +} + +fn get_regex_group_names(s: &str) -> Vec { + GROUPS_NAME_REGEX + .captures_iter(s) + .filter_map(|c| c.get(1).map(|m| m.as_str().to_string())) + .collect() +} + +#[derive(Debug)] +struct GroupRegex { + origin: String, + regex: Regex, + groups: Vec, +} + +impl std::fmt::Display for GroupRegex { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + let groups = self.groups.join(", "); + write!(f, "{}, groups: [{groups}]", self.origin) + } +} + +impl std::str::FromStr for GroupRegex { + type Err = String; + + fn from_str(origin: &str) -> Result { + let groups = get_regex_group_names(origin); + if groups.is_empty() { + return Err(format!("no named group found in regex {origin}")); + } + + let regex = Regex::new(origin).map_err(|e| e.to_string())?; + Ok(GroupRegex { + origin: origin.into(), + regex, + groups, + }) + } +} + +/// only support string value +/// if no value found from a pattern, the target_field will be ignored +#[derive(Debug, Default)] +pub struct RegexProcessor { + fields: Fields, + patterns: Vec, + ignore_missing: bool, +} + +impl RegexProcessor { + fn with_fields(&mut self, fields: Fields) { + self.fields = fields; + } + + fn try_with_patterns(&mut self, patterns: Vec) -> Result<(), String> { + let mut rs = vec![]; + for pattern in patterns { + let gr = pattern.parse()?; + rs.push(gr); + } + self.patterns = rs; + Ok(()) + } + + fn with_ignore_missing(&mut self, ignore_missing: bool) { + self.ignore_missing = ignore_missing; + } + + fn check(self) -> Result { + if self.fields.is_empty() { + return Err(format!( + "no valid field found in {} processor", + PROCESSOR_REGEX + )); + } + + if self.patterns.is_empty() { + return Err(format!( + "no valid pattern found in {} processor", + PROCESSOR_REGEX + )); + } + + Ok(self) + } + + fn process_field(&self, val: &str, field: &Field, gr: &GroupRegex) -> Result { + let mut map = Map::default(); + + if let Some(captures) = gr.regex.captures(val) { + for group in &gr.groups { + if let Some(capture) = captures.name(group) { + let value = capture.as_str().to_string(); + let prefix = match &field.target_field { + Some(s) => s, + None => &field.field, + }; + + let key = format!("{prefix}_{group}"); + + map.insert(key, Value::String(value)); + } + } + } + + Ok(map) + } +} + +impl TryFrom<&yaml_rust::yaml::Hash> for RegexProcessor { + type Error = String; + + fn try_from(value: &yaml_rust::yaml::Hash) -> Result { + let mut processor = RegexProcessor::default(); + + for (k, v) in value.iter() { + let key = k + .as_str() + .ok_or(format!("key must be a string, but got {k:?}"))?; + match key { + FIELD_NAME => { + processor.with_fields(Fields::one(yaml_field(v, FIELD_NAME)?)); + } + FIELDS_NAME => { + processor.with_fields(yaml_fields(v, FIELDS_NAME)?); + } + PATTERNS_NAME => { + processor.try_with_patterns(yaml_strings(v, PATTERNS_NAME)?)?; + } + IGNORE_MISSING_NAME => { + processor.with_ignore_missing(yaml_bool(v, IGNORE_MISSING_NAME)?); + } + _ => {} + } + } + + processor.check() + } +} + +impl Processor for RegexProcessor { + fn kind(&self) -> &str { + PROCESSOR_REGEX + } + + fn ignore_missing(&self) -> bool { + self.ignore_missing + } + + fn fields(&self) -> &Fields { + &self.fields + } + + fn exec_field(&self, val: &Value, field: &Field) -> Result { + match val { + Value::String(val) => { + let mut map = Map::default(); + for gr in &self.patterns { + let m = self.process_field(val, field, gr)?; + map.extend(m); + } + Ok(map) + } + _ => Err(format!( + "{} processor: expect string value, but got {val:?}", + self.kind() + )), + } + } +} +#[cfg(test)] +mod tests { + use itertools::Itertools; + + use super::RegexProcessor; + use crate::etl::field::Fields; + use crate::etl::processor::Processor; + use crate::etl::value::{Map, Value}; + + #[test] + fn test_process() { + let mut processor = RegexProcessor::default(); + + let cc = "[c=c,n=US_CA_SANJOSE,o=55155]"; + let cg = "[a=12.34.567.89,b=12345678,c=g,n=US_CA_SANJOSE,o=20940]"; + let co = "[a=987.654.321.09,c=o]"; + let cp = "[c=p,n=US_CA_SANJOSE,o=55155]"; + let cw = "[c=w,n=US_CA_SANJOSE,o=55155]"; + let breadcrumbs = Value::String([cc, cg, co, cp, cw].iter().join(",")); + + let values = [ + ("breadcrumbs", breadcrumbs.clone()), + ("breadcrumbs_parent", Value::String(cc.to_string())), + ("breadcrumbs_edge", Value::String(cg.to_string())), + ("breadcrumbs_origin", Value::String(co.to_string())), + ("breadcrumbs_peer", Value::String(cp.to_string())), + ("breadcrumbs_wrapper", Value::String(cw.to_string())), + ] + .into_iter() + .map(|(k, v)| (k.to_string(), v)) + .collect(); + let temporary_map = Map { values }; + + { + // single field (with prefix), multiple patterns + let ff = ["breadcrumbs, breadcrumbs"] + .iter() + .map(|f| f.parse().unwrap()) + .collect(); + processor.with_fields(Fields::new(ff).unwrap()); + + let ccr = "(?\\[[^\\[]*c=c[^\\]]*\\])"; + let cgr = "(?\\[[^\\[]*c=g[^\\]]*\\])"; + let cor = "(?\\[[^\\[]*c=o[^\\]]*\\])"; + let cpr = "(?\\[[^\\[]*c=p[^\\]]*\\])"; + let cwr = "(?\\[[^\\[]*c=w[^\\]]*\\])"; + let patterns = [ccr, cgr, cor, cpr, cwr] + .iter() + .map(|p| p.to_string()) + .collect(); + processor.try_with_patterns(patterns).unwrap(); + + let mut map = Map::default(); + map.insert("breadcrumbs", breadcrumbs.clone()); + let processed_val = processor.exec_map(map).unwrap(); + + assert_eq!(processed_val, Value::Map(temporary_map.clone())); + } + + { + // multiple fields (with prefix), multiple patterns + let ff = [ + "breadcrumbs_parent, parent", + "breadcrumbs_edge, edge", + "breadcrumbs_origin, origin", + "breadcrumbs_peer, peer", + "breadcrumbs_wrapper, wrapper", + ] + .iter() + .map(|f| f.parse().unwrap()) + .collect(); + processor.with_fields(Fields::new(ff).unwrap()); + + let patterns = [ + "a=(?[^,\\]]+)", + "b=(?[^,\\]]+)", + "k=(?[^,\\]]+)", + "l=(?[^,\\]]+)", + "m=(?[^,\\]]+)", + "n=(?[^,\\]]+)", + "o=(?[^,\\]]+)", + ] + .iter() + .map(|p| p.to_string()) + .collect(); + processor.try_with_patterns(patterns).unwrap(); + + let new_values = vec![ + ("edge_ip", Value::String("12.34.567.89".to_string())), + ("edge_request_id", Value::String("12345678".to_string())), + ("edge_geo", Value::String("US_CA_SANJOSE".to_string())), + ("edge_asn", Value::String("20940".to_string())), + ("origin_ip", Value::String("987.654.321.09".to_string())), + ("peer_asn", Value::String("55155".to_string())), + ("peer_geo", Value::String("US_CA_SANJOSE".to_string())), + ("parent_asn", Value::String("55155".to_string())), + ("parent_geo", Value::String("US_CA_SANJOSE".to_string())), + ("wrapper_asn", Value::String("55155".to_string())), + ("wrapper_geo", Value::String("US_CA_SANJOSE".to_string())), + ] + .into_iter() + .map(|(k, v)| (k.to_string(), v)) + .collect(); + + let actual_val = processor.exec_map(temporary_map.clone()).unwrap(); + let mut expected_map = temporary_map.clone(); + expected_map.extend(Map { values: new_values }); + + assert_eq!(Value::Map(expected_map), actual_val); + } + } +} diff --git a/src/pipeline/src/etl/processor/urlencoding.rs b/src/pipeline/src/etl/processor/urlencoding.rs new file mode 100644 index 000000000000..f9019fd19126 --- /dev/null +++ b/src/pipeline/src/etl/processor/urlencoding.rs @@ -0,0 +1,177 @@ +// Copyright 2024 Greptime Team +// +// Licensed 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 urlencoding::{decode, encode}; + +use crate::etl::field::{Field, Fields}; +use crate::etl::processor::{ + yaml_bool, yaml_field, yaml_fields, yaml_string, FIELDS_NAME, FIELD_NAME, IGNORE_MISSING_NAME, + METHOD_NAME, +}; +use crate::etl::value::{Map, Value}; + +pub(crate) const PROCESSOR_URL_ENCODING: &str = "urlencoding"; + +#[derive(Debug, Default)] +enum Method { + #[default] + Decode, + Encode, +} + +impl std::fmt::Display for Method { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + match self { + Method::Decode => write!(f, "decode"), + Method::Encode => write!(f, "encode"), + } + } +} + +impl std::str::FromStr for Method { + type Err = String; + + fn from_str(s: &str) -> Result { + match s { + "decode" => Ok(Method::Decode), + "encode" => Ok(Method::Encode), + _ => Err(format!("invalid method: {s}")), + } + } +} + +/// only support string value +#[derive(Debug, Default)] +pub struct UrlEncodingProcessor { + fields: Fields, + method: Method, + ignore_missing: bool, +} + +impl UrlEncodingProcessor { + fn with_fields(&mut self, fields: Fields) { + self.fields = fields; + } + + fn with_ignore_missing(&mut self, ignore_missing: bool) { + self.ignore_missing = ignore_missing; + } + + fn with_method(&mut self, method: Method) { + self.method = method; + } + + fn process_field(&self, val: &str, field: &Field) -> Result { + let processed = match self.method { + Method::Encode => encode(val).to_string(), + Method::Decode => decode(val).map_err(|e| e.to_string())?.into_owned(), + }; + let val = Value::String(processed); + + let key = match field.target_field { + Some(ref target_field) => target_field, + None => field.get_field(), + }; + + Ok(Map::one(key, val)) + } +} + +impl TryFrom<&yaml_rust::yaml::Hash> for UrlEncodingProcessor { + type Error = String; + + fn try_from(value: &yaml_rust::yaml::Hash) -> Result { + let mut processor = UrlEncodingProcessor::default(); + + for (k, v) in value.iter() { + let key = k + .as_str() + .ok_or(format!("key must be a string, but got {k:?}"))?; + match key { + FIELD_NAME => { + processor.with_fields(Fields::one(yaml_field(v, FIELD_NAME)?)); + } + FIELDS_NAME => { + processor.with_fields(yaml_fields(v, FIELDS_NAME)?); + } + + IGNORE_MISSING_NAME => { + processor.with_ignore_missing(yaml_bool(v, IGNORE_MISSING_NAME)?); + } + + METHOD_NAME => { + let method = yaml_string(v, METHOD_NAME)?; + processor.with_method(method.parse()?); + } + + _ => {} + } + } + + Ok(processor) + } +} + +impl crate::etl::processor::Processor for UrlEncodingProcessor { + fn kind(&self) -> &str { + PROCESSOR_URL_ENCODING + } + + fn ignore_missing(&self) -> bool { + self.ignore_missing + } + + fn fields(&self) -> &Fields { + &self.fields + } + + fn exec_field(&self, val: &Value, field: &Field) -> Result { + match val { + Value::String(val) => self.process_field(val, field), + _ => Err(format!( + "{} processor: expect string value, but got {val:?}", + self.kind() + )), + } + } +} + +#[cfg(test)] +mod tests { + use crate::etl::field::{Field, Fields}; + use crate::etl::processor::urlencoding::UrlEncodingProcessor; + use crate::etl::value::{Map, Value}; + + #[test] + fn test_decode_url() { + let field = "url"; + let ff: Field = field.parse().unwrap(); + + let decoded = "//BC/[a=6.7.8.9,c=g,k=0,l=1]"; + let encoded = "%2F%2FBC%2F%5Ba%3D6.7.8.9%2Cc%3Dg%2Ck%3D0%2Cl%3D1%5D"; + + let mut processor = UrlEncodingProcessor::default(); + processor.with_fields(Fields::one(ff.clone())); + + { + let result = processor.process_field(encoded, &ff).unwrap(); + assert_eq!(Map::one(field, Value::String(decoded.into())), result) + } + { + processor.with_method(super::Method::Encode); + let result = processor.process_field(decoded, &ff).unwrap(); + assert_eq!(Map::one(field, Value::String(encoded.into())), result) + } + } +} diff --git a/src/pipeline/src/etl/transform/index.rs b/src/pipeline/src/etl/transform/index.rs new file mode 100644 index 000000000000..b554824f52a3 --- /dev/null +++ b/src/pipeline/src/etl/transform/index.rs @@ -0,0 +1,57 @@ +// Copyright 2024 Greptime Team +// +// Licensed 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. + +const INDEX_TIMESTAMP: &str = "timestamp"; +const INDEX_TAG: &str = "tag"; +const INDEX_FULLTEXT: &str = "fulltext"; + +#[derive(Debug, PartialEq, Eq, Clone, Copy)] +pub enum Index { + Timestamp, + Tag, + Fulltext, +} + +impl std::fmt::Display for Index { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + let index = match self { + Index::Timestamp => INDEX_TIMESTAMP, + Index::Tag => INDEX_TAG, + Index::Fulltext => INDEX_FULLTEXT, + }; + + write!(f, "{}", index) + } +} + +impl TryFrom for Index { + type Error = String; + + fn try_from(value: String) -> Result { + Index::try_from(value.as_str()) + } +} + +impl TryFrom<&str> for Index { + type Error = String; + + fn try_from(value: &str) -> Result { + match value { + INDEX_TIMESTAMP => Ok(Index::Timestamp), + INDEX_TAG => Ok(Index::Tag), + INDEX_FULLTEXT => Ok(Index::Fulltext), + _ => Err(format!("unsupported index type: {}", value)), + } + } +} diff --git a/src/pipeline/src/etl/transform/mod.rs b/src/pipeline/src/etl/transform/mod.rs new file mode 100644 index 000000000000..1c1a0f4141de --- /dev/null +++ b/src/pipeline/src/etl/transform/mod.rs @@ -0,0 +1,205 @@ +// Copyright 2024 Greptime Team +// +// Licensed 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. + +pub mod index; +pub mod transformer; + +use itertools::Itertools; + +use crate::etl::field::Fields; +use crate::etl::processor::{yaml_field, yaml_fields, yaml_string}; +use crate::etl::transform::index::Index; +use crate::etl::value::Value; + +const TRANSFORM_FIELD: &str = "field"; +const TRANSFORM_FIELDS: &str = "fields"; +const TRANSFORM_TYPE: &str = "type"; +const TRANSFORM_INDEX: &str = "index"; +const TRANSFORM_DEFAULT: &str = "default"; + +pub use transformer::greptime::GreptimeTransformer; +// pub use transformer::noop::NoopTransformer; + +pub trait Transformer: std::fmt::Display + Sized + Send + Sync + 'static { + type Output; + + fn new(transforms: Transforms) -> Result; + fn transform(&self, val: crate::etl::value::Value) -> Result; +} + +#[derive(Debug, Default, Clone)] +pub struct Transforms { + transforms: Vec, +} + +impl std::fmt::Display for Transforms { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + let transforms = self + .transforms + .iter() + .map(|field| field.to_string()) + .join(", "); + + write!(f, "{}", transforms) + } +} + +impl std::ops::Deref for Transforms { + type Target = Vec; + + fn deref(&self) -> &Self::Target { + &self.transforms + } +} + +impl std::ops::DerefMut for Transforms { + fn deref_mut(&mut self) -> &mut Self::Target { + &mut self.transforms + } +} + +impl TryFrom<&Vec> for Transforms { + type Error = String; + + fn try_from(docs: &Vec) -> Result { + let mut transforms = vec![]; + + for doc in docs { + let transform: Transform = doc + .as_hash() + .ok_or("transform element must be a map".to_string())? + .try_into()?; + transforms.push(transform); + } + + Ok(Transforms { transforms }) + } +} + +/// only field is required +#[derive(Debug, Clone)] +pub struct Transform { + pub fields: Fields, + + pub type_: Value, + + pub default: Option, + + pub index: Option, +} + +impl std::fmt::Display for Transform { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + let index = if let Some(index) = &self.index { + format!(", index: {}", index) + } else { + "".to_string() + }; + + let fields = format!("field(s): {}", self.fields); + let type_ = format!("type: {}", self.type_); + + write!(f, "{type_}{index}, {fields}") + } +} + +impl Default for Transform { + fn default() -> Self { + Transform { + fields: Fields::default(), + type_: Value::Null, + default: None, + index: None, + } + } +} + +impl Transform { + fn with_fields(&mut self, fields: Fields) { + self.fields = fields; + } + + fn with_type(&mut self, type_: Value) { + self.type_ = type_; + } + + fn try_default(&mut self, default: Value) -> Result<(), String> { + match (&self.type_, &default) { + (Value::Null, _) => Err(format!( + "transform {} type MUST BE set before default {}", + self.fields, &default, + )), + (_, Value::Null) => Ok(()), // if default is not set, then it will be regarded as default null + (_, _) => { + let target = self + .type_ + .parse_str_value(default.to_str_value().as_str())?; + self.default = Some(target); + Ok(()) + } + } + } + + fn with_index(&mut self, index: Index) { + self.index = Some(index); + } + + pub(crate) fn get_default(&self) -> Option<&Value> { + self.default.as_ref() + } +} + +impl TryFrom<&yaml_rust::yaml::Hash> for Transform { + type Error = String; + + fn try_from(hash: &yaml_rust::yaml::Hash) -> Result { + let mut transform = Transform::default(); + + let mut default_opt = None; + + for (k, v) in hash { + let key = k.as_str().ok_or("key must be a string")?; + match key { + TRANSFORM_FIELD => { + transform.with_fields(Fields::one(yaml_field(v, TRANSFORM_FIELD)?)); + } + + TRANSFORM_FIELDS => { + transform.with_fields(yaml_fields(v, TRANSFORM_FIELDS)?); + } + + TRANSFORM_TYPE => { + let t = yaml_string(v, TRANSFORM_TYPE)?; + transform.with_type(Value::parse_str_type(&t)?); + } + + TRANSFORM_INDEX => { + let index = yaml_string(v, TRANSFORM_INDEX)?; + transform.with_index(index.try_into()?); + } + + TRANSFORM_DEFAULT => { + default_opt = Some(Value::try_from(v)?); + } + _ => {} + } + } + + if let Some(default) = default_opt { + transform.try_default(default)?; + } + + Ok(transform) + } +} diff --git a/src/pipeline/src/etl/transform/transformer/greptime/coerce.rs b/src/pipeline/src/etl/transform/transformer/greptime/coerce.rs new file mode 100644 index 000000000000..48b612e3d53a --- /dev/null +++ b/src/pipeline/src/etl/transform/transformer/greptime/coerce.rs @@ -0,0 +1,310 @@ +// Copyright 2024 Greptime Team +// +// Licensed 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 greptime_proto::v1::value::ValueData; +use greptime_proto::v1::{ColumnDataType, ColumnSchema, SemanticType}; + +use crate::etl::transform::index::Index; +use crate::etl::transform::Transform; +use crate::etl::value::{Epoch, Time, Value}; + +impl TryFrom for ValueData { + type Error = String; + + fn try_from(value: Value) -> Result { + match value { + Value::Null => Err("Null type not supported".to_string()), + + Value::Int8(v) => Ok(ValueData::I32Value(v as i32)), + Value::Int16(v) => Ok(ValueData::I32Value(v as i32)), + Value::Int32(v) => Ok(ValueData::I32Value(v)), + Value::Int64(v) => Ok(ValueData::I64Value(v)), + + Value::Uint8(v) => Ok(ValueData::U32Value(v as u32)), + Value::Uint16(v) => Ok(ValueData::U32Value(v as u32)), + Value::Uint32(v) => Ok(ValueData::U32Value(v)), + Value::Uint64(v) => Ok(ValueData::U64Value(v)), + + Value::Float32(v) => Ok(ValueData::F32Value(v)), + Value::Float64(v) => Ok(ValueData::F64Value(v)), + + Value::Boolean(v) => Ok(ValueData::BoolValue(v)), + Value::String(v) => Ok(ValueData::StringValue(v.clone())), + + Value::Time(Time { nanosecond, .. }) => Ok(ValueData::TimeNanosecondValue(nanosecond)), + + Value::Epoch(Epoch::Nanosecond(ns)) => Ok(ValueData::TimestampNanosecondValue(ns)), + Value::Epoch(Epoch::Microsecond(us)) => Ok(ValueData::TimestampMicrosecondValue(us)), + Value::Epoch(Epoch::Millisecond(ms)) => Ok(ValueData::TimestampMillisecondValue(ms)), + Value::Epoch(Epoch::Second(s)) => Ok(ValueData::TimestampSecondValue(s)), + + Value::Array(_) => unimplemented!("Array type not supported"), + Value::Map(_) => unimplemented!("Object type not supported"), + } + } +} + +// TODO(yuanbohan): add fulltext support in datatype_extension +pub(crate) fn coerce_columns(transform: &Transform) -> Result, String> { + let mut columns = Vec::new(); + + for field in transform.fields.iter() { + let column_name = field.get_target_field().to_string(); + + let datatype = coerce_type(transform)? as i32; + + let semantic_type = coerce_semantic_type(transform) as i32; + + let column = ColumnSchema { + column_name, + datatype, + semantic_type, + datatype_extension: None, + }; + columns.push(column); + } + + Ok(columns) +} + +fn coerce_semantic_type(transform: &Transform) -> SemanticType { + match transform.index { + Some(Index::Tag) => SemanticType::Tag, + Some(Index::Timestamp) => SemanticType::Timestamp, + Some(Index::Fulltext) => unimplemented!("Fulltext"), + None => SemanticType::Field, + } +} + +fn coerce_type(transform: &Transform) -> Result { + match transform.type_ { + Value::Int8(_) => Ok(ColumnDataType::Int8), + Value::Int16(_) => Ok(ColumnDataType::Int16), + Value::Int32(_) => Ok(ColumnDataType::Int32), + Value::Int64(_) => Ok(ColumnDataType::Int64), + + Value::Uint8(_) => Ok(ColumnDataType::Uint8), + Value::Uint16(_) => Ok(ColumnDataType::Uint16), + Value::Uint32(_) => Ok(ColumnDataType::Uint32), + Value::Uint64(_) => Ok(ColumnDataType::Uint64), + + Value::Float32(_) => Ok(ColumnDataType::Float32), + Value::Float64(_) => Ok(ColumnDataType::Float64), + + Value::Boolean(_) => Ok(ColumnDataType::Boolean), + Value::String(_) => Ok(ColumnDataType::String), + + Value::Time(_) => Ok(ColumnDataType::TimestampNanosecond), + + Value::Epoch(Epoch::Nanosecond(_)) => Ok(ColumnDataType::TimestampNanosecond), + Value::Epoch(Epoch::Microsecond(_)) => Ok(ColumnDataType::TimestampMicrosecond), + Value::Epoch(Epoch::Millisecond(_)) => Ok(ColumnDataType::TimestampMillisecond), + Value::Epoch(Epoch::Second(_)) => Ok(ColumnDataType::TimestampSecond), + + Value::Array(_) => unimplemented!("Array"), + Value::Map(_) => unimplemented!("Object"), + + Value::Null => Err(format!( + "Null type not supported when to coerce '{}' type", + transform.fields + )), + } +} + +pub(crate) fn coerce_value( + val: &Value, + transform: &Transform, +) -> Result, String> { + match val { + Value::Null => Ok(None), + + Value::Int8(n) => coerce_i64_value(*n as i64, transform), + Value::Int16(n) => coerce_i64_value(*n as i64, transform), + Value::Int32(n) => coerce_i64_value(*n as i64, transform), + Value::Int64(n) => coerce_i64_value(*n, transform), + + Value::Uint8(n) => coerce_u64_value(*n as u64, transform), + Value::Uint16(n) => coerce_u64_value(*n as u64, transform), + Value::Uint32(n) => coerce_u64_value(*n as u64, transform), + Value::Uint64(n) => coerce_u64_value(*n, transform), + + Value::Float32(n) => coerce_f64_value(*n as f64, transform), + Value::Float64(n) => coerce_f64_value(*n, transform), + + Value::Boolean(b) => coerce_bool_value(*b, transform), + Value::String(s) => coerce_string_value(s, transform), + + Value::Time(Time { nanosecond, .. }) => { + Ok(Some(ValueData::TimestampNanosecondValue(*nanosecond))) + } + + Value::Epoch(Epoch::Nanosecond(ns)) => Ok(Some(ValueData::TimestampNanosecondValue(*ns))), + Value::Epoch(Epoch::Microsecond(us)) => Ok(Some(ValueData::TimestampMicrosecondValue(*us))), + Value::Epoch(Epoch::Millisecond(ms)) => Ok(Some(ValueData::TimestampMillisecondValue(*ms))), + Value::Epoch(Epoch::Second(s)) => Ok(Some(ValueData::TimestampSecondValue(*s))), + + Value::Array(_) => unimplemented!("Array type not supported"), + Value::Map(_) => unimplemented!("Object type not supported"), + } +} + +fn coerce_bool_value(b: bool, transform: &Transform) -> Result, String> { + let val = match transform.type_ { + Value::Int8(_) => ValueData::I8Value(b as i32), + Value::Int16(_) => ValueData::I16Value(b as i32), + Value::Int32(_) => ValueData::I32Value(b as i32), + Value::Int64(_) => ValueData::I64Value(b as i64), + + Value::Uint8(_) => ValueData::U8Value(b as u32), + Value::Uint16(_) => ValueData::U16Value(b as u32), + Value::Uint32(_) => ValueData::U32Value(b as u32), + Value::Uint64(_) => ValueData::U64Value(b as u64), + + Value::Float32(_) => ValueData::F32Value(if b { 1.0 } else { 0.0 }), + Value::Float64(_) => ValueData::F64Value(if b { 1.0 } else { 0.0 }), + + Value::Boolean(_) => ValueData::BoolValue(b), + Value::String(_) => ValueData::StringValue(b.to_string()), + + Value::Time(_) => return Err("Boolean type not supported for Time".to_string()), + Value::Epoch(_) => return Err("Boolean type not supported for Epoch".to_string()), + + Value::Array(_) => unimplemented!("Array type not supported"), + Value::Map(_) => unimplemented!("Object type not supported"), + + Value::Null => return Ok(None), + }; + + Ok(Some(val)) +} + +fn coerce_i64_value(n: i64, transform: &Transform) -> Result, String> { + let val = match transform.type_ { + Value::Int8(_) => ValueData::I8Value(n as i32), + Value::Int16(_) => ValueData::I16Value(n as i32), + Value::Int32(_) => ValueData::I32Value(n as i32), + Value::Int64(_) => ValueData::I64Value(n), + + Value::Uint8(_) => ValueData::U8Value(n as u32), + Value::Uint16(_) => ValueData::U16Value(n as u32), + Value::Uint32(_) => ValueData::U32Value(n as u32), + Value::Uint64(_) => ValueData::U64Value(n as u64), + + Value::Float32(_) => ValueData::F32Value(n as f32), + Value::Float64(_) => ValueData::F64Value(n as f64), + + Value::Boolean(_) => ValueData::BoolValue(n != 0), + Value::String(_) => ValueData::StringValue(n.to_string()), + + Value::Time(_) => return Err("Integer type not supported for Time".to_string()), + Value::Epoch(_) => return Err("Integer type not supported for Epoch".to_string()), + + Value::Array(_) => unimplemented!("Array type not supported"), + Value::Map(_) => unimplemented!("Object type not supported"), + + Value::Null => return Ok(None), + }; + + Ok(Some(val)) +} + +fn coerce_u64_value(n: u64, transform: &Transform) -> Result, String> { + let val = match transform.type_ { + Value::Int8(_) => ValueData::I8Value(n as i32), + Value::Int16(_) => ValueData::I16Value(n as i32), + Value::Int32(_) => ValueData::I32Value(n as i32), + Value::Int64(_) => ValueData::I64Value(n as i64), + + Value::Uint8(_) => ValueData::U8Value(n as u32), + Value::Uint16(_) => ValueData::U16Value(n as u32), + Value::Uint32(_) => ValueData::U32Value(n as u32), + Value::Uint64(_) => ValueData::U64Value(n), + + Value::Float32(_) => ValueData::F32Value(n as f32), + Value::Float64(_) => ValueData::F64Value(n as f64), + + Value::Boolean(_) => ValueData::BoolValue(n != 0), + Value::String(_) => ValueData::StringValue(n.to_string()), + + Value::Time(_) => return Err("Integer type not supported for Time".to_string()), + Value::Epoch(_) => return Err("Integer type not supported for Epoch".to_string()), + + Value::Array(_) => unimplemented!("Array type not supported"), + Value::Map(_) => unimplemented!("Object type not supported"), + + Value::Null => return Ok(None), + }; + + Ok(Some(val)) +} + +fn coerce_f64_value(n: f64, transform: &Transform) -> Result, String> { + let val = match transform.type_ { + Value::Int8(_) => ValueData::I8Value(n as i32), + Value::Int16(_) => ValueData::I16Value(n as i32), + Value::Int32(_) => ValueData::I32Value(n as i32), + Value::Int64(_) => ValueData::I64Value(n as i64), + + Value::Uint8(_) => ValueData::U8Value(n as u32), + Value::Uint16(_) => ValueData::U16Value(n as u32), + Value::Uint32(_) => ValueData::U32Value(n as u32), + Value::Uint64(_) => ValueData::U64Value(n as u64), + + Value::Float32(_) => ValueData::F32Value(n as f32), + Value::Float64(_) => ValueData::F64Value(n), + + Value::Boolean(_) => ValueData::BoolValue(n != 0.0), + Value::String(_) => ValueData::StringValue(n.to_string()), + + Value::Time(_) => return Err("Float type not supported for Time".to_string()), + Value::Epoch(_) => return Err("Float type not supported for Epoch".to_string()), + + Value::Array(_) => unimplemented!("Array type not supported"), + Value::Map(_) => unimplemented!("Object type not supported"), + + Value::Null => return Ok(None), + }; + + Ok(Some(val)) +} + +fn coerce_string_value(s: &str, transform: &Transform) -> Result, String> { + let val = match transform.type_ { + Value::Int8(_) => ValueData::I8Value(s.parse::().map_err(|e| e.to_string())?), + Value::Int16(_) => ValueData::I16Value(s.parse::().map_err(|e| e.to_string())?), + Value::Int32(_) => ValueData::I32Value(s.parse::().map_err(|e| e.to_string())?), + Value::Int64(_) => ValueData::I64Value(s.parse::().map_err(|e| e.to_string())?), + + Value::Uint8(_) => ValueData::U8Value(s.parse::().map_err(|e| e.to_string())?), + Value::Uint16(_) => ValueData::U16Value(s.parse::().map_err(|e| e.to_string())?), + Value::Uint32(_) => ValueData::U32Value(s.parse::().map_err(|e| e.to_string())?), + Value::Uint64(_) => ValueData::U64Value(s.parse::().map_err(|e| e.to_string())?), + + Value::Float32(_) => ValueData::F32Value(s.parse::().map_err(|e| e.to_string())?), + Value::Float64(_) => ValueData::F64Value(s.parse::().map_err(|e| e.to_string())?), + + Value::Boolean(_) => ValueData::BoolValue(s.parse::().map_err(|e| e.to_string())?), + Value::String(_) => ValueData::StringValue(s.to_string()), + + Value::Time(_) => return Err("String type not supported for Time".to_string()), + Value::Epoch(_) => return Err("String type not supported for Epoch".to_string()), + + Value::Array(_) => unimplemented!("Array type not supported"), + Value::Map(_) => unimplemented!("Object type not supported"), + + Value::Null => return Ok(None), + }; + + Ok(Some(val)) +} diff --git a/src/pipeline/src/etl/transform/transformer/greptime/mod.rs b/src/pipeline/src/etl/transform/transformer/greptime/mod.rs new file mode 100644 index 000000000000..933f621d6a14 --- /dev/null +++ b/src/pipeline/src/etl/transform/transformer/greptime/mod.rs @@ -0,0 +1,172 @@ +// Copyright 2024 Greptime Team +// +// Licensed 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. + +pub mod coerce; + +use std::collections::HashSet; + +use coerce::{coerce_columns, coerce_value}; +use greptime_proto::v1::{ColumnSchema, Row, Rows, Value as GreptimeValue}; +use itertools::Itertools; + +use crate::etl::field::{Field, Fields}; +use crate::etl::transform::index::Index; +use crate::etl::transform::{Transform, Transformer, Transforms}; +use crate::etl::value::{Array, Epoch, Map, Value}; + +const DEFAULT_GREPTIME_TIMESTAMP_COLUMN: &str = "greptime_timestamp"; + +/// fields not in the columns will be discarded +/// to prevent automatic column creation in GreptimeDB +#[derive(Debug, Clone)] +pub struct GreptimeTransformer { + transforms: Transforms, +} + +impl GreptimeTransformer { + fn default_greptime_timestamp_column() -> Transform { + let ns = chrono::Utc::now().timestamp_nanos_opt().unwrap_or(0); + let type_ = Value::Epoch(Epoch::Nanosecond(ns)); + let default = Some(type_.clone()); + let field = Field::new(DEFAULT_GREPTIME_TIMESTAMP_COLUMN); + let fields = Fields::new(vec![field]).unwrap(); + + Transform { + fields, + type_, + default, + index: Some(Index::Timestamp), + } + } + + fn schemas(&self) -> Result, String> { + let mut schema = vec![]; + for transform in self.transforms.iter() { + schema.extend(coerce_columns(transform)?); + } + Ok(schema) + } + + fn transform_map(&self, map: &Map) -> Result { + let mut values = vec![]; + + for transform in self.transforms.iter() { + for field in transform.fields.iter() { + let value_data = match map.get(field.get_field()) { + Some(val) => coerce_value(val, transform)?, + None if transform.get_default().is_some() => { + coerce_value(transform.get_default().unwrap(), transform)? + } + None => None, + }; + values.push(GreptimeValue { value_data }); + } + } + + Ok(Row { values }) + } + + fn transform_array(&self, arr: &Array) -> Result, String> { + let mut rows = vec![]; + for v in arr.iter() { + match v { + Value::Map(map) => { + let row = self.transform_map(map)?; + rows.push(row); + } + _ => return Err(format!("Expected map, found: {v:?}")), + } + } + Ok(rows) + } +} + +impl std::fmt::Display for GreptimeTransformer { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + writeln!(f, "GreptimeTransformer.\nColumns: {}", self.transforms) + } +} + +impl Transformer for GreptimeTransformer { + type Output = Rows; + + fn new(mut transforms: Transforms) -> Result { + if transforms.is_empty() { + return Err("transform cannot be empty".to_string()); + } + + let mut column_names_set = HashSet::new(); + let mut timestamp_columns = vec![]; + + for transform in transforms.iter() { + let target_fields_set = transform + .fields + .iter() + .map(|f| f.get_target_field()) + .collect::>(); + + let intersections: Vec<_> = column_names_set.intersection(&target_fields_set).collect(); + if !intersections.is_empty() { + let duplicates = intersections.iter().join(","); + return Err(format!( + "column name must be unique, but got duplicated: {duplicates}" + )); + } + + column_names_set.extend(target_fields_set); + + if let Some(idx) = transform.index { + if idx == Index::Timestamp { + match transform.fields.len() { + 1 => timestamp_columns.push(transform.fields.first().unwrap().get_field()), + _ => return Err(format!( + "Illegal to set multiple timestamp Index columns, please set only one: {}", + transform.fields.get_target_fields().join(", ") + )), + } + } + } + } + + match timestamp_columns.len() { + 0 => { + transforms.push(GreptimeTransformer::default_greptime_timestamp_column()); + Ok(GreptimeTransformer { transforms }) + } + 1 => Ok(GreptimeTransformer { transforms }), + _ => { + let columns: String = timestamp_columns.iter().map(|s| s.to_string()).join(", "); + let count = timestamp_columns.len(); + Err( + format!("transform must have exactly one field specified as timestamp Index, but got {count}: {columns}") + ) + } + } + } + + fn transform(&self, value: Value) -> Result { + let schema = self.schemas()?; + match value { + Value::Map(map) => { + let rows = vec![self.transform_map(&map)?]; + Ok(Rows { schema, rows }) + } + Value::Array(arr) => { + let rows = self.transform_array(&arr)?; + Ok(Rows { schema, rows }) + } + _ => Err(format!("Expected map or array, found: {}", value)), + } + } +} diff --git a/src/pipeline/src/etl/transform/transformer/mod.rs b/src/pipeline/src/etl/transform/transformer/mod.rs new file mode 100644 index 000000000000..173aac61bfe5 --- /dev/null +++ b/src/pipeline/src/etl/transform/transformer/mod.rs @@ -0,0 +1,16 @@ +// Copyright 2024 Greptime Team +// +// Licensed 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. + +pub mod greptime; +pub mod noop; diff --git a/src/pipeline/src/etl/transform/transformer/noop.rs b/src/pipeline/src/etl/transform/transformer/noop.rs new file mode 100644 index 000000000000..6bd7a208c981 --- /dev/null +++ b/src/pipeline/src/etl/transform/transformer/noop.rs @@ -0,0 +1,36 @@ +// Copyright 2024 Greptime Team +// +// Licensed 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 crate::etl::transform::{Transformer, Transforms}; +use crate::etl::value::Value; + +pub struct NoopTransformer; + +impl std::fmt::Display for NoopTransformer { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + write!(f, "NoopTransformer") + } +} + +impl Transformer for NoopTransformer { + type Output = Value; + + fn new(_transforms: Transforms) -> Result { + Ok(NoopTransformer) + } + + fn transform(&self, val: Value) -> Result { + Ok(val) + } +} diff --git a/src/pipeline/src/etl/value/array.rs b/src/pipeline/src/etl/value/array.rs new file mode 100644 index 000000000000..617d9beed348 --- /dev/null +++ b/src/pipeline/src/etl/value/array.rs @@ -0,0 +1,56 @@ +// Copyright 2024 Greptime Team +// +// Licensed 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 crate::etl::value::Value; + +#[derive(Debug, Clone, PartialEq, Default)] +pub struct Array { + pub values: Vec, +} + +impl Array { + pub fn new() -> Self { + Array { values: vec![] } + } +} + +impl std::fmt::Display for Array { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + let values = self + .values + .iter() + .map(|v| v.to_string()) + .collect::>() + .join(", "); + write!(f, "[{}]", values) + } +} + +impl std::ops::Deref for Array { + type Target = Vec; + + fn deref(&self) -> &Self::Target { + &self.values + } +} + +impl IntoIterator for Array { + type Item = Value; + + type IntoIter = std::vec::IntoIter; + + fn into_iter(self) -> Self::IntoIter { + self.values.into_iter() + } +} diff --git a/src/pipeline/src/etl/value/map.rs b/src/pipeline/src/etl/value/map.rs new file mode 100644 index 000000000000..3b03ab311fb0 --- /dev/null +++ b/src/pipeline/src/etl/value/map.rs @@ -0,0 +1,58 @@ +// Copyright 2024 Greptime Team +// +// Licensed 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 std::collections::HashMap; + +use crate::etl::value::Value; + +#[derive(Debug, Clone, PartialEq, Default)] +pub struct Map { + pub values: HashMap, +} + +impl Map { + pub fn one(key: impl Into, value: Value) -> Map { + let mut map = Map::default(); + map.insert(key, value); + map + } + + pub fn insert(&mut self, key: impl Into, value: Value) { + self.values.insert(key.into(), value); + } + + pub fn extend(&mut self, Map { values }: Map) { + self.values.extend(values); + } +} + +impl std::ops::Deref for Map { + type Target = HashMap; + + fn deref(&self) -> &Self::Target { + &self.values + } +} + +impl std::fmt::Display for Map { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + let values = self + .values + .iter() + .map(|(k, v)| format!("{}: {}", k, v)) + .collect::>() + .join(", "); + write!(f, "{{{}}}", values) + } +} diff --git a/src/pipeline/src/etl/value/mod.rs b/src/pipeline/src/etl/value/mod.rs new file mode 100644 index 000000000000..a9d7c34feba1 --- /dev/null +++ b/src/pipeline/src/etl/value/mod.rs @@ -0,0 +1,303 @@ +// Copyright 2024 Greptime Team +// +// Licensed 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. + +pub mod array; +pub mod map; +pub mod time; + +pub use std::collections::HashMap; + +pub use array::Array; +pub use map::Map; +pub use time::{Epoch, Time}; + +/// Value can be used as type +/// acts as value: the enclosed value is the actual value +/// acts as type: the enclosed value is the default value +#[derive(Debug, Clone, PartialEq)] +pub enum Value { + // as value: null + // as type: no type specified + Null, + + Int8(i8), + Int16(i16), + Int32(i32), + Int64(i64), + + Uint8(u8), + Uint16(u16), + Uint32(u32), + Uint64(u64), + + Float32(f32), + Float64(f64), + + Boolean(bool), + String(String), + + Time(Time), + Epoch(Epoch), + + Array(Array), + Map(Map), +} + +impl Value { + pub fn is_null(&self) -> bool { + matches!(self, Value::Null) + } + + pub fn parse_str_type(t: &str) -> Result { + let mut parts = t.splitn(2, ','); + let head = parts.next().unwrap_or_default(); + let tail = parts.next().map(|s| s.trim().to_string()); + match head.to_lowercase().as_str() { + "int8" => Ok(Value::Int8(0)), + "int16" => Ok(Value::Int16(0)), + "int32" => Ok(Value::Int32(0)), + "int64" => Ok(Value::Int64(0)), + + "uint8" => Ok(Value::Uint8(0)), + "uint16" => Ok(Value::Uint16(0)), + "uint32" => Ok(Value::Uint32(0)), + "uint64" => Ok(Value::Uint64(0)), + + "float32" => Ok(Value::Float32(0.0)), + "float64" => Ok(Value::Float64(0.0)), + + "boolean" => Ok(Value::Boolean(false)), + "string" => Ok(Value::String("".to_string())), + + "time" => Ok(Value::Time(Time::default())), + "epoch" => match tail { + Some(resolution) if !resolution.is_empty() => match resolution.as_str() { + time::NANOSECOND_RESOLUTION | time::NANO_RESOLUTION | time::NS_RESOLUTION => { + Ok(Value::Epoch(Epoch::Nanosecond(0))) + } + time::MICROSECOND_RESOLUTION | time::MICRO_RESOLUTION | time::US_RESOLUTION => { + Ok(Value::Epoch(Epoch::Microsecond(0))) + } + time::MILLISECOND_RESOLUTION | time::MILLI_RESOLUTION | time::MS_RESOLUTION => { + Ok(Value::Epoch(Epoch::Millisecond(0))) + } + time::SECOND_RESOLUTION | time::SEC_RESOLUTION | time::S_RESOLUTION => { + Ok(Value::Epoch(Epoch::Second(0))) + } + _ => Err(format!( + "invalid resolution: '{resolution}'. Available resolutions: {}", + time::VALID_RESOLUTIONS.join(",") + )), + }, + _ => Err(format!( + "resolution MUST BE set for epoch type: '{t}'. Available resolutions: {}", + time::VALID_RESOLUTIONS.join(", ") + )), + }, + + "array" => Ok(Value::Array(Array::default())), + "map" => Ok(Value::Map(Map::default())), + + _ => Err(format!("failed to parse type: '{t}'")), + } + } + + /// only support string, bool, number, null + pub fn parse_str_value(&self, v: &str) -> Result { + match self { + Value::Int8(_) => v + .parse::() + .map(Value::Int8) + .map_err(|e| format!("failed to parse int8: {}", e)), + Value::Int16(_) => v + .parse::() + .map(Value::Int16) + .map_err(|e| format!("failed to parse int16: {}", e)), + Value::Int32(_) => v + .parse::() + .map(Value::Int32) + .map_err(|e| format!("failed to parse int32: {}", e)), + Value::Int64(_) => v + .parse::() + .map(Value::Int64) + .map_err(|e| format!("failed to parse int64: {}", e)), + + Value::Uint8(_) => v + .parse::() + .map(Value::Uint8) + .map_err(|e| format!("failed to parse uint8: {}", e)), + Value::Uint16(_) => v + .parse::() + .map(Value::Uint16) + .map_err(|e| format!("failed to parse uint16: {}", e)), + Value::Uint32(_) => v + .parse::() + .map(Value::Uint32) + .map_err(|e| format!("failed to parse uint32: {}", e)), + Value::Uint64(_) => v + .parse::() + .map(Value::Uint64) + .map_err(|e| format!("failed to parse uint64: {}", e)), + + Value::Float32(_) => v + .parse::() + .map(Value::Float32) + .map_err(|e| format!("failed to parse float32: {}", e)), + Value::Float64(_) => v + .parse::() + .map(Value::Float64) + .map_err(|e| format!("failed to parse float64: {}", e)), + + Value::Boolean(_) => v + .parse::() + .map(Value::Boolean) + .map_err(|e| format!("failed to parse bool: {}", e)), + Value::String(_) => Ok(Value::String(v.to_string())), + + Value::Null => Ok(Value::Null), + + _ => Err(format!("default value not unsupported for type {}", self)), + } + } + + /// only support string, bool, number, null + pub fn to_str_value(&self) -> String { + match self { + Value::Int8(v) => format!("{}", v), + Value::Int16(v) => format!("{}", v), + Value::Int32(v) => format!("{}", v), + Value::Int64(v) => format!("{}", v), + + Value::Uint8(v) => format!("{}", v), + Value::Uint16(v) => format!("{}", v), + Value::Uint32(v) => format!("{}", v), + Value::Uint64(v) => format!("{}", v), + + Value::Float32(v) => format!("{}", v), + Value::Float64(v) => format!("{}", v), + + Value::Boolean(v) => format!("{}", v), + Value::String(v) => v.to_string(), + + v => v.to_string(), + } + } +} + +impl std::fmt::Display for Value { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + let str = match self { + Value::Null => "null".to_string(), + + Value::Int8(v) => format!("int8({})", v), + Value::Int16(v) => format!("int16({})", v), + Value::Int32(v) => format!("int32({})", v), + Value::Int64(v) => format!("int64({})", v), + + Value::Uint8(v) => format!("uint8({})", v), + Value::Uint16(v) => format!("uint16({})", v), + Value::Uint32(v) => format!("uint32({})", v), + Value::Uint64(v) => format!("uint64({})", v), + + Value::Float32(v) => format!("float32({})", v), + Value::Float64(v) => format!("float64({})", v), + + Value::Boolean(v) => format!("boolean({})", v), + Value::String(v) => format!("string({})", v), + + Value::Time(v) => format!("time({})", v), + Value::Epoch(v) => format!("epoch({})", v), + + Value::Array(v) => format!("{}", v), + Value::Map(v) => format!("{}", v), + }; + + write!(f, "{}", str) + } +} + +impl TryFrom for Value { + type Error = String; + + fn try_from(v: serde_json::Value) -> Result { + match v { + serde_json::Value::Null => Ok(Value::Null), + serde_json::Value::Bool(v) => Ok(Value::Boolean(v)), + serde_json::Value::Number(v) => { + if let Some(v) = v.as_i64() { + Ok(Value::Int64(v)) + } else if let Some(v) = v.as_u64() { + Ok(Value::Uint64(v)) + } else if let Some(v) = v.as_f64() { + Ok(Value::Float64(v)) + } else { + Err(format!("unsupported number type: {}", v)) + } + } + serde_json::Value::String(v) => Ok(Value::String(v)), + serde_json::Value::Array(v) => { + let mut values = vec![]; + for v in v { + values.push(Value::try_from(v)?); + } + Ok(Value::Array(Array { values })) + } + serde_json::Value::Object(v) => { + let mut values = HashMap::new(); + for (k, v) in v { + values.insert(k, Value::try_from(v)?); + } + Ok(Value::Map(Map { values })) + } + } + } +} + +impl TryFrom<&yaml_rust::Yaml> for Value { + type Error = String; + + fn try_from(v: &yaml_rust::Yaml) -> Result { + match v { + yaml_rust::Yaml::Null => Ok(Value::Null), + yaml_rust::Yaml::Boolean(v) => Ok(Value::Boolean(*v)), + yaml_rust::Yaml::Integer(v) => Ok(Value::Int64(*v)), + yaml_rust::Yaml::Real(v) => { + if let Ok(v) = v.parse() { + Ok(Value::Float64(v)) + } else { + Err(format!("failed to parse float64: {}", v)) + } + } + yaml_rust::Yaml::String(v) => Ok(Value::String(v.to_string())), + yaml_rust::Yaml::Array(arr) => { + let mut values = vec![]; + for v in arr { + values.push(Value::try_from(v)?); + } + Ok(Value::Array(Array { values })) + } + yaml_rust::Yaml::Hash(v) => { + let mut values = HashMap::new(); + for (k, v) in v { + let key = k + .as_str() + .ok_or(format!("key in Hash must be a string, but got {v:?}"))?; + values.insert(key.to_string(), Value::try_from(v)?); + } + Ok(Value::Map(Map { values })) + } + _ => Err(format!("unsupported yaml type: {v:?}")), + } + } +} diff --git a/src/pipeline/src/etl/value/time.rs b/src/pipeline/src/etl/value/time.rs new file mode 100644 index 000000000000..5dbc4d4e092e --- /dev/null +++ b/src/pipeline/src/etl/value/time.rs @@ -0,0 +1,187 @@ +// Copyright 2024 Greptime Team +// +// Licensed 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 common_telemetry::error; + +#[derive(Debug, Clone, PartialEq)] +pub struct Time { + pub value: String, + pub nanosecond: i64, + pub format: Option, + pub timezone: Option, + // TODO(yuanbohan): support locale + // pub locale: Option, +} + +impl Time { + pub(crate) fn new(v: impl Into, nanosecond: i64) -> Self { + let value = v.into(); + Time { + value, + nanosecond, + format: None, + timezone: None, + } + } + + pub(crate) fn with_format(&mut self, format: impl Into) { + self.format = Some(format.into()); + } + + pub(crate) fn with_timezone(&mut self, timezone: Option) { + self.timezone = timezone; + } + + pub(crate) fn timestamp_nanos(&self) -> i64 { + self.nanosecond + } + + pub(crate) fn timestamp_micros(&self) -> i64 { + self.nanosecond / 1_000 + } + + pub(crate) fn timestamp_millis(&self) -> i64 { + self.nanosecond / 1_000_000 + } + + pub(crate) fn timestamp(&self) -> i64 { + self.nanosecond / 1_000_000_000 + } +} + +impl Default for Time { + fn default() -> Self { + let dt = chrono::Utc::now(); + let v = dt.to_rfc3339(); + let ns = match dt.timestamp_nanos_opt() { + Some(ns) => ns, + None => { + error!("failed to get nanosecond from timestamp, use 0 instead"); + 0 + } + }; + Time::new(v, ns) + } +} + +impl std::fmt::Display for Time { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + let format = if let Some(format) = &self.format { + format!(", format: {}", format) + } else { + "".to_string() + }; + + let timezone = if let Some(timezone) = &self.timezone { + format!(", timezone: {}", timezone) + } else { + "".to_string() + }; + + write!(f, "{}, format: {}{}", self.value, format, timezone) + } +} + +#[derive(Debug, Clone, PartialEq)] +pub enum Epoch { + Nanosecond(i64), + Microsecond(i64), + Millisecond(i64), + Second(i64), +} + +pub(crate) const NANOSECOND_RESOLUTION: &str = "nanosecond"; +pub(crate) const NANO_RESOLUTION: &str = "nano"; +pub(crate) const NS_RESOLUTION: &str = "ns"; +pub(crate) const MICROSECOND_RESOLUTION: &str = "microsecond"; +pub(crate) const MICRO_RESOLUTION: &str = "micro"; +pub(crate) const US_RESOLUTION: &str = "us"; +pub(crate) const MILLISECOND_RESOLUTION: &str = "millisecond"; +pub(crate) const MILLI_RESOLUTION: &str = "milli"; +pub(crate) const MS_RESOLUTION: &str = "ms"; +pub(crate) const SECOND_RESOLUTION: &str = "second"; +pub(crate) const SEC_RESOLUTION: &str = "sec"; +pub(crate) const S_RESOLUTION: &str = "s"; + +pub(crate) const VALID_RESOLUTIONS: [&str; 12] = [ + NANOSECOND_RESOLUTION, + NANO_RESOLUTION, + NS_RESOLUTION, + MICROSECOND_RESOLUTION, + MICRO_RESOLUTION, + US_RESOLUTION, + MILLISECOND_RESOLUTION, + MILLI_RESOLUTION, + MS_RESOLUTION, + SECOND_RESOLUTION, + SEC_RESOLUTION, + S_RESOLUTION, +]; + +impl Epoch { + pub(crate) fn timestamp_nanos(&self) -> i64 { + match self { + Epoch::Nanosecond(v) => *v, + Epoch::Microsecond(v) => *v * 1_000, + Epoch::Millisecond(v) => *v * 1_000_000, + Epoch::Second(v) => *v * 1_000_000_000, + } + } + + pub(crate) fn timestamp_micros(&self) -> i64 { + match self { + Epoch::Nanosecond(v) => *v / 1_000, + Epoch::Microsecond(v) => *v, + Epoch::Millisecond(v) => *v * 1_000, + Epoch::Second(v) => *v * 1_000_000, + } + } + + pub(crate) fn timestamp_millis(&self) -> i64 { + match self { + Epoch::Nanosecond(v) => *v / 1_000_000, + Epoch::Microsecond(v) => *v / 1_000, + Epoch::Millisecond(v) => *v, + Epoch::Second(v) => *v * 1_000, + } + } + + pub(crate) fn timestamp(&self) -> i64 { + match self { + Epoch::Nanosecond(v) => *v / 1_000_000_000, + Epoch::Microsecond(v) => *v / 1_000_000, + Epoch::Millisecond(v) => *v / 1_000, + Epoch::Second(v) => *v, + } + } +} + +impl Default for Epoch { + fn default() -> Self { + Epoch::Nanosecond(chrono::Utc::now().timestamp_nanos_opt().unwrap_or(0)) + } +} + +impl std::fmt::Display for Epoch { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + let (value, resolution) = match self { + Epoch::Nanosecond(v) => (v, NANOSECOND_RESOLUTION), + Epoch::Microsecond(v) => (v, MICROSECOND_RESOLUTION), + Epoch::Millisecond(v) => (v, MILLISECOND_RESOLUTION), + Epoch::Second(v) => (v, SECOND_RESOLUTION), + }; + + write!(f, "{}, resolution: {}", value, resolution) + } +} diff --git a/src/pipeline/src/lib.rs b/src/pipeline/src/lib.rs index f5acf8d86af1..57f7bf903a4a 100644 --- a/src/pipeline/src/lib.rs +++ b/src/pipeline/src/lib.rs @@ -1,6 +1,7 @@ -pub mod error; -pub mod table; +mod etl; +mod mng; -pub use pipeline::transform::GreptimeTransformer; -pub use pipeline::value::Value; -pub use pipeline::Pipeline; +pub use etl::transform::GreptimeTransformer; +pub use etl::value::Value; +pub use etl::{parse, Content, Pipeline}; +pub use mng::{error, table}; diff --git a/src/pipeline/src/error.rs b/src/pipeline/src/mng/error.rs similarity index 100% rename from src/pipeline/src/error.rs rename to src/pipeline/src/mng/error.rs diff --git a/src/pipeline/src/mng/mod.rs b/src/pipeline/src/mng/mod.rs new file mode 100644 index 000000000000..91ca63d5bac0 --- /dev/null +++ b/src/pipeline/src/mng/mod.rs @@ -0,0 +1,2 @@ +pub mod error; +pub mod table; diff --git a/src/pipeline/src/table.rs b/src/pipeline/src/mng/table.rs similarity index 99% rename from src/pipeline/src/table.rs rename to src/pipeline/src/mng/table.rs index 681301c13650..34dbc81bb4f5 100644 --- a/src/pipeline/src/table.rs +++ b/src/pipeline/src/mng/table.rs @@ -20,8 +20,6 @@ use datatypes::prelude::ScalarVector; use datatypes::vectors::{StringVector, Vector}; use operator::insert::InserterRef; use operator::statement::StatementExecutorRef; -use pipeline::transform::GreptimeTransformer; -use pipeline::{parse, Content, Pipeline}; use query::plan::LogicalPlan; use query::QueryEngineRef; use session::context::{QueryContextBuilder, QueryContextRef}; @@ -34,6 +32,8 @@ use crate::error::{ BuildDfLogicalPlanSnafu, CastTypeSnafu, CollectRecordsSnafu, ExecuteInternalStatementSnafu, InsertPipelineSnafu, ParsePipelineSnafu, PipelineNotFoundSnafu, Result, }; +use crate::etl::transform::GreptimeTransformer; +use crate::etl::{parse, Content, Pipeline}; pub type PipelineTableRef = Arc; diff --git a/src/pipeline/tests/pipeline.rs b/src/pipeline/tests/pipeline.rs new file mode 100644 index 000000000000..0a597f188fd9 --- /dev/null +++ b/src/pipeline/tests/pipeline.rs @@ -0,0 +1,461 @@ +// Copyright 2024 Greptime Team +// +// Licensed 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 common_telemetry::tracing::info; +use greptime_proto::v1::value::ValueData::{ + BoolValue, F64Value, StringValue, TimestampSecondValue, U32Value, U64Value, U8Value, +}; +use greptime_proto::v1::Value as GreptimeValue; +use pipeline::{parse, Content, GreptimeTransformer, Pipeline, Value}; + +// use pipeline::transform::GreptimeTransformer; +// use pipeline::value::Value; +// use pipeline::{parse, Content, Pipeline}; + +#[test] +fn main() { + let input_value_str = r#" + [ + { + "version": 1, + "streamId": "12345", + "cp": "123456", + "reqId": "1239f220", + "reqTimeSec": "1573840000", + "bytes": "4995", + "cliIP": "128.147.28.68", + "statusCode": "206", + "proto": "HTTPS", + "reqHost": "test.hostname.net", + "reqMethod": "GET", + "reqPath": "/path1/path2/file.ext", + "reqPort": "443", + "rspContentLen": "5000", + "rspContentType": "text/html", + "UA": "Mozilla%2F5.0+%28Macintosh%3B+Intel+Mac+OS+X+10_14_3%29", + "tlsOverheadTimeMSec": "0", + "tlsVersion": "TLSv1", + "objSize": "484", + "uncompressedSize": "484", + "overheadBytes": "232", + "totalBytes": "0", + "queryStr": "cmcd=//1.0@V/bl=21600,br=1426,cid=%22akam-email%22,d=6006,mtp=11100,ot=m,sf=h,sid=%229f36f5c9-d6a2-497b-8c73-4b8f694eab749f36f5c9-d6a2-497b-8c73%22,tb=1426,dl=18500,nor=%22../300kbps/track.m4v%22,nrr=%2212323-48763%22,su,bs,rtp=12000,pr=1.08,sf=d,st=v%22", + "breadcrumbs": "//BC/%5Ba=23.33.41.20,c=g,k=0,l=1%5D", + "accLang": "en-US", + "cookie": "cookie-content", + "range": "37334-42356", + "referer": "https%3A%2F%2Ftest.referrer.net%2Fen-US%2Fdocs%2FWeb%2Ftest", + "xForwardedFor": "8.47.28.38", + "maxAgeSec": "3600", + "reqEndTimeMSec": "3", + "errorCode": "ERR_ACCESS_DENIED|fwd_acl", + "turnAroundTimeMSec": "11", + "transferTimeMSec": "125", + "dnsLookupTimeMSec": "50", + "lastByte": "1", + "edgeIP": "23.50.51.173", + "country": "IN", + "state": "Virginia", + "city": "HERNDON", + "serverCountry": "SG", + "billingRegion": "8", + "cacheStatus": "1", + "securityRules": "ULnR_28976|3900000:3900001:3900005:3900006:BOT-ANOMALY-HEADER|", + "ewUsageInfo": "//4380/4.0/1/-/0/4/#1,2\\//4380/4.0/4/-/0/4/#0,0\\//4380/4.0/5/-/1/1/#0,0", + "ewExecutionInfo": "c:4380:7:161:162:161:n:::12473:200|C:4380:3:0:4:0:n:::6967:200|R:4380:20:99:99:1:n:::35982:200", + "customField": "any-custom-value" + } + ] +"#; + let input_value: Value = serde_json::from_str::(input_value_str) + .expect("failed to parse input value") + .try_into() + .expect("failed to convert input value"); + + let pipeline_yaml = r#" +--- +description: Pipeline for Akamai DataStream2 Log + +processors: + - urlencoding: + fields: + - breadcrumbs + - UA + - referer + - queryStr + method: decode + ignore_missing: true + - epoch: + field: reqTimeSec + resolution: second + ignore_missing: true + - regex: + field: breadcrumbs + patterns: + - "(?\\[[^\\[]*c=c[^\\]]*\\])" + - "(?\\[[^\\[]*c=g[^\\]]*\\])" + - "(?\\[[^\\[]*c=o[^\\]]*\\])" + - "(?\\[[^\\[]*c=p[^\\]]*\\])" + - "(?\\[[^\\[]*c=w[^\\]]*\\])" + ignore_missing: true + - regex: + fields: + - breadcrumbs_parent + - breadcrumbs_edge + - breadcrumbs_origin + - breadcrumbs_peer + - breadcrumbs_cloud_wrapper + ignore_missing: true + patterns: + - "a=(?[^,\\]]+)" + - "b=(?[^,\\]]+)" + - "k=(?[^,\\]]+)" + - "l=(?[^,\\]]+)" + - "m=(?[^,\\]]+)" + - "n=(?[^,\\]]+)" + - "o=(?[^,\\]]+)" + - regex: + field: queryStr, cmcd + patterns: + - "(?i)CMCD=//(?[\\d\\.]+)@V/(?.+$)" + ignore_missing: true + - cmcd: + field: cmcd_data, cmcd + ignore_missing: true + +transform: + - fields: + - breadcrumbs + - referer + - queryStr, query_str + - customField, custom_field + - reqId, req_id + - city + - state + - country + - securityRules, security_rules + - ewUsageInfo, ew_usage_info + - ewExecutionInfo, ew_execution_info + - errorCode, error_code + - xForwardedFor, x_forwarded_for + - range + - accLang, acc_lang + - reqMethod, req_method + - reqHost, req_host + - proto + - cliIP, cli_ip + - rspContentType, rsp_content_type + - tlsVersion, tls_version + type: string + - fields: + - version + - cacheStatus, cache_status + - lastByte, last_byte + type: uint8 + - fields: + - streamId, stream_id + - billingRegion, billing_region + - dnsLookupTimeMSec, dns_lookup_time_msec + - transferTimeMSec, transfer_time_msec + - turnAroundTimeMSec, turn_around_time_msec + - reqEndTimeMSec, req_end_time_msec + - maxAgeSec, max_age_sec + - reqPort, req_port + - statusCode, status_code + - cp + - tlsOverheadTimeMSec, tls_overhead_time_msec + type: uint32 + - fields: + - bytes + - rspContentLen, rsp_content_len + - objSize, obj_size + - uncompressedSize, uncompressed_size + - overheadBytes, overhead_bytes + - totalBytes, total_bytes + type: uint64 + - fields: + - UA, user_agent + - cookie + - reqPath, req_path + type: string + # index: fulltext + - field: reqTimeSec, req_time_sec + # epoch time is special, the resolution MUST BE specified + type: epoch, second + index: timestamp + + # the following is from cmcd + - fields: + - cmcd_version + - cmcd_cid, cmcd_content_id + - cmcd_nor, cmcd_next_object_requests + - cmcd_nrr, cmcd_next_range_request + - cmcd_ot, cmcd_object_type + - cmcd_sf, cmcd_streaming_format + - cmcd_sid, cmcd_session_id + - cmcd_st, cmcd_stream_type + - cmcd_v + type: string + - fields: + - cmcd_br, cmcd_encoded_bitrate + - cmcd_bl, cmcd_buffer_length + - cmcd_d, cmcd_object_duration + - cmcd_dl, cmcd_deadline + - cmcd_mtp, cmcd_measured_throughput + - cmcd_rtp, cmcd_requested_max_throughput + - cmcd_tb, cmcd_top_bitrate + type: uint64 + - fields: + - cmcd_pr, cmcd_playback_rate + type: float64 + - fields: + - cmcd_bs, cmcd_buffer_starvation + - cmcd_su, cmcd_startup + type: boolean + + # the following is from breadcrumbs + - fields: + - breadcrumbs_parent_ip + - breadcrumbs_parent_request_id + - breadcrumbs_parent_geo + - breadcrumbs_edge_ip + - breadcrumbs_edge_request_id + - breadcrumbs_edge_geo + - breadcrumbs_origin_ip + - breadcrumbs_origin_request_id + - breadcrumbs_origin_geo + - breadcrumbs_peer_ip + - breadcrumbs_peer_request_id + - breadcrumbs_peer_geo + - breadcrumbs_cloud_wrapper_ip + - breadcrumbs_cloud_wrapper_request_id + - breadcrumbs_cloud_wrapper_geo + type: string + - fields: + - breadcrumbs_parent_request_end_time + - breadcrumbs_parent_turn_around_time + - breadcrumbs_parent_dns_lookup_time + - breadcrumbs_parent_asn + - breadcrumbs_edge_request_end_time + - breadcrumbs_edge_turn_around_time + - breadcrumbs_edge_dns_lookup_time + - breadcrumbs_edge_asn + - breadcrumbs_origin_request_end_time + - breadcrumbs_origin_turn_around_time + - breadcrumbs_origin_dns_lookup_time + - breadcrumbs_origin_asn + - breadcrumbs_peer_request_end_time + - breadcrumbs_peer_turn_around_time + - breadcrumbs_peer_dns_lookup_time + - breadcrumbs_peer_asn + - breadcrumbs_cloud_wrapper_request_end_time + - breadcrumbs_cloud_wrapper_turn_around_time + - breadcrumbs_cloud_wrapper_dns_lookup_time + - breadcrumbs_cloud_wrapper_asn + type: uint32 +"#; + + let expected_values = vec![ + ( + "breadcrumbs", + Some(StringValue("//BC/[a=23.33.41.20,c=g,k=0,l=1]".into())), + ), + ( + "referer", + Some(StringValue( + "https://test.referrer.net/en-US/docs/Web/test".into(), + )), + ), + ( + "query_str", + Some(StringValue("cmcd=//1.0@V/bl=21600,br=1426,cid=\"akam-email\",d=6006,mtp=11100,ot=m,sf=h,sid=\"9f36f5c9-d6a2-497b-8c73-4b8f694eab749f36f5c9-d6a2-497b-8c73\",tb=1426,dl=18500,nor=\"../300kbps/track.m4v\",nrr=\"12323-48763\",su,bs,rtp=12000,pr=1.08,sf=d,st=v\"".into())), + ), + ("custom_field", Some(StringValue("any-custom-value".into()))), + ("req_id", Some(StringValue("1239f220".into()))), + ("city", Some(StringValue("HERNDON".into()))), + ("state", Some(StringValue("Virginia".into()))), + ("country", Some(StringValue("IN".into()))), + ( + "security_rules", + Some(StringValue( + "ULnR_28976|3900000:3900001:3900005:3900006:BOT-ANOMALY-HEADER|".into(), + )), + ), + ( + "ew_usage_info", + Some(StringValue( + "//4380/4.0/1/-/0/4/#1,2\\//4380/4.0/4/-/0/4/#0,0\\//4380/4.0/5/-/1/1/#0,0".into(), + )), + ), + ( + "ew_execution_info", + Some(StringValue("c:4380:7:161:162:161:n:::12473:200|C:4380:3:0:4:0:n:::6967:200|R:4380:20:99:99:1:n:::35982:200".into()))), + ( + "error_code", + Some(StringValue("ERR_ACCESS_DENIED|fwd_acl".into())), + ), + ("x_forwarded_for", Some(StringValue("8.47.28.38".into()))), + ("range", Some(StringValue("37334-42356".into()))), + ("acc_lang", Some(StringValue("en-US".into()))), + ("req_method", Some(StringValue("GET".into()))), + ("req_host", Some(StringValue("test.hostname.net".into()))), + ("proto", Some(StringValue("HTTPS".into()))), + ("cli_ip", Some(StringValue("128.147.28.68".into()))), + ("rsp_content_type", Some(StringValue("text/html".into()))), + ("tls_version", Some(StringValue("TLSv1".into()))), + ("version", Some(U8Value(1))), + ("cache_status", Some(U8Value(1))), + ("last_byte", Some(U8Value(1))), + ("stream_id", Some(U32Value(12345))), + ("billing_region", Some(U32Value(8))), + ("dns_lookup_time_msec", Some(U32Value(50))), + ("transfer_time_msec", Some(U32Value(125))), + ("turn_around_time_msec", Some(U32Value(11))), + ("req_end_time_msec", Some(U32Value(3))), + ("max_age_sec", Some(U32Value(3600))), + ("req_port", Some(U32Value(443))), + ("status_code", Some(U32Value(206))), + ("cp", Some(U32Value(123456))), + ("tls_overhead_time_msec", Some(U32Value(0))), + ("bytes", Some(U64Value(4995))), + ("rsp_content_len", Some(U64Value(5000))), + ("obj_size", Some(U64Value(484))), + ("uncompressed_size", Some(U64Value(484))), + ("overhead_bytes", Some(U64Value(232))), + ("total_bytes", Some(U64Value(0))), + ( + "user_agent", + Some(StringValue( + "Mozilla/5.0+(Macintosh;+Intel+Mac+OS+X+10_14_3)".into(), + )), + ), + ("cookie", Some(StringValue("cookie-content".into()))), + ( + "req_path", + Some(StringValue("/path1/path2/file.ext".into())), + ), + ("req_time_sec", Some(TimestampSecondValue(1573840000))), + ("cmcd_version", Some(StringValue("1.0".into()))), + ( + "cmcd_content_id", + Some(StringValue("\"akam-email\"".into())), + ), + ( + "cmcd_next_object_requests", + Some(StringValue("\"../300kbps/track.m4v\"".into())), + ), + ( + "cmcd_next_range_request", + Some(StringValue("\"12323-48763\"".into())), + ), + ("cmcd_object_type", Some(StringValue("m".into()))), + ("cmcd_streaming_format", Some(StringValue("d".into()))), + ( + "cmcd_session_id", + Some(StringValue( + "\"9f36f5c9-d6a2-497b-8c73-4b8f694eab749f36f5c9-d6a2-497b-8c73\"".into(), + )), + ), + ("cmcd_stream_type", Some(StringValue("v\"".into()))), + ("cmcd_v", None), + ("cmcd_encoded_bitrate", Some(U64Value(1426))), + ("cmcd_buffer_length", Some(U64Value(21600))), + ("cmcd_object_duration", Some(U64Value(6006))), + ("cmcd_deadline", Some(U64Value(18500))), + ("cmcd_measured_throughput", Some(U64Value(11100))), + ("cmcd_requested_max_throughput", Some(U64Value(12000))), + ("cmcd_top_bitrate", Some(U64Value(1426))), + ("cmcd_playback_rate", Some(F64Value(1.08))), + ("cmcd_buffer_starvation", Some(BoolValue(true))), + ("cmcd_startup", Some(BoolValue(true))), + ("breadcrumbs_parent_ip", None), + ("breadcrumbs_parent_request_id", None), + ("breadcrumbs_parent_geo", None), + ( + "breadcrumbs_edge_ip", + Some(StringValue("23.33.41.20".into())), + ), + ("breadcrumbs_edge_request_id", None), + ("breadcrumbs_edge_geo", None), + ("breadcrumbs_origin_ip", None), + ("breadcrumbs_origin_request_id", None), + ("breadcrumbs_origin_geo", None), + ("breadcrumbs_peer_ip", None), + ("breadcrumbs_peer_request_id", None), + ("breadcrumbs_peer_geo", None), + ("breadcrumbs_cloud_wrapper_ip", None), + ("breadcrumbs_cloud_wrapper_request_id", None), + ("breadcrumbs_cloud_wrapper_geo", None), + ("breadcrumbs_parent_request_end_time", None), + ("breadcrumbs_parent_turn_around_time", None), + ("breadcrumbs_parent_dns_lookup_time", None), + ("breadcrumbs_parent_asn", None), + ("breadcrumbs_edge_request_end_time", Some(U32Value(0))), + ("breadcrumbs_edge_turn_around_time", Some(U32Value(1))), + ("breadcrumbs_edge_dns_lookup_time", None), + ("breadcrumbs_edge_asn", None), + ("breadcrumbs_origin_request_end_time", None), + ("breadcrumbs_origin_turn_around_time", None), + ("breadcrumbs_origin_dns_lookup_time", None), + ("breadcrumbs_origin_asn", None), + ("breadcrumbs_peer_request_end_time", None), + ("breadcrumbs_peer_turn_around_time", None), + ("breadcrumbs_peer_dns_lookup_time", None), + ("breadcrumbs_peer_asn", None), + ("breadcrumbs_cloud_wrapper_request_end_time", None), + ("breadcrumbs_cloud_wrapper_turn_around_time", None), + ("breadcrumbs_cloud_wrapper_dns_lookup_time", None), + ("breadcrumbs_cloud_wrapper_asn", None), + ] + .into_iter() + .map(|(_, d)| GreptimeValue { value_data: d }) + .collect::>(); + + let yaml_content = Content::Yaml(pipeline_yaml.into()); + let pipeline: Pipeline = + parse(&yaml_content).expect("failed to parse pipeline"); + let output = pipeline.exec(input_value).expect("failed to exec pipeline"); + + assert_eq!(output.rows.len(), 1); + let values = output.rows.first().unwrap().values.clone(); + assert_eq!(expected_values, values); + + for s in output.schema.iter() { + info!( + "{}({}): {}", + s.column_name, + s.datatype().as_str_name(), + s.semantic_type().as_str_name() + ); + } + info!("\n"); + + let get_schema_name = |ss: &Vec, i: usize| { + let s = ss.get(i).unwrap(); + s.column_name.clone() + }; + + for row in output.rows.iter() { + let values = &row.values; + for i in 0..values.len() { + let val = values.get(i).unwrap(); + info!( + "{}: {:?}, ", + get_schema_name(&output.schema, i), + val.value_data + ); + } + info!("\n"); + } +} diff --git a/src/script/src/table.rs b/src/script/src/table.rs index 1ba160028d1d..5eacf1ff7ce9 100644 --- a/src/script/src/table.rs +++ b/src/script/src/table.rs @@ -292,7 +292,7 @@ impl ScriptsTable { } /// Build the inserted column schemas -fn build_insert_column_schemas() -> Vec { +fn build_insert_column_schemas() -> Vec { vec![ // The schema that script belongs to. PbColumnSchema { diff --git a/src/servers/src/http/handler.rs b/src/servers/src/http/handler.rs index 302c2186eb89..44c83462c845 100644 --- a/src/servers/src/http/handler.rs +++ b/src/servers/src/http/handler.rs @@ -37,9 +37,7 @@ use session::context::QueryContextRef; use snafu::ResultExt; use super::header::collect_plan_metrics; -use crate::error::{ - Error, InsertLogSnafu, ParseJsonSnafu, UnsupportedContentTypeSnafu, -}; +use crate::error::{Error, InsertLogSnafu, ParseJsonSnafu, UnsupportedContentTypeSnafu}; use crate::http::arrow_result::ArrowResponse; use crate::http::csv_result::CsvResponse; use crate::http::error_result::ErrorResponse; From eb9cd22ba04f51e3b7958bb5f6d014e40481e2f3 Mon Sep 17 00:00:00 2001 From: shuiyisong Date: Tue, 4 Jun 2024 15:36:46 +0800 Subject: [PATCH 16/51] chore: fix typo --- typos.toml | 1 + 1 file changed, 1 insertion(+) diff --git a/typos.toml b/typos.toml index 02f2ed6e695a..e03dc5685a3d 100644 --- a/typos.toml +++ b/typos.toml @@ -2,6 +2,7 @@ Pn = "Pn" ue = "ue" worl = "worl" +ot = "ot" [files] extend-exclude = [ From 8d0595c12c75100b827809803eca7eb51b3660c7 Mon Sep 17 00:00:00 2001 From: shuiyisong Date: Tue, 4 Jun 2024 15:43:14 +0800 Subject: [PATCH 17/51] refactor: bring in pipeline a95c9767d7056ab01dd8ca5fa1214456c6ffc72c --- src/pipeline/src/etl/processor/dissect.rs | 417 ++++++++++++++++++++++ src/pipeline/src/etl/processor/mod.rs | 42 ++- src/pipeline/src/etl/value/map.rs | 6 + 3 files changed, 457 insertions(+), 8 deletions(-) create mode 100644 src/pipeline/src/etl/processor/dissect.rs diff --git a/src/pipeline/src/etl/processor/dissect.rs b/src/pipeline/src/etl/processor/dissect.rs new file mode 100644 index 000000000000..06d49bbfeceb --- /dev/null +++ b/src/pipeline/src/etl/processor/dissect.rs @@ -0,0 +1,417 @@ +// Copyright 2024 Greptime Team +// +// Licensed 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 common_telemetry::warn; + +use crate::etl::field::{Field, Fields}; +use crate::etl::processor::{ + yaml_bool, yaml_field, yaml_fields, yaml_parse_strings, yaml_string, Processor, FIELDS_NAME, + FIELD_NAME, IGNORE_MISSING_NAME, PATTERNS_NAME, +}; +use crate::etl::value::{Map, Value}; + +pub(crate) const PROCESSOR_DISSECT: &str = "dissect"; + +const APPEND_SEPARATOR_NAME: &str = "append_separator"; + +#[derive(Debug, PartialEq)] +enum Part { + Split(String), + Key(String), +} + +impl Part { + fn is_empty(&self) -> bool { + match self { + Part::Split(v) => v.is_empty(), + Part::Key(v) => v.is_empty(), + } + } + + fn empty_split() -> Self { + Part::Split(String::new()) + } + + fn empty_key() -> Self { + Part::Key(String::new()) + } +} + +impl std::ops::Deref for Part { + type Target = String; + + fn deref(&self) -> &Self::Target { + match self { + Part::Split(v) => v, + Part::Key(v) => v, + } + } +} + +impl std::ops::DerefMut for Part { + fn deref_mut(&mut self) -> &mut Self::Target { + match self { + Part::Split(v) => v, + Part::Key(v) => v, + } + } +} + +#[derive(Debug, Default)] +struct Pattern { + origin: String, + parts: Vec, +} + +impl std::ops::Deref for Pattern { + type Target = Vec; + + fn deref(&self) -> &Self::Target { + &self.parts + } +} + +impl std::ops::DerefMut for Pattern { + fn deref_mut(&mut self) -> &mut Self::Target { + &mut self.parts + } +} + +impl std::str::FromStr for Pattern { + type Err = String; + + fn from_str(s: &str) -> Result { + let mut parts = vec![]; + let mut cursor = Part::empty_split(); + + let origin = s.to_string(); + let mut last_ch = None; + let chars: Vec = origin.chars().collect(); + + for i in 0..chars.len() { + let ch = chars[i]; + match (ch, &mut cursor) { + // if cursor is Split part, and found %{, then ready to start a Key part + ('%', Part::Split(_)) if i + 1 < chars.len() && chars[i + 1] == '{' => {} + // if cursor is Split part, and found %{, then end the Split part, start the Key part + ('{', Part::Split(_)) if last_ch == Some('%') => { + if !cursor.is_empty() { + parts.push(cursor); + } + + cursor = Part::empty_key(); + } + // if cursor is Split part, and not found % or {, then continue the Split part + (_, Part::Split(_)) => { + cursor.push(ch); + } + // if cursor is Key part, and found }, then end the Key part, start the next Split part + ('}', Part::Key(_)) => { + parts.push(cursor); + cursor = Part::empty_split(); + } + (_, Part::Key(_)) if !is_valid_char(ch) => { + return Err(format!("Invalid character in key: '{ch}'")); + } + (_, Part::Key(_)) => { + cursor.push(ch); + } + } + + last_ch = Some(ch); + } + + let pattern = Self { parts, origin }; + pattern.check()?; + Ok(pattern) + } +} + +impl Pattern { + fn check(&self) -> Result<(), String> { + if self.len() == 0 { + return Err("Empty pattern is not allowed".to_string()); + } + + for i in 0..self.len() { + let this_part = &self[i]; + let next_part = self.get(i + 1); + match (this_part, next_part) { + (Part::Split(split), _) if split.is_empty() => { + return Err("Empty split is not allowed".to_string()); + } + (Part::Key(key1), Some(Part::Key(key2))) => { + return Err(format!( + "consecutive keys are not allowed: '{key1}' '{key2}'" + )); + } + _ => {} + } + } + Ok(()) + } +} + +impl std::fmt::Display for Pattern { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + write!(f, "{}", self.origin) + } +} + +#[derive(Debug, Default)] +pub struct DissectProcessor { + fields: Fields, + patterns: Vec, + ignore_missing: bool, + + // The character(s) that separate the appended fields. Default is an empty string. + append_separator: Option, +} + +impl DissectProcessor { + fn with_fields(&mut self, fields: Fields) { + self.fields = fields; + } + + fn with_ignore_missing(&mut self, ignore_missing: bool) { + self.ignore_missing = ignore_missing; + } + + fn with_patterns(&mut self, patterns: Vec) { + self.patterns = patterns; + } + + fn with_append_separator(&mut self, append_separator: String) { + self.append_separator = Some(append_separator); + } + + fn process_pattern(chs: &[char], pattern: &Pattern) -> Result { + let mut map = Map::default(); + let mut pos = 0; + + for i in 0..pattern.len() { + let this_part = &pattern[i]; + let next_part = pattern.get(i + 1); + match (this_part, next_part) { + (Part::Split(split), _) => { + let split_chs = split.chars().collect::>(); + let split_len = split_chs.len(); + if pos + split_len > chs.len() { + return Err(format!("'{split}' exceeds the input",)); + } + + if &chs[pos..pos + split_len] != split_chs.as_slice() { + return Err(format!( + "'{split}' does not match the input '{}'", + chs[pos..pos + split_len].iter().collect::() + )); + } + + pos += split_len; + } + (Part::Key(key), None) => { + let value = chs[pos..].iter().collect::(); + map.insert(key.clone(), Value::String(value)); + } + + (Part::Key(key), Some(Part::Split(split))) => match split.chars().next() { + None => return Err("Empty split is not allowed".to_string()), + Some(stop) => { + let mut end = pos; + while end < chs.len() && chs[end] != stop { + end += 1; + } + + if end == chs.len() { + return Err("No matching split found".to_string()); + } + + let value = chs[pos..end].iter().collect::(); + map.insert(key.clone(), Value::String(value)); + pos = end; + } + }, + (Part::Key(key1), Some(Part::Key(key2))) => { + return Err(format!( + "consecutive keys are not allowed: '{key1}' '{key2}'" + )); + } + } + } + + Ok(map) + } + + fn process(&self, val: &str) -> Result { + let chs = val.chars().collect::>(); + + for pattern in &self.patterns { + if let Ok(map) = DissectProcessor::process_pattern(&chs, pattern) { + return Ok(map); + } + } + + Err("No matching pattern found".to_string()) + } +} + +impl TryFrom<&yaml_rust::yaml::Hash> for DissectProcessor { + type Error = String; + + fn try_from(value: &yaml_rust::yaml::Hash) -> Result { + let mut processor = Self::default(); + + for (k, v) in value.iter() { + let key = k + .as_str() + .ok_or(format!("key must be a string, but got '{k:?}'"))?; + + match key { + FIELD_NAME => processor.with_fields(Fields::one(yaml_field(v, FIELD_NAME)?)), + FIELDS_NAME => processor.with_fields(yaml_fields(v, FIELDS_NAME)?), + PATTERNS_NAME => { + let patterns = yaml_parse_strings(v, PATTERNS_NAME)?; + processor.with_patterns(patterns); + } + IGNORE_MISSING_NAME => { + processor.with_ignore_missing(yaml_bool(v, IGNORE_MISSING_NAME)?) + } + APPEND_SEPARATOR_NAME => { + processor.with_append_separator(yaml_string(v, APPEND_SEPARATOR_NAME)?) + } + _ => {} + } + } + + Ok(processor) + } +} + +impl Processor for DissectProcessor { + fn kind(&self) -> &str { + PROCESSOR_DISSECT + } + + fn ignore_missing(&self) -> bool { + self.ignore_missing + } + + fn fields(&self) -> &Fields { + &self.fields + } + + fn exec_field(&self, val: &Value, _field: &Field) -> Result { + match val { + Value::String(val) => match self.process(val) { + Ok(map) => Ok(map), + Err(e) => { + warn!("dissect processor: {}", e); + Ok(Map::default()) + } + }, + _ => Err(format!( + "{} processor: expect string value, but got {val:?}", + self.kind() + )), + } + } +} + +fn is_valid_char(ch: char) -> bool { + ch.is_alphanumeric() || ch == '_' +} + +#[cfg(test)] +mod tests { + use std::collections::HashMap; + + use super::{DissectProcessor, Part, Pattern}; + use crate::etl::value::{Map, Value}; + + #[test] + fn test_pattern() { + let cases = [( + "%{clientip} %{ident} %{auth} [%{timestamp}] \"%{verb} %{request} HTTP/%{httpversion}\" %{status} %{size}", + vec![ + Part::Key("clientip".chars().collect()), + Part::Split(" ".chars().collect()), + Part::Key("ident".chars().collect()), + Part::Split(" ".chars().collect()), + Part::Key("auth".chars().collect()), + Part::Split(" [".chars().collect()), + Part::Key("timestamp".chars().collect()), + Part::Split("] \"".chars().collect()), + Part::Key("verb".chars().collect()), + Part::Split(" ".chars().collect()), + Part::Key("request".chars().collect()), + Part::Split(" HTTP/".chars().collect()), + Part::Key("httpversion".chars().collect()), + Part::Split("\" ".chars().collect()), + Part::Key("status".chars().collect()), + Part::Split(" ".chars().collect()), + Part::Key("size".chars().collect()), + ], + )]; + + for (pattern, expected) in cases.into_iter() { + let p: Pattern = pattern.parse().unwrap(); + assert_eq!(p.parts, expected); + } + } + + #[test] + fn test_process() { + let assert = |pattern_str: &str, input: &str, expected: HashMap| { + let chs = input.chars().collect::>(); + let pattern = pattern_str.parse().unwrap(); + let map = DissectProcessor::process_pattern(&chs, &pattern).unwrap(); + + assert_eq!(map, Map::from(expected)); + }; + + let expected = [ + ("timestamp", "30/Apr/1998:22:00:52 +0000"), + ("status", "200"), + ("clientip", "1.2.3.4"), + ("ident", "-"), + ("size", "3171"), + ( + "request", + "/english/venues/cities/images/montpellier/18.gif", + ), + ("auth", "-"), + ("verb", "GET"), + ("httpversion", "1.0"), + ] + .into_iter() + .map(|(k, v)| (k.to_string(), Value::String(v.to_string()))) + .collect::>(); + + { + // pattern start with Key + let pattern_str = "%{clientip} %{ident} %{auth} [%{timestamp}] \"%{verb} %{request} HTTP/%{httpversion}\" %{status} %{size}"; + let input = "1.2.3.4 - - [30/Apr/1998:22:00:52 +0000] \"GET /english/venues/cities/images/montpellier/18.gif HTTP/1.0\" 200 3171"; + + assert(pattern_str, input, expected.clone()); + } + + { + // pattern start with Split + let pattern_str = " %{clientip} %{ident} %{auth} [%{timestamp}] \"%{verb} %{request} HTTP/%{httpversion}\" %{status} %{size}"; + let input = " 1.2.3.4 - - [30/Apr/1998:22:00:52 +0000] \"GET /english/venues/cities/images/montpellier/18.gif HTTP/1.0\" 200 3171"; + + assert(pattern_str, input, expected); + } + } +} diff --git a/src/pipeline/src/etl/processor/mod.rs b/src/pipeline/src/etl/processor/mod.rs index c04414b87cbd..81f324a43285 100644 --- a/src/pipeline/src/etl/processor/mod.rs +++ b/src/pipeline/src/etl/processor/mod.rs @@ -15,6 +15,7 @@ pub mod cmcd; pub mod csv; pub mod date; +pub mod dissect; pub mod epoch; pub mod letter; pub mod regex; @@ -26,6 +27,7 @@ use cmcd::CMCDProcessor; use common_telemetry::warn; use csv::CsvProcessor; use date::DateProcessor; +use dissect::DissectProcessor; use epoch::EpochProcessor; use letter::LetterProcessor; use regex::RegexProcessor; @@ -38,6 +40,7 @@ const FIELD_NAME: &str = "field"; const FIELDS_NAME: &str = "fields"; const IGNORE_MISSING_NAME: &str = "ignore_missing"; const METHOD_NAME: &str = "method"; +const PATTERNS_NAME: &str = "patterns"; // const IF_NAME: &str = "if"; // const IGNORE_FAILURE_NAME: &str = "ignore_failure"; @@ -53,14 +56,14 @@ pub trait Processor: std::fmt::Debug + Send + Sync + 'static { true } - fn exec_field(&self, _val: &Value, _field: &Field) -> Result { - Ok(Map::default()) + /// default behavior does nothing and returns the input value + fn exec_field(&self, val: &Value, field: &Field) -> Result { + Ok(Map::one(field.get_field(), val.clone())) } fn exec_map(&self, mut map: Map) -> Result { for ff @ Field { field, .. } in self.fields().iter() { - let val = map.get(field); - match val { + match map.get(field) { Some(v) => { map.extend(self.exec_field(v, ff)?); } @@ -158,6 +161,7 @@ fn parse_processor(doc: &yaml_rust::Yaml) -> Result, String> cmcd::PROCESSOR_CMCD => Arc::new(CMCDProcessor::try_from(value)?), csv::PROCESSOR_CSV => Arc::new(CsvProcessor::try_from(value)?), date::PROCESSOR_DATE => Arc::new(DateProcessor::try_from(value)?), + dissect::PROCESSOR_DISSECT => Arc::new(DissectProcessor::try_from(value)?), epoch::PROCESSOR_EPOCH => Arc::new(EpochProcessor::try_from(value)?), letter::PROCESSOR_LETTER => Arc::new(LetterProcessor::try_from(value)?), regex::PROCESSOR_REGEX => Arc::new(RegexProcessor::try_from(value)?), @@ -188,11 +192,33 @@ pub(crate) fn yaml_bool(v: &yaml_rust::Yaml, field: &str) -> Result Result { - yaml_string(v, field)?.parse() +pub(crate) fn yaml_parse_string(v: &yaml_rust::Yaml, field: &str) -> Result +where + T: std::str::FromStr, + T::Err: ToString, +{ + yaml_string(v, field)? + .parse::() + .map_err(|e| e.to_string()) +} + +pub(crate) fn yaml_parse_strings(v: &yaml_rust::Yaml, field: &str) -> Result, String> +where + T: std::str::FromStr, + T::Err: ToString, +{ + yaml_strings(v, field).and_then(|v| { + v.into_iter() + .map(|s| s.parse::().map_err(|e| e.to_string())) + .collect() + }) } pub(crate) fn yaml_fields(v: &yaml_rust::Yaml, field: &str) -> Result { - let ff = yaml_strings(v, field).and_then(|v| v.into_iter().map(|s| s.parse()).collect())?; - Fields::new(ff) + let v = yaml_parse_strings(v, field)?; + Fields::new(v) +} + +pub(crate) fn yaml_field(v: &yaml_rust::Yaml, field: &str) -> Result { + yaml_parse_string(v, field) } diff --git a/src/pipeline/src/etl/value/map.rs b/src/pipeline/src/etl/value/map.rs index 3b03ab311fb0..8e41c92fc0dd 100644 --- a/src/pipeline/src/etl/value/map.rs +++ b/src/pipeline/src/etl/value/map.rs @@ -37,6 +37,12 @@ impl Map { } } +impl From> for Map { + fn from(values: HashMap) -> Self { + Map { values } + } +} + impl std::ops::Deref for Map { type Target = HashMap; From 061b14e4c787261e0995ad7510efda234e2e9a8a Mon Sep 17 00:00:00 2001 From: shuiyisong Date: Tue, 4 Jun 2024 15:48:19 +0800 Subject: [PATCH 18/51] chore: fix typo and license header --- src/pipeline/src/etl/field.rs | 2 +- src/pipeline/src/etl/mod.rs | 2 +- src/pipeline/src/etl/processor/cmcd.rs | 2 +- src/pipeline/src/etl/processor/csv.rs | 2 +- src/pipeline/src/etl/processor/date.rs | 2 +- src/pipeline/src/etl/processor/dissect.rs | 2 +- src/pipeline/src/etl/processor/epoch.rs | 2 +- src/pipeline/src/etl/processor/letter.rs | 2 +- src/pipeline/src/etl/processor/mod.rs | 2 +- src/pipeline/src/etl/processor/regex.rs | 2 +- src/pipeline/src/etl/processor/urlencoding.rs | 2 +- src/pipeline/src/etl/transform/index.rs | 2 +- src/pipeline/src/etl/transform/mod.rs | 2 +- .../etl/transform/transformer/greptime/coerce.rs | 2 +- .../src/etl/transform/transformer/greptime/mod.rs | 2 +- src/pipeline/src/etl/transform/transformer/mod.rs | 2 +- src/pipeline/src/etl/transform/transformer/noop.rs | 2 +- src/pipeline/src/etl/value/array.rs | 2 +- src/pipeline/src/etl/value/map.rs | 2 +- src/pipeline/src/etl/value/mod.rs | 2 +- src/pipeline/src/etl/value/time.rs | 2 +- src/pipeline/src/lib.rs | 14 ++++++++++++++ src/pipeline/src/mng/mod.rs | 14 ++++++++++++++ src/pipeline/src/mng/table.rs | 14 ++++++++++++++ src/pipeline/tests/pipeline.rs | 2 +- src/servers/src/error.rs | 8 ++++---- src/servers/src/http/handler.rs | 2 +- 27 files changed, 69 insertions(+), 27 deletions(-) diff --git a/src/pipeline/src/etl/field.rs b/src/pipeline/src/etl/field.rs index 9d76b540953f..34181be5f4a3 100644 --- a/src/pipeline/src/etl/field.rs +++ b/src/pipeline/src/etl/field.rs @@ -1,4 +1,4 @@ -// Copyright 2024 Greptime Team +// Copyright 2023 Greptime Team // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/src/pipeline/src/etl/mod.rs b/src/pipeline/src/etl/mod.rs index 74c6cd96c547..4e4595479482 100644 --- a/src/pipeline/src/etl/mod.rs +++ b/src/pipeline/src/etl/mod.rs @@ -1,4 +1,4 @@ -// Copyright 2024 Greptime Team +// Copyright 2023 Greptime Team // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/src/pipeline/src/etl/processor/cmcd.rs b/src/pipeline/src/etl/processor/cmcd.rs index 7001ddb5b493..256d6f05da11 100644 --- a/src/pipeline/src/etl/processor/cmcd.rs +++ b/src/pipeline/src/etl/processor/cmcd.rs @@ -1,4 +1,4 @@ -// Copyright 2024 Greptime Team +// Copyright 2023 Greptime Team // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/src/pipeline/src/etl/processor/csv.rs b/src/pipeline/src/etl/processor/csv.rs index 1cd110922892..ae578d79e1f6 100644 --- a/src/pipeline/src/etl/processor/csv.rs +++ b/src/pipeline/src/etl/processor/csv.rs @@ -1,4 +1,4 @@ -// Copyright 2024 Greptime Team +// Copyright 2023 Greptime Team // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/src/pipeline/src/etl/processor/date.rs b/src/pipeline/src/etl/processor/date.rs index 6715522793a7..9c4037900a7c 100644 --- a/src/pipeline/src/etl/processor/date.rs +++ b/src/pipeline/src/etl/processor/date.rs @@ -1,4 +1,4 @@ -// Copyright 2024 Greptime Team +// Copyright 2023 Greptime Team // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/src/pipeline/src/etl/processor/dissect.rs b/src/pipeline/src/etl/processor/dissect.rs index 06d49bbfeceb..2af008ceac86 100644 --- a/src/pipeline/src/etl/processor/dissect.rs +++ b/src/pipeline/src/etl/processor/dissect.rs @@ -1,4 +1,4 @@ -// Copyright 2024 Greptime Team +// Copyright 2023 Greptime Team // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/src/pipeline/src/etl/processor/epoch.rs b/src/pipeline/src/etl/processor/epoch.rs index feee2fa8d717..96a8695c9f76 100644 --- a/src/pipeline/src/etl/processor/epoch.rs +++ b/src/pipeline/src/etl/processor/epoch.rs @@ -1,4 +1,4 @@ -// Copyright 2024 Greptime Team +// Copyright 2023 Greptime Team // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/src/pipeline/src/etl/processor/letter.rs b/src/pipeline/src/etl/processor/letter.rs index 1c2fcf9eacfc..e533536769d2 100644 --- a/src/pipeline/src/etl/processor/letter.rs +++ b/src/pipeline/src/etl/processor/letter.rs @@ -1,4 +1,4 @@ -// Copyright 2024 Greptime Team +// Copyright 2023 Greptime Team // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/src/pipeline/src/etl/processor/mod.rs b/src/pipeline/src/etl/processor/mod.rs index 81f324a43285..96e8a629f252 100644 --- a/src/pipeline/src/etl/processor/mod.rs +++ b/src/pipeline/src/etl/processor/mod.rs @@ -1,4 +1,4 @@ -// Copyright 2024 Greptime Team +// Copyright 2023 Greptime Team // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/src/pipeline/src/etl/processor/regex.rs b/src/pipeline/src/etl/processor/regex.rs index 078deef603b5..8aba43436155 100644 --- a/src/pipeline/src/etl/processor/regex.rs +++ b/src/pipeline/src/etl/processor/regex.rs @@ -1,4 +1,4 @@ -// Copyright 2024 Greptime Team +// Copyright 2023 Greptime Team // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/src/pipeline/src/etl/processor/urlencoding.rs b/src/pipeline/src/etl/processor/urlencoding.rs index f9019fd19126..c0d1669f85de 100644 --- a/src/pipeline/src/etl/processor/urlencoding.rs +++ b/src/pipeline/src/etl/processor/urlencoding.rs @@ -1,4 +1,4 @@ -// Copyright 2024 Greptime Team +// Copyright 2023 Greptime Team // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/src/pipeline/src/etl/transform/index.rs b/src/pipeline/src/etl/transform/index.rs index b554824f52a3..674df720f8c3 100644 --- a/src/pipeline/src/etl/transform/index.rs +++ b/src/pipeline/src/etl/transform/index.rs @@ -1,4 +1,4 @@ -// Copyright 2024 Greptime Team +// Copyright 2023 Greptime Team // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/src/pipeline/src/etl/transform/mod.rs b/src/pipeline/src/etl/transform/mod.rs index 1c1a0f4141de..991aa05df644 100644 --- a/src/pipeline/src/etl/transform/mod.rs +++ b/src/pipeline/src/etl/transform/mod.rs @@ -1,4 +1,4 @@ -// Copyright 2024 Greptime Team +// Copyright 2023 Greptime Team // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/src/pipeline/src/etl/transform/transformer/greptime/coerce.rs b/src/pipeline/src/etl/transform/transformer/greptime/coerce.rs index 48b612e3d53a..6b077a22dca5 100644 --- a/src/pipeline/src/etl/transform/transformer/greptime/coerce.rs +++ b/src/pipeline/src/etl/transform/transformer/greptime/coerce.rs @@ -1,4 +1,4 @@ -// Copyright 2024 Greptime Team +// Copyright 2023 Greptime Team // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/src/pipeline/src/etl/transform/transformer/greptime/mod.rs b/src/pipeline/src/etl/transform/transformer/greptime/mod.rs index 933f621d6a14..bbbfa0e9104b 100644 --- a/src/pipeline/src/etl/transform/transformer/greptime/mod.rs +++ b/src/pipeline/src/etl/transform/transformer/greptime/mod.rs @@ -1,4 +1,4 @@ -// Copyright 2024 Greptime Team +// Copyright 2023 Greptime Team // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/src/pipeline/src/etl/transform/transformer/mod.rs b/src/pipeline/src/etl/transform/transformer/mod.rs index 173aac61bfe5..87bd16b4feca 100644 --- a/src/pipeline/src/etl/transform/transformer/mod.rs +++ b/src/pipeline/src/etl/transform/transformer/mod.rs @@ -1,4 +1,4 @@ -// Copyright 2024 Greptime Team +// Copyright 2023 Greptime Team // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/src/pipeline/src/etl/transform/transformer/noop.rs b/src/pipeline/src/etl/transform/transformer/noop.rs index 6bd7a208c981..40b82d5ee760 100644 --- a/src/pipeline/src/etl/transform/transformer/noop.rs +++ b/src/pipeline/src/etl/transform/transformer/noop.rs @@ -1,4 +1,4 @@ -// Copyright 2024 Greptime Team +// Copyright 2023 Greptime Team // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/src/pipeline/src/etl/value/array.rs b/src/pipeline/src/etl/value/array.rs index 617d9beed348..a401cf00ab67 100644 --- a/src/pipeline/src/etl/value/array.rs +++ b/src/pipeline/src/etl/value/array.rs @@ -1,4 +1,4 @@ -// Copyright 2024 Greptime Team +// Copyright 2023 Greptime Team // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/src/pipeline/src/etl/value/map.rs b/src/pipeline/src/etl/value/map.rs index 8e41c92fc0dd..47041f15350d 100644 --- a/src/pipeline/src/etl/value/map.rs +++ b/src/pipeline/src/etl/value/map.rs @@ -1,4 +1,4 @@ -// Copyright 2024 Greptime Team +// Copyright 2023 Greptime Team // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/src/pipeline/src/etl/value/mod.rs b/src/pipeline/src/etl/value/mod.rs index a9d7c34feba1..a8daa5fa6149 100644 --- a/src/pipeline/src/etl/value/mod.rs +++ b/src/pipeline/src/etl/value/mod.rs @@ -1,4 +1,4 @@ -// Copyright 2024 Greptime Team +// Copyright 2023 Greptime Team // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/src/pipeline/src/etl/value/time.rs b/src/pipeline/src/etl/value/time.rs index 5dbc4d4e092e..cca883f33231 100644 --- a/src/pipeline/src/etl/value/time.rs +++ b/src/pipeline/src/etl/value/time.rs @@ -1,4 +1,4 @@ -// Copyright 2024 Greptime Team +// Copyright 2023 Greptime Team // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/src/pipeline/src/lib.rs b/src/pipeline/src/lib.rs index 57f7bf903a4a..ce2f4f6d78a2 100644 --- a/src/pipeline/src/lib.rs +++ b/src/pipeline/src/lib.rs @@ -1,3 +1,17 @@ +// Copyright 2023 Greptime Team +// +// Licensed 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. + mod etl; mod mng; diff --git a/src/pipeline/src/mng/mod.rs b/src/pipeline/src/mng/mod.rs index 91ca63d5bac0..cb4854dc181e 100644 --- a/src/pipeline/src/mng/mod.rs +++ b/src/pipeline/src/mng/mod.rs @@ -1,2 +1,16 @@ +// Copyright 2023 Greptime Team +// +// Licensed 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. + pub mod error; pub mod table; diff --git a/src/pipeline/src/mng/table.rs b/src/pipeline/src/mng/table.rs index 34dbc81bb4f5..7863982a66b0 100644 --- a/src/pipeline/src/mng/table.rs +++ b/src/pipeline/src/mng/table.rs @@ -1,3 +1,17 @@ +// Copyright 2023 Greptime Team +// +// Licensed 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 std::collections::HashMap; use std::sync::{Arc, RwLock}; diff --git a/src/pipeline/tests/pipeline.rs b/src/pipeline/tests/pipeline.rs index 0a597f188fd9..869bd13c78f3 100644 --- a/src/pipeline/tests/pipeline.rs +++ b/src/pipeline/tests/pipeline.rs @@ -1,4 +1,4 @@ -// Copyright 2024 Greptime Team +// Copyright 2023 Greptime Team // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/src/servers/src/error.rs b/src/servers/src/error.rs index f4f4e277764f..d4a849496817 100644 --- a/src/servers/src/error.rs +++ b/src/servers/src/error.rs @@ -558,13 +558,13 @@ pub enum Error { location: Location, }, - #[snafu(display("Failed to conver to structed log"))] - ToStructedLog { + #[snafu(display("Failed to convert to structured log"))] + ToStructuredLog { #[snafu(implicit)] location: Location, }, - #[snafu(display("Unsupport content type: {:?}", content_type))] + #[snafu(display("Unsupported content type: {:?}", content_type))] UnsupportedContentType { content_type: ContentType, #[snafu(implicit)] @@ -685,7 +685,7 @@ impl ErrorExt for Error { | MysqlValueConversion { .. } | UnexpectedPhysicalTable { .. } | ParseJson { .. } - | ToStructedLog { .. } + | ToStructuredLog { .. } | UnsupportedContentType { .. } | InsertLog { .. } | TimestampOverflow { .. } => StatusCode::InvalidArguments, diff --git a/src/servers/src/http/handler.rs b/src/servers/src/http/handler.rs index 44c83462c845..fd9623864e3f 100644 --- a/src/servers/src/http/handler.rs +++ b/src/servers/src/http/handler.rs @@ -79,7 +79,7 @@ pub struct LogIngesterQueryParams { } fn parse_space_separated_log(payload: String) -> Result { - // ToStructedLogSnafu + // ToStructuredLogSnafu let _log = payload.split_whitespace().collect::>(); // TODO (qtang): implement this todo!() From c152472383ab4f3a6058a1eaeac51445aa55f7e6 Mon Sep 17 00:00:00 2001 From: shuiyisong Date: Tue, 4 Jun 2024 17:27:02 +0800 Subject: [PATCH 19/51] refactor: move http event handler to a separate file --- src/frontend/src/instance/log_handler.rs | 25 ++-- src/frontend/src/pipeline.rs | 5 +- src/servers/src/http.rs | 5 +- src/servers/src/http/event.rs | 140 +++++++++++++++++++++++ src/servers/src/http/handler.rs | 112 +----------------- src/servers/src/query_handler.rs | 9 +- 6 files changed, 162 insertions(+), 134 deletions(-) create mode 100644 src/servers/src/http/event.rs diff --git a/src/frontend/src/instance/log_handler.rs b/src/frontend/src/instance/log_handler.rs index 35541314c2ef..dd34614b3ae9 100644 --- a/src/frontend/src/instance/log_handler.rs +++ b/src/frontend/src/instance/log_handler.rs @@ -18,7 +18,7 @@ use auth::{PermissionChecker, PermissionCheckerRef, PermissionReq}; use client::Output; use common_error::ext::BoxedError; use pipeline::{GreptimeTransformer, Pipeline}; -use servers::error::{AuthSnafu, ExecuteGrpcRequestSnafu}; +use servers::error::{AuthSnafu, ExecuteGrpcRequestSnafu, Result as ServerResult}; use servers::query_handler::LogHandler; use session::context::QueryContextRef; use snafu::ResultExt; @@ -31,7 +31,7 @@ impl LogHandler for Instance { &self, log: RowInsertRequests, ctx: QueryContextRef, - ) -> servers::error::Result { + ) -> ServerResult { self.plugins .get::() .as_ref() @@ -43,9 +43,9 @@ impl LogHandler for Instance { async fn get_pipeline( &self, - query_ctx: QueryContextRef, name: &str, - ) -> servers::error::Result> { + query_ctx: QueryContextRef, + ) -> ServerResult> { self.pipeline_operator .get_pipeline(query_ctx, name) .await @@ -55,24 +55,17 @@ impl LogHandler for Instance { async fn insert_pipeline( &self, - query_ctx: QueryContextRef, name: &str, content_type: &str, pipeline: &str, - ) -> servers::error::Result<()> { + query_ctx: QueryContextRef, + ) -> ServerResult<()> { self.pipeline_operator - .insert_pipeline(query_ctx, name, content_type, pipeline) + .insert_pipeline(name, content_type, pipeline, query_ctx) .await - .map_err(BoxedError::new) - .context(servers::error::InsertPipelineSnafu { name })?; - Ok(()) } - async fn delete_pipeline( - &self, - _query_ctx: QueryContextRef, - _name: &str, - ) -> servers::error::Result<()> { + async fn delete_pipeline(&self, _name: &str, _query_ctx: QueryContextRef) -> ServerResult<()> { todo!("delete_pipeline") } } @@ -82,7 +75,7 @@ impl Instance { &self, log: RowInsertRequests, ctx: QueryContextRef, - ) -> servers::error::Result { + ) -> ServerResult { self.inserter .handle_log_inserts(log, ctx, self.statement_executor.as_ref()) .await diff --git a/src/frontend/src/pipeline.rs b/src/frontend/src/pipeline.rs index 3dd107f7dcac..92fc4efb460a 100644 --- a/src/frontend/src/pipeline.rs +++ b/src/frontend/src/pipeline.rs @@ -26,6 +26,7 @@ use operator::statement::StatementExecutorRef; use pipeline::table::{PipelineTable, PipelineTableRef}; use pipeline::{GreptimeTransformer, Pipeline}; use query::QueryEngineRef; +use servers::error::Result as ServerResult; use session::context::{QueryContext, QueryContextRef}; use snafu::{OptionExt, ResultExt}; use table::TableRef; @@ -210,11 +211,11 @@ impl PipelineOperator { pub async fn insert_pipeline( &self, - query_ctx: QueryContextRef, name: &str, content_type: &str, pipeline: &str, - ) -> servers::error::Result<()> { + query_ctx: QueryContextRef, + ) -> ServerResult<()> { self.create_pipeline_table_if_not_exists(query_ctx.current_catalog()) .await .map_err(|e| { diff --git a/src/servers/src/http.rs b/src/servers/src/http.rs index c54decff4160..d9d8b6ec2731 100644 --- a/src/servers/src/http.rs +++ b/src/servers/src/http.rs @@ -73,6 +73,7 @@ use crate::query_handler::{ use crate::server::Server; pub mod authorize; +pub mod event; pub mod handler; pub mod header; pub mod influxdb; @@ -711,8 +712,8 @@ impl HttpServer { fn route_log(log_handler: LogHandlerRef) -> Router { Router::new() - .route("/logs", routing::post(handler::log_ingester)) - .route("/pipelines", routing::post(handler::add_pipeline)) + .route("/logs", routing::post(event::log_ingester)) + .route("/pipelines", routing::post(event::add_pipeline)) .layer( ServiceBuilder::new() .layer(HandleErrorLayer::new(handle_error)) diff --git a/src/servers/src/http/event.rs b/src/servers/src/http/event.rs new file mode 100644 index 000000000000..908160c17f39 --- /dev/null +++ b/src/servers/src/http/event.rs @@ -0,0 +1,140 @@ +// Copyright 2023 Greptime Team +// +// Licensed 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 std::collections::HashMap; + +use api::v1::{RowInsertRequest, RowInsertRequests, Rows}; +use axum::extract::{Json, Query, State, TypedHeader}; +use axum::headers::ContentType; +use axum::Extension; +use common_telemetry::error; +use pipeline::Value as PipelineValue; +use schemars::JsonSchema; +use serde::{Deserialize, Serialize}; +use serde_json::Value; +use session::context::QueryContextRef; +use snafu::{OptionExt, ResultExt}; + +use crate::error::{ + InsertLogSnafu, InvalidParameterSnafu, ParseJsonSnafu, Result, UnsupportedContentTypeSnafu, +}; +use crate::http::greptime_result_v1::GreptimedbV1Response; +use crate::http::HttpResponse; +use crate::query_handler::LogHandlerRef; + +#[derive(Debug, Default, Serialize, Deserialize, JsonSchema)] +pub struct LogIngesterQueryParams { + pub table_name: Option, + pub db: Option, + pub pipeline_name: Option, +} + +#[axum_macros::debug_handler] +pub async fn add_pipeline( + State(handler): State, + Extension(query_ctx): Extension, + Json(payload): Json, +) -> Result { + let name = payload["name"].as_str().context(InvalidParameterSnafu { + reason: "name is required in payload", + })?; + let pipeline = payload["pipeline"] + .as_str() + .context(InvalidParameterSnafu { + reason: "pipeline is required in payload", + })?; + + let content_type = "yaml"; + let result = handler + .insert_pipeline(name, content_type, pipeline, query_ctx) + .await; + + result.map(|_| "ok".to_string()).map_err(|e| { + error!(e; "failed to insert pipeline"); + e + }) +} + +#[axum_macros::debug_handler] +pub async fn log_ingester( + State(state): State, + Query(query_params): Query, + Extension(query_ctx): Extension, + TypedHeader(content_type): TypedHeader, + payload: String, +) -> Result { + let value; + // TODO (qtang): we should decide json or jsonl + if content_type == ContentType::json() { + value = serde_json::from_str(&payload).context(ParseJsonSnafu)?; + // TODO (qtang): we should decide which content type to support + // form_url_cncoded type is only placeholder + } else if content_type == ContentType::form_url_encoded() { + value = parse_space_separated_log(payload)?; + } else { + return UnsupportedContentTypeSnafu { content_type }.fail(); + } + log_ingester_inner(state, query_params, query_ctx, value) + .await + .or_else(|e| InsertLogSnafu { msg: e }.fail()) +} + +fn parse_space_separated_log(payload: String) -> Result { + // ToStructuredLogSnafu + let _log = payload.split_whitespace().collect::>(); + // TODO (qtang): implement this + todo!() +} + +async fn log_ingester_inner( + state: LogHandlerRef, + query_params: LogIngesterQueryParams, + query_ctx: QueryContextRef, + payload: Value, +) -> std::result::Result { + let pipeline_id = query_params + .pipeline_name + .ok_or("pipeline_name is required".to_string())?; + + let pipeline_data = PipelineValue::try_from(payload)?; + + let pipeline = state + .get_pipeline(&pipeline_id, query_ctx.clone()) + .await + .map_err(|e| e.to_string())?; + let transformed_data: Rows = pipeline.exec(pipeline_data)?; + + let table_name = query_params + .table_name + .ok_or("table_name is required".to_string())?; + + let insert_request = RowInsertRequest { + rows: Some(transformed_data), + table_name: table_name.clone(), + }; + let insert_requests = RowInsertRequests { + inserts: vec![insert_request], + }; + state + .insert_log(insert_requests, query_ctx) + .await + .map(|_| { + HttpResponse::GreptimedbV1(GreptimedbV1Response { + output: vec![], + execution_time_ms: 0, + resp_metrics: HashMap::new(), + }) + }) + .map_err(|e| e.to_string()) +} diff --git a/src/servers/src/http/handler.rs b/src/servers/src/http/handler.rs index fd9623864e3f..79f60639d272 100644 --- a/src/servers/src/http/handler.rs +++ b/src/servers/src/http/handler.rs @@ -17,9 +17,7 @@ use std::env; use std::time::Instant; use aide::transform::TransformOperation; -use api::v1::{RowInsertRequest, RowInsertRequests, Rows}; -use axum::extract::{Json, Query, State, TypedHeader}; -use axum::headers::ContentType; +use axum::extract::{Json, Query, State}; use axum::response::{IntoResponse, Response}; use axum::{Extension, Form}; use common_error::ext::ErrorExt; @@ -28,16 +26,13 @@ use common_plugins::GREPTIME_EXEC_WRITE_COST; use common_query::{Output, OutputData}; use common_recordbatch::util; use common_telemetry::tracing; -use pipeline::Value as PipelineValue; use query::parser::{PromQuery, DEFAULT_LOOKBACK_STRING}; use schemars::JsonSchema; use serde::{Deserialize, Serialize}; use serde_json::Value; use session::context::QueryContextRef; -use snafu::ResultExt; use super::header::collect_plan_metrics; -use crate::error::{Error, InsertLogSnafu, ParseJsonSnafu, UnsupportedContentTypeSnafu}; use crate::http::arrow_result::ArrowResponse; use crate::http::csv_result::CsvResponse; use crate::http::error_result::ErrorResponse; @@ -50,7 +45,6 @@ use crate::http::{ }; use crate::metrics_handler::MetricsHandler; use crate::query_handler::sql::ServerSqlQueryHandlerRef; -use crate::query_handler::LogHandlerRef; #[derive(Debug, Default, Serialize, Deserialize, JsonSchema)] pub struct SqlQuery { @@ -71,110 +65,6 @@ pub struct SqlQuery { pub limit: Option, } -#[derive(Debug, Default, Serialize, Deserialize, JsonSchema)] -pub struct LogIngesterQueryParams { - pub table_name: Option, - pub db: Option, - pub pipeline_name: Option, -} - -fn parse_space_separated_log(payload: String) -> Result { - // ToStructuredLogSnafu - let _log = payload.split_whitespace().collect::>(); - // TODO (qtang): implement this - todo!() -} - -async fn log_ingester_inner( - state: LogHandlerRef, - query_params: LogIngesterQueryParams, - query_ctx: QueryContextRef, - payload: Value, -) -> Result { - let pipeline_id = query_params - .pipeline_name - .ok_or("pipeline_name is required".to_string())?; - - let pipeline_data = PipelineValue::try_from(payload)?; - - let pipeline = state - .get_pipeline(query_ctx.clone(), &pipeline_id) - .await - .map_err(|e| e.to_string())?; - let transformed_data: Rows = pipeline.exec(pipeline_data)?; - - let table_name = query_params - .table_name - .ok_or("table_name is required".to_string())?; - - let insert_request = RowInsertRequest { - rows: Some(transformed_data), - table_name: table_name.clone(), - }; - let insert_requests = RowInsertRequests { - inserts: vec![insert_request], - }; - state - .insert_log(insert_requests, query_ctx) - .await - .map(|_| { - HttpResponse::GreptimedbV1(GreptimedbV1Response { - output: vec![], - execution_time_ms: 0, - resp_metrics: HashMap::new(), - }) - }) - .map_err(|e| e.to_string()) -} - -/// handler to log ingester -#[axum_macros::debug_handler] -pub async fn log_ingester( - State(state): State, - Query(query_params): Query, - Extension(query_ctx): Extension, - TypedHeader(content_type): TypedHeader, - payload: String, -) -> Result { - let value; - // TODO (qtang): we should decide json or jsonl - if content_type == ContentType::json() { - value = serde_json::from_str(&payload).context(ParseJsonSnafu)?; - // TODO (qtang): we should decide which content type to support - // form_url_cncoded type is only placeholder - } else if content_type == ContentType::form_url_encoded() { - value = parse_space_separated_log(payload)?; - } else { - return UnsupportedContentTypeSnafu { content_type }.fail(); - } - log_ingester_inner(state, query_params, query_ctx, value) - .await - .or_else(|e| InsertLogSnafu { msg: e }.fail()) -} - -#[axum_macros::debug_handler] -pub async fn add_pipeline( - State(_state): State, - Query(_query_params): Query, - Extension(_query_ctx): Extension, - TypedHeader(_content_type): TypedHeader, - Json(paylod): Json, -) -> String { - let name = paylod["name"].as_str().unwrap(); - let pipeline = paylod["pipeline"].as_str().unwrap(); - let content_type = "yaml"; - let result = _state - .insert_pipeline(_query_ctx, name, content_type, pipeline) - .await; - match result { - Ok(_) => String::from("ok"), - Err(e) => { - common_telemetry::error!("failed to insert pipeline.{e:?}"); - e.to_string() - } - } -} - /// Handler to execute sql #[axum_macros::debug_handler] #[tracing::instrument(skip_all, fields(protocol = "http", request_type = "sql"))] diff --git a/src/servers/src/query_handler.rs b/src/servers/src/query_handler.rs index c2ef268d494f..d6a280f4a685 100644 --- a/src/servers/src/query_handler.rs +++ b/src/servers/src/query_handler.rs @@ -124,17 +124,20 @@ pub trait OpenTelemetryProtocolHandler { #[async_trait] pub trait LogHandler { async fn insert_log(&self, log: RowInsertRequests, ctx: QueryContextRef) -> Result; + async fn get_pipeline( &self, - query_ctx: QueryContextRef, name: &str, + query_ctx: QueryContextRef, ) -> Result>; + async fn insert_pipeline( &self, - query_ctx: QueryContextRef, name: &str, content_type: &str, pipeline: &str, + query_ctx: QueryContextRef, ) -> Result<()>; - async fn delete_pipeline(&self, query_ctx: QueryContextRef, name: &str) -> Result<()>; + + async fn delete_pipeline(&self, name: &str, query_ctx: QueryContextRef) -> Result<()>; } From ddea3c19191744056e44105d8998bd3643f02d92 Mon Sep 17 00:00:00 2001 From: paomian Date: Tue, 4 Jun 2024 17:49:45 +0800 Subject: [PATCH 20/51] chore: add test for pipeline --- Cargo.lock | 2 + src/frontend/Cargo.toml | 4 +- src/frontend/src/pipeline.rs | 168 +++++++++++++++++++++++++++++--- src/pipeline/src/lib.rs | 1 + src/pipeline/src/table.rs | 6 +- src/servers/src/http/handler.rs | 4 +- 6 files changed, 167 insertions(+), 18 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index f9186ec6b204..8f1c2e446862 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3923,6 +3923,7 @@ dependencies = [ "common-time", "common-version", "datanode", + "datatypes", "futures", "humantime-serde", "lazy_static", @@ -3939,6 +3940,7 @@ dependencies = [ "raft-engine", "script", "serde", + "serde_json", "servers", "session", "snafu 0.8.3", diff --git a/src/frontend/Cargo.toml b/src/frontend/Cargo.toml index a359a56702b5..d5489028a916 100644 --- a/src/frontend/Cargo.toml +++ b/src/frontend/Cargo.toml @@ -63,7 +63,7 @@ toml.workspace = true tonic.workspace = true [dev-dependencies] -catalog.workspace = true +catalog = { workspace = true, features = ["testing"] } common-test-util.workspace = true datanode.workspace = true futures = "0.3" @@ -71,3 +71,5 @@ meta-srv = { workspace = true, features = ["mock"] } strfmt = "0.2" tower.workspace = true uuid.workspace = true +datatypes.workspace = true +serde_json.workspace = true diff --git a/src/frontend/src/pipeline.rs b/src/frontend/src/pipeline.rs index 75f4e2c1c703..19e65a716ca0 100644 --- a/src/frontend/src/pipeline.rs +++ b/src/frontend/src/pipeline.rs @@ -110,7 +110,7 @@ impl PipelineOperator { ); } - pub async fn create_pipeline_table_if_not_exists(&self, catalog: &str) -> Result<()> { + async fn create_pipeline_table_if_not_exists(&self, catalog: &str) -> Result<()> { if self.get_pipeline_table_from_cache(catalog).is_some() { return Ok(()); } @@ -173,24 +173,21 @@ impl PipelineOperator { } pub fn get_pipeline_table_from_cache(&self, catalog: &str) -> Option> { - // FIXME (qtang): we should impl this self.tables.read().unwrap().get(catalog).cloned() } - pub async fn insert_and_compile( + async fn insert_and_compile( &self, catalog: &str, schema: &str, name: &str, content_type: &str, pipeline: &str, - ) -> Result<()> { - let _compiled_pipeline = PipelineTable::::compile_pipeline(pipeline) - .map_err(BoxedError::new) - .context(InsertPipelineSnafu { name })?; + ) -> Result> { self.get_pipeline_table_from_cache(catalog) - // FIXME (qtang): we should add error handling here - .unwrap() + .with_context(|| TableNotFoundSnafu { + table_name: PIPELINE_TABLE_NAME, + })? .insert_and_compile(schema, name, content_type, pipeline) .await .map_err(|e| { @@ -199,8 +196,7 @@ impl PipelineOperator { } BoxedError::new(e) }) - .context(InsertPipelineSnafu { name })?; - Ok(()) + .context(InsertPipelineSnafu { name }) } } @@ -277,3 +273,153 @@ impl PipelineOperator { Ok(()) } } + +#[cfg(test)] +mod tests { + use std::sync::Arc; + + use api::v1::greptime_request::Request; + use async_trait::async_trait; + use catalog::memory::MemoryCatalogManager; + use common_query::Output; + use common_recordbatch::RecordBatch; + use datatypes::prelude::ConcreteDataType; + use datatypes::schema::{ColumnSchema, Schema}; + use datatypes::vectors::{StringVector, VectorRef}; + use pipeline::Value as PipelineValue; + use query::QueryEngineFactory; + use serde_json::Value; + use servers::query_handler::grpc::GrpcQueryHandler; + use session::context::{QueryContext, QueryContextRef}; + use table::test_util::MemTable; + + use crate::error::{Error, Result}; + use crate::pipeline::PipelineOperator; + + struct MockGrpcQueryHandler; + + #[async_trait] + impl GrpcQueryHandler for MockGrpcQueryHandler { + type Error = Error; + + async fn do_query(&self, _query: Request, _ctx: QueryContextRef) -> Result { + Ok(Output::new_with_affected_rows(1)) + } + } + + pub fn setup_pipeline_operator(schema: &str, name: &str, pipeline: &str) -> PipelineOperator { + let column_schemas = vec![ + ColumnSchema::new("pipeline", ConcreteDataType::string_datatype(), false), + ColumnSchema::new("schema", ConcreteDataType::string_datatype(), false), + ColumnSchema::new("name", ConcreteDataType::string_datatype(), false), + ]; + + let columns: Vec = vec![ + Arc::new(StringVector::from(vec![pipeline])), + Arc::new(StringVector::from(vec![schema])), + Arc::new(StringVector::from(vec![name])), + ]; + + let schema = Arc::new(Schema::new(column_schemas)); + let recordbatch = RecordBatch::new(schema, columns).unwrap(); + + let table = MemTable::new_with_catalog( + "pipelines", + recordbatch, + 1, + "greptime".to_string(), + "greptime_private".to_string(), + vec![], + ); + + let catalog_manager = MemoryCatalogManager::new_with_table(table.clone()); + + let factory = QueryEngineFactory::new(catalog_manager.clone(), None, None, None, false); + let query_engine = factory.query_engine(); + let pipeline_operator = PipelineOperator { + grpc_handler: arc_swap::ArcSwap::new(Arc::new(Arc::new(MockGrpcQueryHandler) as _)), + catalog_manager, + query_engine, + tables: Default::default(), + }; + pipeline_operator + } + + #[tokio::test] + async fn test_pipeline_table() { + let catalog = "greptime"; + let schema = "schema"; + let name = "test"; + let pipeline_content = r#" +--- +processors: + - date: + field: time + formats: + - "%Y-%m-%d %H:%M:%S%.3f" + ignore_missing: true + +transform: + - fields: + - id1 + - id2 + type: int32 + - fields: + - type + - log + type: string + - field: time + type: time + index: timestamp"#; + let pipeline_operator = setup_pipeline_operator(schema, name, pipeline_content); + + let data = r#"{"time": "2024-05-25 20:16:37.308", "id1": "1852", "id2": "1852", "type": "E", "log": "SOAProxy WindowCtrlManager: enSunshadeOpe :4\n"}"#; + let data: Value = serde_json::from_str(data).unwrap(); + let pipeline_data = PipelineValue::try_from(data).unwrap(); + let pipeline_table = pipeline_operator.get_pipeline_table_from_cache(catalog); + assert!(pipeline_table.is_none()); + let query_ctx = QueryContextRef::new(QueryContext::with(catalog, schema)); + let pipeline = pipeline_operator + .get_pipeline(query_ctx.clone(), name) + .await + .unwrap(); + + let result = pipeline.exec(pipeline_data.clone()).unwrap(); + assert_eq!(result.schema.len(), 5); + let name_v2 = "test2"; + let pipeline_content_v2 = r#" +--- +processors: + - date: + field: time + formats: + - "%Y-%m-%d %H:%M:%S%.3f" + ignore_missing: true + +transform: + - fields: + - id1 + - id2 + type: string + - fields: + - type + - log + type: string + - field: time + type: time + index: timestamp"#; + + let _ = pipeline_operator + .insert_and_compile(catalog, schema, name_v2, "yaml", pipeline_content_v2) + .await + .unwrap(); + + let pipeline = pipeline_operator + .get_pipeline(query_ctx, name_v2) + .await + .unwrap(); + let result = pipeline.exec(pipeline_data).unwrap(); + let scheam = result.schema; + assert_eq!(scheam.len(), 5); + } +} diff --git a/src/pipeline/src/lib.rs b/src/pipeline/src/lib.rs index e102ce489a3e..dab3c080a7dc 100644 --- a/src/pipeline/src/lib.rs +++ b/src/pipeline/src/lib.rs @@ -3,3 +3,4 @@ pub mod table; pub use pipeline::transform::GreptimeTransformer; pub use pipeline::Pipeline; +pub use pipeline::value::Value; diff --git a/src/pipeline/src/table.rs b/src/pipeline/src/table.rs index b6d0a31d5a39..1398b01a32a3 100644 --- a/src/pipeline/src/table.rs +++ b/src/pipeline/src/table.rs @@ -279,7 +279,7 @@ impl PipelineTable { name: &str, content_type: &str, pipeline: &str, - ) -> Result<()> { + ) -> Result> { let compiled_pipeline = Self::compile_pipeline(pipeline)?; self.insert_pipeline_to_pipeline_table(schema, name, content_type, pipeline) @@ -287,10 +287,10 @@ impl PipelineTable { self.pipelines.write().unwrap().insert( Self::generate_pipeline_cache_key(schema, name), - compiled_pipeline, + compiled_pipeline.clone(), ); - Ok(()) + Ok(compiled_pipeline) } async fn find_pipeline_by_name(&self, schema: &str, name: &str) -> Result { diff --git a/src/servers/src/http/handler.rs b/src/servers/src/http/handler.rs index 09c19c42260a..8c7db1127ee5 100644 --- a/src/servers/src/http/handler.rs +++ b/src/servers/src/http/handler.rs @@ -37,9 +37,7 @@ use session::context::QueryContextRef; use snafu::ResultExt; use super::header::collect_plan_metrics; -use crate::error::{ - Error, InsertLogSnafu, ParseJsonSnafu, UnsupportedContentTypeSnafu, -}; +use crate::error::{Error, InsertLogSnafu, ParseJsonSnafu, UnsupportedContentTypeSnafu}; use crate::http::arrow_result::ArrowResponse; use crate::http::csv_result::CsvResponse; use crate::http::error_result::ErrorResponse; From 423e51e60b23e16e072adc60b9a61876455fa7b4 Mon Sep 17 00:00:00 2001 From: shuiyisong Date: Tue, 4 Jun 2024 20:03:27 +0800 Subject: [PATCH 21/51] chore: fmt --- src/frontend/Cargo.toml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/frontend/Cargo.toml b/src/frontend/Cargo.toml index d5489028a916..56f4ab904ae2 100644 --- a/src/frontend/Cargo.toml +++ b/src/frontend/Cargo.toml @@ -66,10 +66,10 @@ tonic.workspace = true catalog = { workspace = true, features = ["testing"] } common-test-util.workspace = true datanode.workspace = true +datatypes.workspace = true futures = "0.3" meta-srv = { workspace = true, features = ["mock"] } +serde_json.workspace = true strfmt = "0.2" tower.workspace = true uuid.workspace = true -datatypes.workspace = true -serde_json.workspace = true From 8066eb341431ee4bb7530dfa659f851a70336112 Mon Sep 17 00:00:00 2001 From: shuiyisong Date: Tue, 4 Jun 2024 20:22:21 +0800 Subject: [PATCH 22/51] refactor: bring in pipeline 7d2402701877901871dd1294a65ac937605a6a93 --- src/pipeline/src/etl/processor/dissect.rs | 597 +++++++++++++++++++--- 1 file changed, 533 insertions(+), 64 deletions(-) diff --git a/src/pipeline/src/etl/processor/dissect.rs b/src/pipeline/src/etl/processor/dissect.rs index 2af008ceac86..063996152a7d 100644 --- a/src/pipeline/src/etl/processor/dissect.rs +++ b/src/pipeline/src/etl/processor/dissect.rs @@ -25,17 +25,131 @@ pub(crate) const PROCESSOR_DISSECT: &str = "dissect"; const APPEND_SEPARATOR_NAME: &str = "append_separator"; +#[derive(Debug, PartialEq)] +enum StartModifier { + Append(Option), + NamedSkip, + MapKey, + MapVal, +} + +impl std::fmt::Display for StartModifier { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + match self { + StartModifier::Append(o) => match o { + Some(v) => write!(f, "+/{v}"), + None => write!(f, "+"), + }, + StartModifier::NamedSkip => write!(f, "?"), + StartModifier::MapKey => write!(f, "*"), + StartModifier::MapVal => write!(f, "&"), + } + } +} + +#[derive(Debug, PartialEq)] +struct EndModifier; + +impl std::fmt::Display for EndModifier { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + write!(f, "->",) + } +} + +#[derive(Debug, PartialEq, Default)] +struct Name { + name: String, + start_modifier: Option, + end_modifier: Option, +} + +impl Name { + fn is_name_empty(&self) -> bool { + self.name.is_empty() + } + + fn is_empty(&self) -> bool { + self.name.is_empty() && self.start_modifier.is_none() && self.end_modifier.is_none() + } + + fn try_start_modifier(&mut self, modifier: StartModifier) -> Result<(), String> { + match &self.start_modifier { + Some(m) => Err(format!("'{m}' modifier already set, but found {modifier}",)), + None => { + self.start_modifier = Some(modifier); + Ok(()) + } + } + } + + fn try_append_order(&mut self, order: u32) -> Result<(), String> { + match &mut self.start_modifier { + Some(StartModifier::Append(o)) => match o { + Some(n) => Err(format!( + "Append Order modifier is already set to '{n}', cannot be set to {order}" + )), + None => { + *o = Some(order); + Ok(()) + } + }, + Some(m) => Err(format!( + "Order can only be set to Append Modifier, current modifier is {m}" + )), + None => Err("Order can only be set to Append Modifier".to_string()), + } + } + + fn try_end_modifier(&mut self) -> Result<(), String> { + match &self.end_modifier { + Some(m) => Err(format!("End modifier already set: '{m}'")), + None => { + self.end_modifier = Some(EndModifier); + Ok(()) + } + } + } + + fn is_append_modifier_set(&self) -> bool { + matches!(self.start_modifier, Some(StartModifier::Append(_))) + } + + fn is_start_modifier_set(&self) -> bool { + self.start_modifier.is_some() + } + + fn is_end_modifier_set(&self) -> bool { + self.end_modifier.is_some() + } +} + +impl std::fmt::Display for Name { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + write!(f, "{}", self.name) + } +} + +impl From<&str> for Name { + fn from(value: &str) -> Self { + Name { + name: value.to_string(), + start_modifier: None, + end_modifier: None, + } + } +} + #[derive(Debug, PartialEq)] enum Part { Split(String), - Key(String), + Name(Name), } impl Part { fn is_empty(&self) -> bool { match self { Part::Split(v) => v.is_empty(), - Part::Key(v) => v.is_empty(), + Part::Name(v) => v.is_empty(), } } @@ -43,27 +157,14 @@ impl Part { Part::Split(String::new()) } - fn empty_key() -> Self { - Part::Key(String::new()) - } -} - -impl std::ops::Deref for Part { - type Target = String; - - fn deref(&self) -> &Self::Target { - match self { - Part::Split(v) => v, - Part::Key(v) => v, - } + fn empty_name() -> Self { + Part::Name(Name::default()) } -} -impl std::ops::DerefMut for Part { - fn deref_mut(&mut self) -> &mut Self::Target { + fn push(&mut self, ch: char) { match self { - Part::Split(v) => v, - Part::Key(v) => v, + Part::Split(v) => v.push(ch), + Part::Name(v) => v.name.push(ch), } } } @@ -96,40 +197,102 @@ impl std::str::FromStr for Pattern { let mut cursor = Part::empty_split(); let origin = s.to_string(); - let mut last_ch = None; let chars: Vec = origin.chars().collect(); - for i in 0..chars.len() { - let ch = chars[i]; + let mut pos = 0; + while pos < chars.len() { + let ch = chars[pos]; match (ch, &mut cursor) { - // if cursor is Split part, and found %{, then ready to start a Key part - ('%', Part::Split(_)) if i + 1 < chars.len() && chars[i + 1] == '{' => {} - // if cursor is Split part, and found %{, then end the Split part, start the Key part - ('{', Part::Split(_)) if last_ch == Some('%') => { + // if cursor is Split part, and found %{, then ready to start a Name part + ('%', Part::Split(_)) if matches!(chars.get(pos + 1), Some('{')) => { if !cursor.is_empty() { parts.push(cursor); } - cursor = Part::empty_key(); + cursor = Part::empty_name(); + pos += 1; // skip '{' } // if cursor is Split part, and not found % or {, then continue the Split part (_, Part::Split(_)) => { cursor.push(ch); } - // if cursor is Key part, and found }, then end the Key part, start the next Split part - ('}', Part::Key(_)) => { + // if cursor is Name part, and found }, then end the Name part, start the next Split part + ('}', Part::Name(_)) => { parts.push(cursor); cursor = Part::empty_split(); } - (_, Part::Key(_)) if !is_valid_char(ch) => { - return Err(format!("Invalid character in key: '{ch}'")); + ('+', Part::Name(name)) if !name.is_start_modifier_set() => { + name.try_start_modifier(StartModifier::Append(None))?; + } + ('/', Part::Name(name)) if name.is_append_modifier_set() => { + let mut order = 0; + let mut j = pos + 1; + while j < chars.len() { + let digit = chars[j]; + if digit.is_ascii_digit() { + order = order * 10 + digit.to_digit(10).unwrap(); + j += 1; + } else { + break; + } + } + + if j == pos + 1 { + return Err(format!( + "Invalid Pattern: '{s}'. Digit order must be set after '/'", + )); + } + + name.try_append_order(order)?; + pos = j - 1; // this will change the position to the last digit of the order + } + ('?', Part::Name(name)) if !name.is_start_modifier_set() => { + name.try_start_modifier(StartModifier::NamedSkip)?; + } + ('*', Part::Name(name)) if !name.is_start_modifier_set() => { + name.try_start_modifier(StartModifier::MapKey)?; + } + ('&', Part::Name(name)) if !name.is_start_modifier_set() => { + name.try_start_modifier(StartModifier::MapVal)?; + } + ('-', Part::Name(name)) if !name.is_end_modifier_set() => { + if let Some('>') = chars.get(pos + 1) { + } else { + return Err(format!( + "Invalid Pattern: '{s}'. expected '->' but only '-'", + )); + } + + if let Some('}') = chars.get(pos + 2) { + } else { + return Err(format!("Invalid Pattern: '{s}'. expected '}}' after '->'",)); + } + + name.try_end_modifier()?; + pos += 1; // only skip '>', the next loop will skip '}' } - (_, Part::Key(_)) => { + (_, Part::Name(name)) if !is_valid_char(ch) => { + let tail: String = if name.is_name_empty() { + format!("Invalid '{ch}'") + } else { + format!("Invalid '{ch}' in '{name}'") + }; + return Err(format!("Invalid Pattern: '{s}'. {tail}")); + } + (_, Part::Name(_)) => { cursor.push(ch); } } - last_ch = Some(ch); + pos += 1; + } + + match cursor { + Part::Split(ref split) if !split.is_empty() => parts.push(cursor), + Part::Name(name) if !name.is_empty() => { + return Err(format!("Invalid Pattern: '{s}'. '{name}' is not closed")) + } + _ => {} } let pattern = Self { parts, origin }; @@ -144,21 +307,71 @@ impl Pattern { return Err("Empty pattern is not allowed".to_string()); } + let mut map_items = std::collections::HashSet::new(); + for i in 0..self.len() { let this_part = &self[i]; let next_part = self.get(i + 1); match (this_part, next_part) { (Part::Split(split), _) if split.is_empty() => { - return Err("Empty split is not allowed".to_string()); + return Err(format!( + "Invalid Pattern: '{}'. Empty split is not allowed", + self.origin + )); } - (Part::Key(key1), Some(Part::Key(key2))) => { + (Part::Name(name1), Some(Part::Name(name2))) => { return Err(format!( - "consecutive keys are not allowed: '{key1}' '{key2}'" + "Invalid Pattern: '{}'. consecutive names are not allowed: '{}' '{}'", + self.origin, name1, name2 )); } + (Part::Name(name), _) if name.is_name_empty() => { + if let Some(ref m) = name.start_modifier { + return Err(format!( + "Invalid Pattern: '{}'. only '{}' modifier is invalid", + self.origin, m + )); + } + } + (Part::Name(name), _) => match name.start_modifier { + Some(StartModifier::MapKey) => { + if map_items.contains(&name.name) { + return Err(format!( + "Invalid Pattern: '{}'. Duplicate map key: '{}'", + self.origin, name.name + )); + } else { + map_items.insert(name.name.clone()); + } + } + Some(StartModifier::MapVal) => { + if !map_items.contains(&name.name) { + return Err(format!( + "Invalid Pattern: '{}'. Map key not found: '{}'", + self.origin, name.name + )); + } else { + map_items.remove(&name.name); + } + } + _ => {} + }, _ => {} } } + + if !map_items.is_empty() { + return Err(format!( + "Invalid Pattern: '{}'. Matched value not found for: '{}'", + self.origin, + map_items + .iter() + .map(|s| s.as_str()) + .collect::>() + .join(",") + )); + } + Ok(()) } } @@ -204,6 +417,7 @@ impl DissectProcessor { let this_part = &pattern[i]; let next_part = pattern.get(i + 1); match (this_part, next_part) { + // if Split part, and exactly matches, then move pos split.len() forward (Part::Split(split), _) => { let split_chs = split.chars().collect::>(); let split_len = split_chs.len(); @@ -220,12 +434,14 @@ impl DissectProcessor { pos += split_len; } - (Part::Key(key), None) => { + // if Name part is the last part, then the rest of the input is the value + (Part::Name(name), None) => { let value = chs[pos..].iter().collect::(); - map.insert(key.clone(), Value::String(value)); + map.insert(name.to_string(), Value::String(value)); } - (Part::Key(key), Some(Part::Split(split))) => match split.chars().next() { + // if Name part, and next part is Split, then find the matched value of the name + (Part::Name(name), Some(Part::Split(split))) => match split.chars().next() { None => return Err("Empty split is not allowed".to_string()), Some(stop) => { let mut end = pos; @@ -238,13 +454,13 @@ impl DissectProcessor { } let value = chs[pos..end].iter().collect::(); - map.insert(key.clone(), Value::String(value)); + map.insert(name.to_string(), Value::String(value)); pos = end; } }, - (Part::Key(key1), Some(Part::Key(key2))) => { + (Part::Name(name1), Some(Part::Name(name2))) => { return Err(format!( - "consecutive keys are not allowed: '{key1}' '{key2}'" + "consecutive names are not allowed: '{name1}' '{name2}'" )); } } @@ -336,31 +552,31 @@ fn is_valid_char(ch: char) -> bool { mod tests { use std::collections::HashMap; - use super::{DissectProcessor, Part, Pattern}; + use super::{DissectProcessor, EndModifier, Part, Pattern}; use crate::etl::value::{Map, Value}; #[test] - fn test_pattern() { + fn test_dissect_simple_pattern() { let cases = [( "%{clientip} %{ident} %{auth} [%{timestamp}] \"%{verb} %{request} HTTP/%{httpversion}\" %{status} %{size}", vec![ - Part::Key("clientip".chars().collect()), - Part::Split(" ".chars().collect()), - Part::Key("ident".chars().collect()), - Part::Split(" ".chars().collect()), - Part::Key("auth".chars().collect()), - Part::Split(" [".chars().collect()), - Part::Key("timestamp".chars().collect()), - Part::Split("] \"".chars().collect()), - Part::Key("verb".chars().collect()), - Part::Split(" ".chars().collect()), - Part::Key("request".chars().collect()), - Part::Split(" HTTP/".chars().collect()), - Part::Key("httpversion".chars().collect()), - Part::Split("\" ".chars().collect()), - Part::Key("status".chars().collect()), - Part::Split(" ".chars().collect()), - Part::Key("size".chars().collect()), + Part::Name("clientip".into()), + Part::Split(" ".into()), + Part::Name("ident".into()), + Part::Split(" ".into()), + Part::Name("auth".into()), + Part::Split(" [".into()), + Part::Name("timestamp".into()), + Part::Split("] \"".into()), + Part::Name("verb".into()), + Part::Split(" ".into()), + Part::Name("request".into()), + Part::Split(" HTTP/".into()), + Part::Name("httpversion".into()), + Part::Split("\" ".into()), + Part::Name("status".into()), + Part::Split(" ".into()), + Part::Name("size".into()), ], )]; @@ -371,7 +587,260 @@ mod tests { } #[test] - fn test_process() { + fn test_dissect_modifier_pattern() { + let cases = [ + ( + "%{ts->} %{level}", + vec![ + Part::Name(super::Name { + name: "ts".into(), + start_modifier: None, + end_modifier: Some(EndModifier), + }), + Part::Split(" ".into()), + Part::Name("level".into()), + ], + ), + ( + "[%{ts}]%{->}[%{level}]", + vec![ + Part::Split("[".into()), + Part::Name(super::Name { + name: "ts".into(), + start_modifier: None, + end_modifier: None, + }), + Part::Split("]".into()), + Part::Name(super::Name { + name: "".into(), + start_modifier: None, + end_modifier: Some(EndModifier), + }), + Part::Split("[".into()), + Part::Name(super::Name { + name: "level".into(), + start_modifier: None, + end_modifier: None, + }), + Part::Split("]".into()), + ], + ), + ( + "%{+name} %{+name} %{+name} %{+name}", + vec![ + Part::Name(super::Name { + name: "name".into(), + start_modifier: Some(super::StartModifier::Append(None)), + end_modifier: None, + }), + Part::Split(" ".into()), + Part::Name(super::Name { + name: "name".into(), + start_modifier: Some(super::StartModifier::Append(None)), + end_modifier: None, + }), + Part::Split(" ".into()), + Part::Name(super::Name { + name: "name".into(), + start_modifier: Some(super::StartModifier::Append(None)), + end_modifier: None, + }), + Part::Split(" ".into()), + Part::Name(super::Name { + name: "name".into(), + start_modifier: Some(super::StartModifier::Append(None)), + end_modifier: None, + }), + ], + ), + ( + "%{+name/2} %{+name/4} %{+name/3} %{+name/1}", + vec![ + Part::Name(super::Name { + name: "name".into(), + start_modifier: Some(super::StartModifier::Append(Some(2))), + end_modifier: None, + }), + Part::Split(" ".into()), + Part::Name(super::Name { + name: "name".into(), + start_modifier: Some(super::StartModifier::Append(Some(4))), + end_modifier: None, + }), + Part::Split(" ".into()), + Part::Name(super::Name { + name: "name".into(), + start_modifier: Some(super::StartModifier::Append(Some(3))), + end_modifier: None, + }), + Part::Split(" ".into()), + Part::Name(super::Name { + name: "name".into(), + start_modifier: Some(super::StartModifier::Append(Some(1))), + end_modifier: None, + }), + ], + ), + ( + "%{clientip} %{?ident} %{?auth} [%{timestamp}]", + vec![ + Part::Name(super::Name { + name: "clientip".into(), + start_modifier: None, + end_modifier: None, + }), + Part::Split(" ".into()), + Part::Name(super::Name { + name: "ident".into(), + start_modifier: Some(super::StartModifier::NamedSkip), + end_modifier: None, + }), + Part::Split(" ".into()), + Part::Name(super::Name { + name: "auth".into(), + start_modifier: Some(super::StartModifier::NamedSkip), + end_modifier: None, + }), + Part::Split(" [".into()), + Part::Name(super::Name { + name: "timestamp".into(), + start_modifier: None, + end_modifier: None, + }), + Part::Split("]".into()), + ], + ), + ( + "[%{ts}] [%{level}] %{*p1}:%{&p1} %{*p2}:%{&p2}", + vec![ + Part::Split("[".into()), + Part::Name(super::Name { + name: "ts".into(), + start_modifier: None, + end_modifier: None, + }), + Part::Split("] [".into()), + Part::Name(super::Name { + name: "level".into(), + start_modifier: None, + end_modifier: None, + }), + Part::Split("] ".into()), + Part::Name(super::Name { + name: "p1".into(), + start_modifier: Some(super::StartModifier::MapKey), + end_modifier: None, + }), + Part::Split(":".into()), + Part::Name(super::Name { + name: "p1".into(), + start_modifier: Some(super::StartModifier::MapVal), + end_modifier: None, + }), + Part::Split(" ".into()), + Part::Name(super::Name { + name: "p2".into(), + start_modifier: Some(super::StartModifier::MapKey), + end_modifier: None, + }), + Part::Split(":".into()), + Part::Name(super::Name { + name: "p2".into(), + start_modifier: Some(super::StartModifier::MapVal), + end_modifier: None, + }), + ], + ), + ]; + + for (pattern, expected) in cases.into_iter() { + let p: Pattern = pattern.parse().unwrap(); + assert_eq!(p.parts, expected); + } + } + + #[test] + fn test_dissect_invalid_pattern() { + let cases = [ + ("", "Empty pattern is not allowed"), + ( + "%{name1}%{name2}", + "Invalid Pattern: '%{name1}%{name2}'. consecutive names are not allowed: 'name1' 'name2'" + ), + ( + "%{} %{ident", + "Invalid Pattern: '%{} %{ident'. 'ident' is not closed", + ), + ( + "%{->clientip} ", + "Invalid Pattern: '%{->clientip} '. expected '}' after '->'", + ), + ( + "%{/clientip} ", + "Invalid Pattern: '%{/clientip} '. Invalid '/'", + ), + ( + "%{+?clientip} ", + "Invalid Pattern: '%{+?clientip} '. Invalid '?'", + ), + ( + "%{+clientip/} ", + "Invalid Pattern: '%{+clientip/} '. Digit order must be set after '/'", + ), + ( + "%{+clientip/a} ", + "Invalid Pattern: '%{+clientip/a} '. Digit order must be set after '/'", + ), + ( + "%{clientip/1} ", + "Invalid Pattern: '%{clientip/1} '. Invalid '/' in 'clientip'", + ), + ( + "%{+clientip/1/2} ", + "Append Order modifier is already set to '1', cannot be set to 2", + ), + ( + "%{+/1} ", + "Invalid Pattern: '%{+/1} '. only '+/1' modifier is invalid", + ), + ( + "%{+} ", + "Invalid Pattern: '%{+} '. only '+' modifier is invalid", + ), + ( + "%{?} ", + "Invalid Pattern: '%{?} '. only '?' modifier is invalid", + ), + ( + "%{*} ", + "Invalid Pattern: '%{*} '. only '*' modifier is invalid", + ), + ( + "%{&} ", + "Invalid Pattern: '%{&} '. only '&' modifier is invalid", + ), + ( + "%{*ip}", + "Invalid Pattern: '%{*ip}'. Matched value not found for: 'ip'", + ), + ( + "%{*ip} %{*ip}", + "Invalid Pattern: '%{*ip} %{*ip}'. Duplicate map key: 'ip'", + ), + ( + "%{*ip} %{&ip2}", + "Invalid Pattern: '%{*ip} %{&ip2}'. Map key not found: 'ip2'", + ), + ]; + + for (pattern, expected) in cases.into_iter() { + let err = pattern.parse::().unwrap_err(); + assert_eq!(err, expected); + } + } + + #[test] + fn test_dissect_process() { let assert = |pattern_str: &str, input: &str, expected: HashMap| { let chs = input.chars().collect::>(); let pattern = pattern_str.parse().unwrap(); @@ -399,7 +868,7 @@ mod tests { .collect::>(); { - // pattern start with Key + // pattern start with Name let pattern_str = "%{clientip} %{ident} %{auth} [%{timestamp}] \"%{verb} %{request} HTTP/%{httpversion}\" %{status} %{size}"; let input = "1.2.3.4 - - [30/Apr/1998:22:00:52 +0000] \"GET /english/venues/cities/images/montpellier/18.gif HTTP/1.0\" 200 3171"; From e2a2e504e83c4f25bf51f8f1a182ff962b0f6eec Mon Sep 17 00:00:00 2001 From: shuiyisong Date: Wed, 5 Jun 2024 00:56:39 +0800 Subject: [PATCH 23/51] refactor: move `pipeline_operator` to `pipeline` crate --- src/frontend/src/error.rs | 26 -------- src/frontend/src/instance.rs | 2 +- src/frontend/src/instance/builder.rs | 2 +- src/frontend/src/instance/log_handler.rs | 6 +- src/frontend/src/lib.rs | 1 - src/pipeline/src/lib.rs | 2 +- src/pipeline/src/mng/error.rs | 22 ++++++- src/pipeline/src/mng/mod.rs | 1 + .../src/mng/pipeline_operator.rs} | 62 ++++--------------- src/servers/src/error.rs | 19 ++---- 10 files changed, 43 insertions(+), 100 deletions(-) rename src/{frontend/src/pipeline.rs => pipeline/src/mng/pipeline_operator.rs} (78%) diff --git a/src/frontend/src/error.rs b/src/frontend/src/error.rs index 82b2155f021b..e7b7a19885d7 100644 --- a/src/frontend/src/error.rs +++ b/src/frontend/src/error.rs @@ -297,29 +297,6 @@ pub enum Error { source: table::error::Error, }, - #[snafu(display("Failed to insert pipeline to pipeline table, name: {}", name))] - InsertPipeline { - name: String, - #[snafu(implicit)] - location: Location, - source: BoxedError, - }, - - #[snafu(display("Failed to parse pipeline"))] - ParsePipeline { - #[snafu(implicit)] - location: Location, - source: BoxedError, - }, - - #[snafu(display("Failed to get pipeline to pipeline table, name: {}", name))] - GetPipeline { - name: String, - #[snafu(implicit)] - location: Location, - source: BoxedError, - }, - #[snafu(display("Unsupported format: {:?}", format))] UnsupportedFormat { #[snafu(implicit)] @@ -406,9 +383,6 @@ impl ErrorExt for Error { | Error::EmptyData { .. } | Error::ColumnNoneDefaultValue { .. } | Error::IncompleteGrpcRequest { .. } - | Error::InsertPipeline { .. } - | Error::ParsePipeline { .. } - | Error::GetPipeline { .. } | Error::InvalidTlsConfig { .. } => StatusCode::InvalidArguments, Error::NotSupported { .. } => StatusCode::Unsupported, diff --git a/src/frontend/src/instance.rs b/src/frontend/src/instance.rs index 2d4434222cc5..59265d5d1af4 100644 --- a/src/frontend/src/instance.rs +++ b/src/frontend/src/instance.rs @@ -49,6 +49,7 @@ use meta_client::MetaClientOptions; use operator::delete::DeleterRef; use operator::insert::InserterRef; use operator::statement::StatementExecutor; +use pipeline::pipeline_operator::PipelineOperator; use prometheus::HistogramTimer; use query::metrics::OnDone; use query::parser::{PromQuery, QueryLanguageParser, QueryStatement}; @@ -89,7 +90,6 @@ use crate::error::{ }; use crate::frontend::FrontendOptions; use crate::heartbeat::HeartbeatTask; -use crate::pipeline::PipelineOperator; use crate::script::ScriptExecutor; #[async_trait] diff --git a/src/frontend/src/instance/builder.rs b/src/frontend/src/instance/builder.rs index 2ad668ef1f19..ae8d77dd20b5 100644 --- a/src/frontend/src/instance/builder.rs +++ b/src/frontend/src/instance/builder.rs @@ -30,6 +30,7 @@ use operator::request::Requester; use operator::statement::{StatementExecutor, StatementExecutorRef}; use operator::table::TableMutationOperator; use partition::manager::PartitionRuleManager; +use pipeline::pipeline_operator::PipelineOperator; use query::QueryEngineFactory; use servers::server::ServerHandlers; use snafu::OptionExt; @@ -38,7 +39,6 @@ use crate::error::{self, Result}; use crate::heartbeat::HeartbeatTask; use crate::instance::region_query::FrontendRegionQueryHandler; use crate::instance::Instance; -use crate::pipeline::PipelineOperator; use crate::script::ScriptExecutor; /// The frontend [`Instance`] builder. diff --git a/src/frontend/src/instance/log_handler.rs b/src/frontend/src/instance/log_handler.rs index dd34614b3ae9..26e37498e591 100644 --- a/src/frontend/src/instance/log_handler.rs +++ b/src/frontend/src/instance/log_handler.rs @@ -18,7 +18,7 @@ use auth::{PermissionChecker, PermissionCheckerRef, PermissionReq}; use client::Output; use common_error::ext::BoxedError; use pipeline::{GreptimeTransformer, Pipeline}; -use servers::error::{AuthSnafu, ExecuteGrpcRequestSnafu, Result as ServerResult}; +use servers::error::{AuthSnafu, ExecuteGrpcRequestSnafu, PipelineSnafu, Result as ServerResult}; use servers::query_handler::LogHandler; use session::context::QueryContextRef; use snafu::ResultExt; @@ -49,8 +49,7 @@ impl LogHandler for Instance { self.pipeline_operator .get_pipeline(query_ctx, name) .await - .map_err(BoxedError::new) - .context(servers::error::GetPipelineSnafu { name }) + .context(PipelineSnafu) } async fn insert_pipeline( @@ -63,6 +62,7 @@ impl LogHandler for Instance { self.pipeline_operator .insert_pipeline(name, content_type, pipeline, query_ctx) .await + .context(PipelineSnafu) } async fn delete_pipeline(&self, _name: &str, _query_ctx: QueryContextRef) -> ServerResult<()> { diff --git a/src/frontend/src/lib.rs b/src/frontend/src/lib.rs index 83fcbb3d09d2..de800b0b41c6 100644 --- a/src/frontend/src/lib.rs +++ b/src/frontend/src/lib.rs @@ -19,7 +19,6 @@ pub mod frontend; pub mod heartbeat; pub mod instance; pub(crate) mod metrics; -mod pipeline; mod script; pub mod server; pub mod service_config; diff --git a/src/pipeline/src/lib.rs b/src/pipeline/src/lib.rs index ce2f4f6d78a2..b0f16bf8a62d 100644 --- a/src/pipeline/src/lib.rs +++ b/src/pipeline/src/lib.rs @@ -18,4 +18,4 @@ mod mng; pub use etl::transform::GreptimeTransformer; pub use etl::value::Value; pub use etl::{parse, Content, Pipeline}; -pub use mng::{error, table}; +pub use mng::{error, pipeline_operator, table}; diff --git a/src/pipeline/src/mng/error.rs b/src/pipeline/src/mng/error.rs index a6182282dad8..d65f5b44c5ed 100644 --- a/src/pipeline/src/mng/error.rs +++ b/src/pipeline/src/mng/error.rs @@ -39,23 +39,23 @@ pub enum Error { #[snafu(display("Failed to insert pipeline to pipelines table, name: {}", name))] InsertPipeline { name: String, + source: BoxedError, #[snafu(implicit)] location: Location, - source: BoxedError, }, #[snafu(display("Failed to parse pipeline"))] ParsePipeline { + source: BoxedError, #[snafu(implicit)] location: Location, - source: BoxedError, }, #[snafu(display("Pipeline not found, name: {}", name))] PipelineNotFound { + name: String, #[snafu(implicit)] location: Location, - name: String, }, #[snafu(display("Failed to collect record batch"))] @@ -86,6 +86,20 @@ pub enum Error { #[snafu(implicit)] location: Location, }, + + #[snafu(display("General catalog error"))] + Catalog { + source: catalog::error::Error, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("Failed to create table"))] + CreateTable { + source: operator::error::Error, + #[snafu(implicit)] + location: Location, + }, } pub type Result = std::result::Result; @@ -102,6 +116,8 @@ impl ErrorExt for Error { ParsePipeline { .. } => StatusCode::InvalidArguments, BuildDfLogicalPlan { .. } => StatusCode::Internal, ExecuteInternalStatement { source, .. } => source.status_code(), + Catalog { source, .. } => source.status_code(), + CreateTable { source, .. } => source.status_code(), } } diff --git a/src/pipeline/src/mng/mod.rs b/src/pipeline/src/mng/mod.rs index cb4854dc181e..06160869bfea 100644 --- a/src/pipeline/src/mng/mod.rs +++ b/src/pipeline/src/mng/mod.rs @@ -14,3 +14,4 @@ pub mod error; pub mod table; +pub mod pipeline_operator; diff --git a/src/frontend/src/pipeline.rs b/src/pipeline/src/mng/pipeline_operator.rs similarity index 78% rename from src/frontend/src/pipeline.rs rename to src/pipeline/src/mng/pipeline_operator.rs index e94252395ddf..f64391662bab 100644 --- a/src/frontend/src/pipeline.rs +++ b/src/pipeline/src/mng/pipeline_operator.rs @@ -18,22 +18,17 @@ use std::sync::{Arc, RwLock}; use api::v1::CreateTableExpr; use catalog::{CatalogManagerRef, RegisterSystemTableRequest}; use common_catalog::consts::{default_engine, DEFAULT_PRIVATE_SCHEMA_NAME}; -use common_catalog::format_full_table_name; -use common_error::ext::{BoxedError, ErrorExt}; -use common_telemetry::{error, info}; +use common_telemetry::info; use operator::insert::InserterRef; use operator::statement::StatementExecutorRef; -use pipeline::table::{PipelineTable, PipelineTableRef}; -use pipeline::{GreptimeTransformer, Pipeline}; use query::QueryEngineRef; -use servers::error::Result as ServerResult; use session::context::{QueryContext, QueryContextRef}; use snafu::{OptionExt, ResultExt}; use table::TableRef; -use crate::error::{ - CatalogSnafu, GetPipelineSnafu, InsertPipelineSnafu, Result, TableNotFoundSnafu, -}; +use crate::error::{CatalogSnafu, CreateTableSnafu, PipelineTableNotFoundSnafu, Result}; +use crate::table::{PipelineTable, PipelineTableRef}; +use crate::{GreptimeTransformer, Pipeline}; pub const PIPELINE_TABLE_NAME: &str = "pipelines"; @@ -119,16 +114,15 @@ impl PipelineOperator { None, Arc::new(QueryContext::with(catalog, &schema)), ) - .await?; + .await + .context(CreateTableSnafu)?; let table = self .catalog_manager .table(catalog, &schema, &table_name) .await .context(CatalogSnafu)? - .with_context(|| TableNotFoundSnafu { - table_name: format_full_table_name(catalog, &schema, &table_name), - })?; + .context(PipelineTableNotFoundSnafu)?; if let Some(open_hook) = open_hook { (open_hook)(table.clone()).await.context(CatalogSnafu)?; @@ -157,18 +151,9 @@ impl PipelineOperator { pipeline: &str, ) -> Result> { self.get_pipeline_table_from_cache(catalog) - .with_context(|| TableNotFoundSnafu { - table_name: PIPELINE_TABLE_NAME, - })? + .context(PipelineTableNotFoundSnafu)? .insert_and_compile(schema, name, content_type, pipeline) .await - .map_err(|e| { - if e.status_code().should_log_error() { - error!(e; "Failed to insert pipeline"); - } - BoxedError::new(e) - }) - .context(InsertPipelineSnafu { name }) } } @@ -196,13 +181,9 @@ impl PipelineOperator { self.create_pipeline_table_if_not_exists(query_ctx.current_catalog()) .await?; self.get_pipeline_table_from_cache(query_ctx.current_catalog()) - .context(TableNotFoundSnafu { - table_name: PIPELINE_TABLE_NAME, - })? + .context(PipelineTableNotFoundSnafu)? .get_pipeline(query_ctx.current_schema(), name) .await - .map_err(BoxedError::new) - .context(GetPipelineSnafu { name }) } pub async fn insert_pipeline( @@ -211,19 +192,9 @@ impl PipelineOperator { content_type: &str, pipeline: &str, query_ctx: QueryContextRef, - ) -> ServerResult<()> { + ) -> Result<()> { self.create_pipeline_table_if_not_exists(query_ctx.current_catalog()) - .await - .map_err(|e| { - if e.status_code().should_log_error() { - error!(e; "Failed to create pipeline table"); - } - - servers::error::InternalSnafu { - err_msg: e.to_string(), - } - .build() - })?; + .await?; self.insert_and_compile( query_ctx.current_catalog(), @@ -232,16 +203,7 @@ impl PipelineOperator { content_type, pipeline, ) - .await - .map_err(|e| { - if e.status_code().should_log_error() { - error!(e; "Failed to insert pipeline"); - } - - BoxedError::new(e) - }) - .context(servers::error::InsertPipelineSnafu { name })?; - + .await?; Ok(()) } } diff --git a/src/servers/src/error.rs b/src/servers/src/error.rs index d4a849496817..f3b28548d883 100644 --- a/src/servers/src/error.rs +++ b/src/servers/src/error.rs @@ -149,20 +149,11 @@ pub enum Error { source: BoxedError, }, - #[snafu(display("Failed to insert pipeline with name: {}", name))] - InsertPipeline { - name: String, + #[snafu(display("Pipeline management api error"))] + Pipeline { + source: pipeline::error::Error, #[snafu(implicit)] location: Location, - source: BoxedError, - }, - - #[snafu(display("Failed to parse pipeline with name: {}", name))] - GetPipeline { - name: String, - #[snafu(implicit)] - location: Location, - source: BoxedError, }, #[snafu(display("Failed to execute script by name: {}", name))] @@ -652,8 +643,6 @@ impl ErrorExt for Error { CollectRecordbatch { .. } => StatusCode::EngineExecuteQuery, InsertScript { source, .. } - | InsertPipeline { source, .. } - | GetPipeline { source, .. } | ExecuteScript { source, .. } | ExecuteQuery { source, .. } | ExecutePlan { source, .. } @@ -661,6 +650,8 @@ impl ErrorExt for Error { | ExecuteGrpcRequest { source, .. } | CheckDatabaseValidity { source, .. } => source.status_code(), + Pipeline { source, .. } => source.status_code(), + NotSupported { .. } | InvalidParameter { .. } | InvalidQuery { .. } From 209a1a381f6aa47ba5720e07f4b2f8c9abed2f01 Mon Sep 17 00:00:00 2001 From: shuiyisong Date: Wed, 5 Jun 2024 01:20:55 +0800 Subject: [PATCH 24/51] chore: minor update --- src/pipeline/src/mng/error.rs | 11 +++++------ src/pipeline/src/mng/mod.rs | 2 +- src/pipeline/src/mng/pipeline_operator.rs | 4 ++-- src/pipeline/src/mng/table.rs | 8 ++------ 4 files changed, 10 insertions(+), 15 deletions(-) diff --git a/src/pipeline/src/mng/error.rs b/src/pipeline/src/mng/error.rs index d65f5b44c5ed..c5a17a46a70f 100644 --- a/src/pipeline/src/mng/error.rs +++ b/src/pipeline/src/mng/error.rs @@ -14,7 +14,7 @@ use std::any::Any; -use common_error::ext::{BoxedError, ErrorExt}; +use common_error::ext::ErrorExt; use common_error::status_code::StatusCode; use common_macro::stack_trace_debug; use snafu::{Location, Snafu}; @@ -36,17 +36,16 @@ pub enum Error { location: Location, }, - #[snafu(display("Failed to insert pipeline to pipelines table, name: {}", name))] + #[snafu(display("Failed to insert pipeline to pipelines table"))] InsertPipeline { - name: String, - source: BoxedError, + source: operator::error::Error, #[snafu(implicit)] location: Location, }, - #[snafu(display("Failed to parse pipeline"))] + #[snafu(display("Failed to parse pipeline: {}", reason))] ParsePipeline { - source: BoxedError, + reason: String, #[snafu(implicit)] location: Location, }, diff --git a/src/pipeline/src/mng/mod.rs b/src/pipeline/src/mng/mod.rs index 06160869bfea..95ffb5822ec3 100644 --- a/src/pipeline/src/mng/mod.rs +++ b/src/pipeline/src/mng/mod.rs @@ -13,5 +13,5 @@ // limitations under the License. pub mod error; -pub mod table; pub mod pipeline_operator; +pub mod table; diff --git a/src/pipeline/src/mng/pipeline_operator.rs b/src/pipeline/src/mng/pipeline_operator.rs index f64391662bab..af1a814b5800 100644 --- a/src/pipeline/src/mng/pipeline_operator.rs +++ b/src/pipeline/src/mng/pipeline_operator.rs @@ -203,7 +203,7 @@ impl PipelineOperator { content_type, pipeline, ) - .await?; - Ok(()) + .await + .map(|_| ()) } } diff --git a/src/pipeline/src/mng/table.rs b/src/pipeline/src/mng/table.rs index 6034b85205ab..b7574238f569 100644 --- a/src/pipeline/src/mng/table.rs +++ b/src/pipeline/src/mng/table.rs @@ -20,8 +20,6 @@ use api::v1::{ ColumnDataType, ColumnDef, ColumnSchema as PbColumnSchema, Row, RowInsertRequest, RowInsertRequests, Rows, SemanticType, }; -use common_error::ext::{BoxedError, PlainError}; -use common_error::status_code::StatusCode; use common_query::OutputData; use common_recordbatch::util as record_util; use common_telemetry::info; @@ -202,8 +200,7 @@ impl PipelineTable { pub fn compile_pipeline(pipeline: &str) -> Result> { let yaml_content = Content::Yaml(pipeline.into()); parse::(&yaml_content) - .map_err(|e| BoxedError::new(PlainError::new(e, StatusCode::InvalidArguments))) - .context(ParsePipelineSnafu) + .map_err(|e| ParsePipelineSnafu { reason: e }.build()) } fn generate_pipeline_cache_key(schema: &str, name: &str) -> String { @@ -262,8 +259,7 @@ impl PipelineTable { &self.statement_executor, ) .await - .map_err(BoxedError::new) - .context(InsertPipelineSnafu { name })?; + .context(InsertPipelineSnafu)?; info!( "Inserted pipeline: {} into {} table: {}, output: {:?}.", From c110adb4d7f8ba443f560d94e6d9b8d06ba06bdb Mon Sep 17 00:00:00 2001 From: shuiyisong Date: Wed, 5 Jun 2024 10:38:35 +0800 Subject: [PATCH 25/51] refactor: bring in pipeline 1711f4d46687bada72426d88cda417899e0ae3a4 --- src/pipeline/src/etl/README.md | 13 + src/pipeline/src/etl/processor/dissect.rs | 392 +++++++++++++++++----- 2 files changed, 325 insertions(+), 80 deletions(-) create mode 100644 src/pipeline/src/etl/README.md diff --git a/src/pipeline/src/etl/README.md b/src/pipeline/src/etl/README.md new file mode 100644 index 000000000000..e512ea31f7b5 --- /dev/null +++ b/src/pipeline/src/etl/README.md @@ -0,0 +1,13 @@ +# pipeline + +ETL capability + +## processors + +refer [elastic ingest processor][elastic-ingest-processor] for detail + +### Example + +Go to [pipeline](../../tests/pipeline.rs) + +[elastic-ingest-processor]: https://www.elastic.co/guide/en/elasticsearch/reference/current/processors.html diff --git a/src/pipeline/src/etl/processor/dissect.rs b/src/pipeline/src/etl/processor/dissect.rs index 063996152a7d..005b104f5b14 100644 --- a/src/pipeline/src/etl/processor/dissect.rs +++ b/src/pipeline/src/etl/processor/dissect.rs @@ -12,7 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::{HashMap, HashSet}; + use common_telemetry::warn; +use itertools::Itertools; use crate::etl::field::{Field, Fields}; use crate::etl::processor::{ @@ -307,7 +310,8 @@ impl Pattern { return Err("Empty pattern is not allowed".to_string()); } - let mut map_items = std::collections::HashSet::new(); + let mut map_keys = HashSet::new(); + let mut map_vals = HashSet::new(); for i in 0..self.len() { let this_part = &self[i]; @@ -335,23 +339,23 @@ impl Pattern { } (Part::Name(name), _) => match name.start_modifier { Some(StartModifier::MapKey) => { - if map_items.contains(&name.name) { + if map_keys.contains(&name.name) { return Err(format!( "Invalid Pattern: '{}'. Duplicate map key: '{}'", self.origin, name.name )); } else { - map_items.insert(name.name.clone()); + map_keys.insert(&name.name); } } Some(StartModifier::MapVal) => { - if !map_items.contains(&name.name) { + if map_vals.contains(&name.name) { return Err(format!( - "Invalid Pattern: '{}'. Map key not found: '{}'", + "Invalid Pattern: '{}'. Duplicate map val: '{}'", self.origin, name.name )); } else { - map_items.remove(&name.name); + map_vals.insert(&name.name); } } _ => {} @@ -360,12 +364,12 @@ impl Pattern { } } - if !map_items.is_empty() { + if map_keys != map_vals { return Err(format!( - "Invalid Pattern: '{}'. Matched value not found for: '{}'", + "Invalid Pattern: '{}'. key and value not matched: '{}'", self.origin, - map_items - .iter() + map_keys + .symmetric_difference(&map_vals) .map(|s| s.as_str()) .collect::>() .join(",") @@ -409,10 +413,47 @@ impl DissectProcessor { self.append_separator = Some(append_separator); } - fn process_pattern(chs: &[char], pattern: &Pattern) -> Result { + fn process_pattern(&self, chs: &[char], pattern: &Pattern) -> Result { let mut map = Map::default(); let mut pos = 0; + let mut appends: HashMap> = HashMap::new(); + let mut maps: HashMap = HashMap::new(); + + let mut process_name_value = |name: &Name, value: String| { + let name_str = name.to_string(); + match name.start_modifier { + Some(StartModifier::NamedSkip) => { + // do nothing, ignore this match + } + Some(StartModifier::Append(order)) => { + appends + .entry(name_str) + .or_default() + .push((value, order.unwrap_or_default())); + } + Some(StartModifier::MapKey) => match maps.get(&name_str) { + Some(map_val) => { + map.insert(value, Value::String(map_val.to_string())); + } + None => { + maps.insert(name_str, value); + } + }, + Some(StartModifier::MapVal) => match maps.get(&name_str) { + Some(map_key) => { + map.insert(map_key, Value::String(value)); + } + None => { + maps.insert(name_str, value); + } + }, + None => { + map.insert(name.to_string(), Value::String(value)); + } + } + }; + for i in 0..pattern.len() { let this_part = &pattern[i]; let next_part = pattern.get(i + 1); @@ -434,38 +475,60 @@ impl DissectProcessor { pos += split_len; } + + (Part::Name(name1), Some(Part::Name(name2))) => { + return Err(format!( + "consecutive names are not allowed: '{name1}' '{name2}'" + )); + } + // if Name part is the last part, then the rest of the input is the value (Part::Name(name), None) => { let value = chs[pos..].iter().collect::(); - map.insert(name.to_string(), Value::String(value)); + process_name_value(name, value); } // if Name part, and next part is Split, then find the matched value of the name - (Part::Name(name), Some(Part::Split(split))) => match split.chars().next() { - None => return Err("Empty split is not allowed".to_string()), - Some(stop) => { - let mut end = pos; - while end < chs.len() && chs[end] != stop { - end += 1; - } - - if end == chs.len() { - return Err("No matching split found".to_string()); - } + (Part::Name(name), Some(Part::Split(split))) => { + let stop = split + .chars() + .next() + .ok_or("Empty split is not allowed".to_string())?; // this won't happen + let mut end = pos; + while end < chs.len() && chs[end] != stop { + end += 1; + } + if !name.is_name_empty() { let value = chs[pos..end].iter().collect::(); - map.insert(name.to_string(), Value::String(value)); - pos = end; + process_name_value(name, value); } - }, - (Part::Name(name1), Some(Part::Name(name2))) => { - return Err(format!( - "consecutive names are not allowed: '{name1}' '{name2}'" - )); + + if name.is_end_modifier_set() { + while end < chs.len() && chs[end] == stop { + end += 1; + } + end -= 1; // leave the last stop character to match the next split + } + + pos = end; } } } + if !appends.is_empty() { + let sep = match self.append_separator { + Some(ref sep) => sep, + None => " ", + }; + + for (name, mut values) in appends { + values.sort_by(|a, b| a.1.cmp(&b.1)); + let value = values.into_iter().map(|(a, _)| a).join(sep); + map.insert(name, Value::String(value)); + } + } + Ok(map) } @@ -473,7 +536,7 @@ impl DissectProcessor { let chs = val.chars().collect::>(); for pattern in &self.patterns { - if let Ok(map) = DissectProcessor::process_pattern(&chs, pattern) { + if let Ok(map) = self.process_pattern(&chs, pattern) { return Ok(map); } } @@ -552,9 +615,19 @@ fn is_valid_char(ch: char) -> bool { mod tests { use std::collections::HashMap; - use super::{DissectProcessor, EndModifier, Part, Pattern}; + use super::{DissectProcessor, EndModifier, Name, Part, Pattern, StartModifier}; use crate::etl::value::{Map, Value}; + fn assert(pattern_str: &str, input: &str, expected: HashMap) { + let chs = input.chars().collect::>(); + let pattern = pattern_str.parse().unwrap(); + + let processor = DissectProcessor::default(); + let map = processor.process_pattern(&chs, &pattern).unwrap(); + + assert_eq!(map, Map::from(expected), "pattern: {}", pattern_str); + } + #[test] fn test_dissect_simple_pattern() { let cases = [( @@ -589,10 +662,26 @@ mod tests { #[test] fn test_dissect_modifier_pattern() { let cases = [ + ( + "%{} %{}", + vec![ + Part::Name(Name { + name: "".into(), + start_modifier: None, + end_modifier: None, + }), + Part::Split(" ".into()), + Part::Name(Name { + name: "".into(), + start_modifier: None, + end_modifier: None, + }), + ], + ), ( "%{ts->} %{level}", vec![ - Part::Name(super::Name { + Part::Name(Name { name: "ts".into(), start_modifier: None, end_modifier: Some(EndModifier), @@ -605,19 +694,19 @@ mod tests { "[%{ts}]%{->}[%{level}]", vec![ Part::Split("[".into()), - Part::Name(super::Name { + Part::Name(Name { name: "ts".into(), start_modifier: None, end_modifier: None, }), Part::Split("]".into()), - Part::Name(super::Name { + Part::Name(Name { name: "".into(), start_modifier: None, end_modifier: Some(EndModifier), }), Part::Split("[".into()), - Part::Name(super::Name { + Part::Name(Name { name: "level".into(), start_modifier: None, end_modifier: None, @@ -628,27 +717,27 @@ mod tests { ( "%{+name} %{+name} %{+name} %{+name}", vec![ - Part::Name(super::Name { + Part::Name(Name { name: "name".into(), - start_modifier: Some(super::StartModifier::Append(None)), + start_modifier: Some(StartModifier::Append(None)), end_modifier: None, }), Part::Split(" ".into()), - Part::Name(super::Name { + Part::Name(Name { name: "name".into(), - start_modifier: Some(super::StartModifier::Append(None)), + start_modifier: Some(StartModifier::Append(None)), end_modifier: None, }), Part::Split(" ".into()), - Part::Name(super::Name { + Part::Name(Name { name: "name".into(), - start_modifier: Some(super::StartModifier::Append(None)), + start_modifier: Some(StartModifier::Append(None)), end_modifier: None, }), Part::Split(" ".into()), - Part::Name(super::Name { + Part::Name(Name { name: "name".into(), - start_modifier: Some(super::StartModifier::Append(None)), + start_modifier: Some(StartModifier::Append(None)), end_modifier: None, }), ], @@ -656,27 +745,27 @@ mod tests { ( "%{+name/2} %{+name/4} %{+name/3} %{+name/1}", vec![ - Part::Name(super::Name { + Part::Name(Name { name: "name".into(), - start_modifier: Some(super::StartModifier::Append(Some(2))), + start_modifier: Some(StartModifier::Append(Some(2))), end_modifier: None, }), Part::Split(" ".into()), - Part::Name(super::Name { + Part::Name(Name { name: "name".into(), - start_modifier: Some(super::StartModifier::Append(Some(4))), + start_modifier: Some(StartModifier::Append(Some(4))), end_modifier: None, }), Part::Split(" ".into()), - Part::Name(super::Name { + Part::Name(Name { name: "name".into(), - start_modifier: Some(super::StartModifier::Append(Some(3))), + start_modifier: Some(StartModifier::Append(Some(3))), end_modifier: None, }), Part::Split(" ".into()), - Part::Name(super::Name { + Part::Name(Name { name: "name".into(), - start_modifier: Some(super::StartModifier::Append(Some(1))), + start_modifier: Some(StartModifier::Append(Some(1))), end_modifier: None, }), ], @@ -684,25 +773,25 @@ mod tests { ( "%{clientip} %{?ident} %{?auth} [%{timestamp}]", vec![ - Part::Name(super::Name { + Part::Name(Name { name: "clientip".into(), start_modifier: None, end_modifier: None, }), Part::Split(" ".into()), - Part::Name(super::Name { + Part::Name(Name { name: "ident".into(), - start_modifier: Some(super::StartModifier::NamedSkip), + start_modifier: Some(StartModifier::NamedSkip), end_modifier: None, }), Part::Split(" ".into()), - Part::Name(super::Name { + Part::Name(Name { name: "auth".into(), - start_modifier: Some(super::StartModifier::NamedSkip), + start_modifier: Some(StartModifier::NamedSkip), end_modifier: None, }), Part::Split(" [".into()), - Part::Name(super::Name { + Part::Name(Name { name: "timestamp".into(), start_modifier: None, end_modifier: None, @@ -714,39 +803,55 @@ mod tests { "[%{ts}] [%{level}] %{*p1}:%{&p1} %{*p2}:%{&p2}", vec![ Part::Split("[".into()), - Part::Name(super::Name { + Part::Name(Name { name: "ts".into(), start_modifier: None, end_modifier: None, }), Part::Split("] [".into()), - Part::Name(super::Name { + Part::Name(Name { name: "level".into(), start_modifier: None, end_modifier: None, }), Part::Split("] ".into()), - Part::Name(super::Name { + Part::Name(Name { name: "p1".into(), - start_modifier: Some(super::StartModifier::MapKey), + start_modifier: Some(StartModifier::MapKey), end_modifier: None, }), Part::Split(":".into()), - Part::Name(super::Name { + Part::Name(Name { name: "p1".into(), - start_modifier: Some(super::StartModifier::MapVal), + start_modifier: Some(StartModifier::MapVal), end_modifier: None, }), Part::Split(" ".into()), - Part::Name(super::Name { + Part::Name(Name { name: "p2".into(), - start_modifier: Some(super::StartModifier::MapKey), + start_modifier: Some(StartModifier::MapKey), end_modifier: None, }), Part::Split(":".into()), - Part::Name(super::Name { + Part::Name(Name { name: "p2".into(), - start_modifier: Some(super::StartModifier::MapVal), + start_modifier: Some(StartModifier::MapVal), + end_modifier: None, + }), + ], + ), + ( + "%{&p1}:%{*p1}", + vec![ + Part::Name(Name { + name: "p1".into(), + start_modifier: Some(StartModifier::MapVal), + end_modifier: None, + }), + Part::Split(":".into()), + Part::Name(Name { + name: "p1".into(), + start_modifier: Some(StartModifier::MapKey), end_modifier: None, }), ], @@ -821,15 +926,15 @@ mod tests { ), ( "%{*ip}", - "Invalid Pattern: '%{*ip}'. Matched value not found for: 'ip'", + "Invalid Pattern: '%{*ip}'. key and value not matched: 'ip'" ), ( "%{*ip} %{*ip}", "Invalid Pattern: '%{*ip} %{*ip}'. Duplicate map key: 'ip'", ), ( - "%{*ip} %{&ip2}", - "Invalid Pattern: '%{*ip} %{&ip2}'. Map key not found: 'ip2'", + "%{*ip1} %{&ip2}", + "Invalid Pattern: '%{*ip1} %{&ip2}'. key and value not matched: 'ip1,ip2'" ), ]; @@ -841,14 +946,6 @@ mod tests { #[test] fn test_dissect_process() { - let assert = |pattern_str: &str, input: &str, expected: HashMap| { - let chs = input.chars().collect::>(); - let pattern = pattern_str.parse().unwrap(); - let map = DissectProcessor::process_pattern(&chs, &pattern).unwrap(); - - assert_eq!(map, Map::from(expected)); - }; - let expected = [ ("timestamp", "30/Apr/1998:22:00:52 +0000"), ("status", "200"), @@ -865,7 +962,7 @@ mod tests { ] .into_iter() .map(|(k, v)| (k.to_string(), Value::String(v.to_string()))) - .collect::>(); + .collect::>(); { // pattern start with Name @@ -883,4 +980,139 @@ mod tests { assert(pattern_str, input, expected); } } + + #[test] + fn test_dissect_right_padding_modifier() { + let cases = [ + ( + "%{ts->} %{level}", + "1998-08-10T17:15:42,466 WARN", + [("ts", "1998-08-10T17:15:42,466"), ("level", "WARN")], + ), + ( + "[%{ts}]%{->}[%{level}]", + "[1998-08-10T17:15:42,466] [WARN]", + [("ts", "1998-08-10T17:15:42,466"), ("level", "WARN")], + ), + ( + "[%{ts}]%{->}[%{level}]", + "[1998-08-10T17:15:42,466] [[[[WARN]", + [("ts", "1998-08-10T17:15:42,466"), ("level", "WARN")], + ), + ] + .into_iter() + .map(|(pattern, input, expected)| { + let map = expected + .into_iter() + .map(|(k, v)| (k.to_string(), Value::String(v.to_string()))); + (pattern, input, map) + }); + + for (pattern_str, input, expected) in cases { + assert( + pattern_str, + input, + expected.collect::>(), + ); + } + } + + #[test] + fn test_dissect_append_modifier() { + let cases = [ + ( + "%{+name} %{+name} %{+name} %{+name}", + "john jacob jingleheimer schmidt", + [("name", "john jacob jingleheimer schmidt")], + ), + ( + "%{+name/2} %{+name/4} %{+name/3} %{+name/1}", + "john jacob jingleheimer schmidt", + [("name", "schmidt john jingleheimer jacob")], + ), + ] + .into_iter() + .map(|(pattern, input, expected)| { + let map = expected + .into_iter() + .map(|(k, v)| (k.to_string(), Value::String(v.to_string()))); + (pattern, input, map) + }); + + for (pattern_str, input, expected) in cases { + assert( + pattern_str, + input, + expected.collect::>(), + ); + } + } + + #[test] + fn test_dissect_named_skip_modifier() { + let cases = [( + "%{clientip} %{?ident} %{?auth} [%{timestamp}]", + "1.2.3.4 - - [30/Apr/1998:22:00:52 +0000]", + [ + ("clientip", "1.2.3.4"), + ("timestamp", "30/Apr/1998:22:00:52 +0000"), + ], + )] + .into_iter() + .map(|(pattern, input, expected)| { + let map = expected + .into_iter() + .map(|(k, v)| (k.to_string(), Value::String(v.to_string()))); + (pattern, input, map) + }); + + for (pattern_str, input, expected) in cases { + assert( + pattern_str, + input, + expected.collect::>(), + ); + } + } + + #[test] + fn test_dissect_reference_keys() { + let cases = [ + ( + "[%{ts}] [%{level}] %{*p1}:%{&p1} %{*p2}:%{&p2}", + "[2018-08-10T17:15:42,466] [ERR] ip:1.2.3.4 error:REFUSED", + [ + ("ts", "2018-08-10T17:15:42,466"), + ("level", "ERR"), + ("ip", "1.2.3.4"), + ("error", "REFUSED"), + ], + ), + ( + "[%{ts}] [%{level}] %{&p1}:%{*p1} %{*p2}:%{&p2}", + "[2018-08-10T17:15:42,466] [ERR] ip:1.2.3.4 error:REFUSED", + [ + ("ts", "2018-08-10T17:15:42,466"), + ("level", "ERR"), + ("1.2.3.4", "ip"), + ("error", "REFUSED"), + ], + ), + ] + .into_iter() + .map(|(pattern, input, expected)| { + let map = expected + .into_iter() + .map(|(k, v)| (k.to_string(), Value::String(v.to_string()))); + (pattern, input, map) + }); + + for (pattern_str, input, expected) in cases { + assert( + pattern_str, + input, + expected.collect::>(), + ); + } + } } From 1047dd7b27db525e9eb916238da5330a0c9d908c Mon Sep 17 00:00:00 2001 From: shuiyisong Date: Wed, 5 Jun 2024 11:32:35 +0800 Subject: [PATCH 26/51] chore: add log --- src/servers/src/http/event.rs | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/servers/src/http/event.rs b/src/servers/src/http/event.rs index 908160c17f39..6f6a67d93fb5 100644 --- a/src/servers/src/http/event.rs +++ b/src/servers/src/http/event.rs @@ -18,7 +18,7 @@ use api::v1::{RowInsertRequest, RowInsertRequests, Rows}; use axum::extract::{Json, Query, State, TypedHeader}; use axum::headers::ContentType; use axum::Extension; -use common_telemetry::error; +use common_telemetry::{error, info}; use pipeline::Value as PipelineValue; use schemars::JsonSchema; use serde::{Deserialize, Serialize}; @@ -74,6 +74,9 @@ pub async fn log_ingester( TypedHeader(content_type): TypedHeader, payload: String, ) -> Result { + // TODO(shuiyisong): remove debug log + info!("[log_input]: {}", payload); + let value; // TODO (qtang): we should decide json or jsonl if content_type == ContentType::json() { From 2ff2fda44b3ab8fb17457977aa0df4ba373e7805 Mon Sep 17 00:00:00 2001 From: shuiyisong Date: Wed, 5 Jun 2024 11:54:17 +0800 Subject: [PATCH 27/51] chore: add log --- src/servers/src/http/event.rs | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/src/servers/src/http/event.rs b/src/servers/src/http/event.rs index 6f6a67d93fb5..496a0962bcc4 100644 --- a/src/servers/src/http/event.rs +++ b/src/servers/src/http/event.rs @@ -15,10 +15,12 @@ use std::collections::HashMap; use api::v1::{RowInsertRequest, RowInsertRequests, Rows}; -use axum::extract::{Json, Query, State, TypedHeader}; +use axum::extract::{Json, Query, State}; use axum::headers::ContentType; use axum::Extension; use common_telemetry::{error, info}; +use headers::HeaderMapExt; +use http::HeaderMap; use pipeline::Value as PipelineValue; use schemars::JsonSchema; use serde::{Deserialize, Serialize}; @@ -71,11 +73,17 @@ pub async fn log_ingester( State(state): State, Query(query_params): Query, Extension(query_ctx): Extension, - TypedHeader(content_type): TypedHeader, + // TypedHeader(content_type): TypedHeader, + headers: HeaderMap, payload: String, ) -> Result { // TODO(shuiyisong): remove debug log - info!("[log_input]: {}", payload); + info!("[log_header]: {:?}", headers); + info!("[log_payload]: {:?}", payload); + + let content_type = headers + .typed_get::() + .unwrap_or(ContentType::text()); let value; // TODO (qtang): we should decide json or jsonl From 8b6a65240149b086d9e15ae389f8145a59f70caf Mon Sep 17 00:00:00 2001 From: shuiyisong Date: Wed, 5 Jun 2024 12:48:12 +0800 Subject: [PATCH 28/51] chore: remove open hook --- src/pipeline/src/mng/pipeline_operator.rs | 11 +---------- 1 file changed, 1 insertion(+), 10 deletions(-) diff --git a/src/pipeline/src/mng/pipeline_operator.rs b/src/pipeline/src/mng/pipeline_operator.rs index af1a814b5800..0555321a7189 100644 --- a/src/pipeline/src/mng/pipeline_operator.rs +++ b/src/pipeline/src/mng/pipeline_operator.rs @@ -87,7 +87,7 @@ impl PipelineOperator { let RegisterSystemTableRequest { create_table_expr: mut expr, - open_hook, + open_hook: _, } = self.create_table_request(catalog); if let Some(table) = self @@ -96,12 +96,7 @@ impl PipelineOperator { .await .context(CatalogSnafu)? { - if let Some(open_hook) = open_hook { - (open_hook)(table.clone()).await.context(CatalogSnafu)?; - } - self.add_pipeline_table_to_cache(catalog, table); - return Ok(()); } @@ -124,10 +119,6 @@ impl PipelineOperator { .context(CatalogSnafu)? .context(PipelineTableNotFoundSnafu)?; - if let Some(open_hook) = open_hook { - (open_hook)(table.clone()).await.context(CatalogSnafu)?; - } - info!( "Created scripts table {}.", table.table_info().full_table_name() From 1298b0aac08ac3bed136e844c6e47535d0eeaea8 Mon Sep 17 00:00:00 2001 From: shuiyisong Date: Wed, 5 Jun 2024 16:13:26 +0800 Subject: [PATCH 29/51] chore: minor update --- src/pipeline/src/mng/error.rs | 13 +++- src/pipeline/src/mng/pipeline_operator.rs | 41 +++++------ src/servers/src/http/event.rs | 85 +++++++++++------------ 3 files changed, 73 insertions(+), 66 deletions(-) diff --git a/src/pipeline/src/mng/error.rs b/src/pipeline/src/mng/error.rs index c5a17a46a70f..4a62aea9932a 100644 --- a/src/pipeline/src/mng/error.rs +++ b/src/pipeline/src/mng/error.rs @@ -99,6 +99,13 @@ pub enum Error { #[snafu(implicit)] location: Location, }, + + #[snafu(display("Failed to execute pipeline, reason: {}", reason))] + ExecPipeline { + reason: String, + #[snafu(implicit)] + location: Location, + }, } pub type Result = std::result::Result; @@ -113,7 +120,11 @@ impl ErrorExt for Error { CollectRecords { source, .. } => source.status_code(), PipelineNotFound { .. } => StatusCode::InvalidArguments, ParsePipeline { .. } => StatusCode::InvalidArguments, - BuildDfLogicalPlan { .. } => StatusCode::Internal, + BuildDfLogicalPlan { .. } + // should we put it internal? + // since pipeline is already compiled + // it's most likely an user input error + | ExecPipeline { .. } => StatusCode::Internal, ExecuteInternalStatement { source, .. } => source.status_code(), Catalog { source, .. } => source.status_code(), CreateTable { source, .. } => source.status_code(), diff --git a/src/pipeline/src/mng/pipeline_operator.rs b/src/pipeline/src/mng/pipeline_operator.rs index 0555321a7189..f137cdfd769f 100644 --- a/src/pipeline/src/mng/pipeline_operator.rs +++ b/src/pipeline/src/mng/pipeline_operator.rs @@ -22,7 +22,7 @@ use common_telemetry::info; use operator::insert::InserterRef; use operator::statement::StatementExecutorRef; use query::QueryEngineRef; -use session::context::{QueryContext, QueryContextRef}; +use session::context::QueryContextRef; use snafu::{OptionExt, ResultExt}; use table::TableRef; @@ -80,7 +80,11 @@ impl PipelineOperator { ); } - async fn create_pipeline_table_if_not_exists(&self, catalog: &str) -> Result<()> { + async fn create_pipeline_table_if_not_exists(&self, ctx: QueryContextRef) -> Result<()> { + let catalog_str = ctx.current_catalog().to_owned(); + let catalog = catalog_str.as_str(); + + // exist in cache if self.get_pipeline_table_from_cache(catalog).is_some() { return Ok(()); } @@ -90,6 +94,7 @@ impl PipelineOperator { open_hook: _, } = self.create_table_request(catalog); + // exist in catalog, just open if let Some(table) = self .catalog_manager .table(&expr.catalog_name, &expr.schema_name, &expr.table_name) @@ -103,15 +108,13 @@ impl PipelineOperator { let schema = expr.schema_name.clone(); let table_name = expr.table_name.clone(); + // create table self.statement_executor - .create_table_inner( - &mut expr, - None, - Arc::new(QueryContext::with(catalog, &schema)), - ) + .create_table_inner(&mut expr, None, ctx) .await .context(CreateTableSnafu)?; + // get from catalog let table = self .catalog_manager .table(catalog, &schema, &table_name) @@ -124,6 +127,7 @@ impl PipelineOperator { table.table_info().full_table_name() ); + // put to cache self.add_pipeline_table_to_cache(catalog, table); Ok(()) @@ -135,15 +139,14 @@ impl PipelineOperator { async fn insert_and_compile( &self, - catalog: &str, - schema: &str, + ctx: QueryContextRef, name: &str, content_type: &str, pipeline: &str, ) -> Result> { - self.get_pipeline_table_from_cache(catalog) + self.get_pipeline_table_from_cache(ctx.current_catalog()) .context(PipelineTableNotFoundSnafu)? - .insert_and_compile(schema, name, content_type, pipeline) + .insert_and_compile(ctx.current_schema(), name, content_type, pipeline) .await } } @@ -169,7 +172,7 @@ impl PipelineOperator { query_ctx: QueryContextRef, name: &str, ) -> Result> { - self.create_pipeline_table_if_not_exists(query_ctx.current_catalog()) + self.create_pipeline_table_if_not_exists(query_ctx.clone()) .await?; self.get_pipeline_table_from_cache(query_ctx.current_catalog()) .context(PipelineTableNotFoundSnafu)? @@ -184,17 +187,11 @@ impl PipelineOperator { pipeline: &str, query_ctx: QueryContextRef, ) -> Result<()> { - self.create_pipeline_table_if_not_exists(query_ctx.current_catalog()) + self.create_pipeline_table_if_not_exists(query_ctx.clone()) .await?; - self.insert_and_compile( - query_ctx.current_catalog(), - query_ctx.current_schema(), - name, - content_type, - pipeline, - ) - .await - .map(|_| ()) + self.insert_and_compile(query_ctx, name, content_type, pipeline) + .await + .map(|_| ()) } } diff --git a/src/servers/src/http/event.rs b/src/servers/src/http/event.rs index 496a0962bcc4..4d1ebe2e2172 100644 --- a/src/servers/src/http/event.rs +++ b/src/servers/src/http/event.rs @@ -21,6 +21,8 @@ use axum::Extension; use common_telemetry::{error, info}; use headers::HeaderMapExt; use http::HeaderMap; +use mime_guess::mime; +use pipeline::error::{CastTypeSnafu, ExecPipelineSnafu}; use pipeline::Value as PipelineValue; use schemars::JsonSchema; use serde::{Deserialize, Serialize}; @@ -29,7 +31,7 @@ use session::context::QueryContextRef; use snafu::{OptionExt, ResultExt}; use crate::error::{ - InsertLogSnafu, InvalidParameterSnafu, ParseJsonSnafu, Result, UnsupportedContentTypeSnafu, + InvalidParameterSnafu, ParseJsonSnafu, PipelineSnafu, Result, UnsupportedContentTypeSnafu, }; use crate::http::greptime_result_v1::GreptimedbV1Response; use crate::http::HttpResponse; @@ -70,7 +72,7 @@ pub async fn add_pipeline( #[axum_macros::debug_handler] pub async fn log_ingester( - State(state): State, + State(handler): State, Query(query_params): Query, Extension(query_ctx): Extension, // TypedHeader(content_type): TypedHeader, @@ -80,25 +82,29 @@ pub async fn log_ingester( // TODO(shuiyisong): remove debug log info!("[log_header]: {:?}", headers); info!("[log_payload]: {:?}", payload); - let content_type = headers .typed_get::() .unwrap_or(ContentType::text()); - let value; - // TODO (qtang): we should decide json or jsonl - if content_type == ContentType::json() { - value = serde_json::from_str(&payload).context(ParseJsonSnafu)?; - // TODO (qtang): we should decide which content type to support - // form_url_cncoded type is only placeholder - } else if content_type == ContentType::form_url_encoded() { - value = parse_space_separated_log(payload)?; - } else { - return UnsupportedContentTypeSnafu { content_type }.fail(); - } - log_ingester_inner(state, query_params, query_ctx, value) - .await - .or_else(|e| InsertLogSnafu { msg: e }.fail()) + let pipeline_name = query_params.pipeline_name.context(InvalidParameterSnafu { + reason: "pipeline_name is required", + })?; + let table_name = query_params.table_name.context(InvalidParameterSnafu { + reason: "table_name is required", + })?; + + let m: mime::Mime = content_type.clone().into(); + let value = match m.subtype() { + // TODO (qtang): we should decide json or jsonl + mime::JSON => serde_json::from_str(&payload).context(ParseJsonSnafu)?, + // TODO (qtang): we should decide which content type to support + // form_url_cncoded type is only placeholder + mime::WWW_FORM_URLENCODED => parse_space_separated_log(payload)?, + // add more content type support + _ => UnsupportedContentTypeSnafu { content_type }.fail()?, + }; + + log_ingester_inner(handler, pipeline_name, table_name, value, query_ctx).await } fn parse_space_separated_log(payload: String) -> Result { @@ -110,25 +116,22 @@ fn parse_space_separated_log(payload: String) -> Result { async fn log_ingester_inner( state: LogHandlerRef, - query_params: LogIngesterQueryParams, - query_ctx: QueryContextRef, + pipeline_name: String, + table_name: String, payload: Value, -) -> std::result::Result { - let pipeline_id = query_params - .pipeline_name - .ok_or("pipeline_name is required".to_string())?; - - let pipeline_data = PipelineValue::try_from(payload)?; + query_ctx: QueryContextRef, +) -> Result { + let pipeline_data = PipelineValue::try_from(payload) + .map_err(|reason| CastTypeSnafu { msg: reason }.build()) + .context(PipelineSnafu)?; let pipeline = state - .get_pipeline(&pipeline_id, query_ctx.clone()) - .await - .map_err(|e| e.to_string())?; - let transformed_data: Rows = pipeline.exec(pipeline_data)?; - - let table_name = query_params - .table_name - .ok_or("table_name is required".to_string())?; + .get_pipeline(&pipeline_name, query_ctx.clone()) + .await?; + let transformed_data: Rows = pipeline + .exec(pipeline_data) + .map_err(|reason| ExecPipelineSnafu { reason }.build()) + .context(PipelineSnafu)?; let insert_request = RowInsertRequest { rows: Some(transformed_data), @@ -137,15 +140,11 @@ async fn log_ingester_inner( let insert_requests = RowInsertRequests { inserts: vec![insert_request], }; - state - .insert_log(insert_requests, query_ctx) - .await - .map(|_| { - HttpResponse::GreptimedbV1(GreptimedbV1Response { - output: vec![], - execution_time_ms: 0, - resp_metrics: HashMap::new(), - }) + state.insert_log(insert_requests, query_ctx).await.map(|_| { + HttpResponse::GreptimedbV1(GreptimedbV1Response { + output: vec![], + execution_time_ms: 0, + resp_metrics: HashMap::new(), }) - .map_err(|e| e.to_string()) + }) } From ea548b0304a0a50e80f65793a03809984ec09610 Mon Sep 17 00:00:00 2001 From: shuiyisong Date: Wed, 5 Jun 2024 16:19:47 +0800 Subject: [PATCH 30/51] chore: fix fmt --- src/pipeline/src/mng/error.rs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/pipeline/src/mng/error.rs b/src/pipeline/src/mng/error.rs index 4a62aea9932a..1b67bd5055ba 100644 --- a/src/pipeline/src/mng/error.rs +++ b/src/pipeline/src/mng/error.rs @@ -120,11 +120,10 @@ impl ErrorExt for Error { CollectRecords { source, .. } => source.status_code(), PipelineNotFound { .. } => StatusCode::InvalidArguments, ParsePipeline { .. } => StatusCode::InvalidArguments, - BuildDfLogicalPlan { .. } - // should we put it internal? + // should we put `ExecPipeline` in internal? // since pipeline is already compiled // it's most likely an user input error - | ExecPipeline { .. } => StatusCode::Internal, + BuildDfLogicalPlan { .. } | ExecPipeline { .. } => StatusCode::Internal, ExecuteInternalStatement { source, .. } => source.status_code(), Catalog { source, .. } => source.status_code(), CreateTable { source, .. } => source.status_code(), From 6c88b89130fbc37fdc1d40517b24747cd9609dff Mon Sep 17 00:00:00 2001 From: shuiyisong Date: Wed, 5 Jun 2024 17:07:28 +0800 Subject: [PATCH 31/51] chore: minor update --- src/frontend/src/instance/log_handler.rs | 1 + src/pipeline/src/mng/table.rs | 18 ++++++++-------- src/servers/src/http/event.rs | 26 +++--------------------- 3 files changed, 14 insertions(+), 31 deletions(-) diff --git a/src/frontend/src/instance/log_handler.rs b/src/frontend/src/instance/log_handler.rs index 26e37498e591..ecc22b4935ec 100644 --- a/src/frontend/src/instance/log_handler.rs +++ b/src/frontend/src/instance/log_handler.rs @@ -66,6 +66,7 @@ impl LogHandler for Instance { } async fn delete_pipeline(&self, _name: &str, _query_ctx: QueryContextRef) -> ServerResult<()> { + // todo(qtang): impl delete todo!("delete_pipeline") } } diff --git a/src/pipeline/src/mng/table.rs b/src/pipeline/src/mng/table.rs index b7574238f569..fb1a78a1e51b 100644 --- a/src/pipeline/src/mng/table.rs +++ b/src/pipeline/src/mng/table.rs @@ -350,24 +350,26 @@ impl PipelineTable { .context(CollectRecordsSnafu)?; ensure!(!records.is_empty(), PipelineNotFoundSnafu { name }); + // assume schema + name is unique for now + ensure!( + records.len() == 1 && records[0].num_columns() == 1, + PipelineNotFoundSnafu { name } + ); - assert_eq!(records.len(), 1); - assert_eq!(records[0].num_columns(), 1); - - let script_column = records[0].column(0); - let script_column = script_column + let pipeline_column = records[0].column(0); + let pipeline_column = pipeline_column .as_any() .downcast_ref::() .with_context(|| CastTypeSnafu { msg: format!( "can't downcast {:?} array into string vector", - script_column.data_type() + pipeline_column.data_type() ), })?; - assert_eq!(script_column.len(), 1); + ensure!(pipeline_column.len() == 1, PipelineNotFoundSnafu { name }); // Safety: asserted above - Ok(script_column.get_data(0).unwrap().to_string()) + Ok(pipeline_column.get_data(0).unwrap().to_string()) } } diff --git a/src/servers/src/http/event.rs b/src/servers/src/http/event.rs index 4d1ebe2e2172..a08688b55309 100644 --- a/src/servers/src/http/event.rs +++ b/src/servers/src/http/event.rs @@ -17,10 +17,8 @@ use std::collections::HashMap; use api::v1::{RowInsertRequest, RowInsertRequests, Rows}; use axum::extract::{Json, Query, State}; use axum::headers::ContentType; -use axum::Extension; -use common_telemetry::{error, info}; -use headers::HeaderMapExt; -use http::HeaderMap; +use axum::{Extension, TypedHeader}; +use common_telemetry::error; use mime_guess::mime; use pipeline::error::{CastTypeSnafu, ExecPipelineSnafu}; use pipeline::Value as PipelineValue; @@ -75,17 +73,9 @@ pub async fn log_ingester( State(handler): State, Query(query_params): Query, Extension(query_ctx): Extension, - // TypedHeader(content_type): TypedHeader, - headers: HeaderMap, + TypedHeader(content_type): TypedHeader, payload: String, ) -> Result { - // TODO(shuiyisong): remove debug log - info!("[log_header]: {:?}", headers); - info!("[log_payload]: {:?}", payload); - let content_type = headers - .typed_get::() - .unwrap_or(ContentType::text()); - let pipeline_name = query_params.pipeline_name.context(InvalidParameterSnafu { reason: "pipeline_name is required", })?; @@ -97,9 +87,6 @@ pub async fn log_ingester( let value = match m.subtype() { // TODO (qtang): we should decide json or jsonl mime::JSON => serde_json::from_str(&payload).context(ParseJsonSnafu)?, - // TODO (qtang): we should decide which content type to support - // form_url_cncoded type is only placeholder - mime::WWW_FORM_URLENCODED => parse_space_separated_log(payload)?, // add more content type support _ => UnsupportedContentTypeSnafu { content_type }.fail()?, }; @@ -107,13 +94,6 @@ pub async fn log_ingester( log_ingester_inner(handler, pipeline_name, table_name, value, query_ctx).await } -fn parse_space_separated_log(payload: String) -> Result { - // ToStructuredLogSnafu - let _log = payload.split_whitespace().collect::>(); - // TODO (qtang): implement this - todo!() -} - async fn log_ingester_inner( state: LogHandlerRef, pipeline_name: String, From eeed85ebdf7e8b9995af9cbaa41e87916c82a1ca Mon Sep 17 00:00:00 2001 From: shuiyisong Date: Wed, 5 Jun 2024 17:13:12 +0800 Subject: [PATCH 32/51] chore: rename desc for pipeline table --- src/pipeline/src/mng/pipeline_operator.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/pipeline/src/mng/pipeline_operator.rs b/src/pipeline/src/mng/pipeline_operator.rs index f137cdfd769f..8540fa797be6 100644 --- a/src/pipeline/src/mng/pipeline_operator.rs +++ b/src/pipeline/src/mng/pipeline_operator.rs @@ -48,7 +48,7 @@ impl PipelineOperator { catalog_name: catalog.to_string(), schema_name: DEFAULT_PRIVATE_SCHEMA_NAME.to_string(), table_name: PIPELINE_TABLE_NAME.to_string(), - desc: "GreptimeDB scripts table for Python".to_string(), + desc: "GreptimeDB pipeline table for Log".to_string(), column_defs, time_index, primary_keys, From f77d20bc181f77dcf06a8a3eca30788b0e982d19 Mon Sep 17 00:00:00 2001 From: shuiyisong Date: Wed, 5 Jun 2024 17:23:23 +0800 Subject: [PATCH 33/51] refactor: remove updated_at in pipelines --- src/pipeline/src/mng/table.rs | 17 ----------------- 1 file changed, 17 deletions(-) diff --git a/src/pipeline/src/mng/table.rs b/src/pipeline/src/mng/table.rs index fb1a78a1e51b..040924f799de 100644 --- a/src/pipeline/src/mng/table.rs +++ b/src/pipeline/src/mng/table.rs @@ -80,7 +80,6 @@ impl PipelineTable { let content_type = "content_type"; let pipeline_content = "pipeline"; let created_at = "created_at"; - let updated_at = "updated_at"; ( created_at.to_string(), @@ -135,15 +134,6 @@ impl PipelineTable { comment: "".to_string(), datatype_extension: None, }, - ColumnDef { - name: updated_at.to_string(), - data_type: ColumnDataType::TimestampMillisecond as i32, - is_nullable: false, - default_constraint: vec![], - semantic_type: SemanticType::Field as i32, - comment: "".to_string(), - datatype_extension: None, - }, ], ) } @@ -180,12 +170,6 @@ impl PipelineTable { semantic_type: SemanticType::Timestamp.into(), ..Default::default() }, - PbColumnSchema { - column_name: "updated_at".to_string(), - datatype: ColumnDataType::TimestampMillisecond.into(), - semantic_type: SemanticType::Field.into(), - ..Default::default() - }, ] } @@ -241,7 +225,6 @@ impl PipelineTable { ValueData::StringValue(content_type.to_string()).into(), ValueData::StringValue(pipeline.to_string()).into(), ValueData::TimestampMillisecondValue(now).into(), - ValueData::TimestampMillisecondValue(now).into(), ], }], }), From 581567530d5c53f144aa17e3c1a1418c3048474a Mon Sep 17 00:00:00 2001 From: paomian Date: Wed, 5 Jun 2024 17:38:18 +0800 Subject: [PATCH 34/51] chore: add more content type support for log inserter api --- src/servers/src/http/event.rs | 57 ++++++++++++++++++++++++++++++----- 1 file changed, 50 insertions(+), 7 deletions(-) diff --git a/src/servers/src/http/event.rs b/src/servers/src/http/event.rs index a08688b55309..7c1e01f0ff2c 100644 --- a/src/servers/src/http/event.rs +++ b/src/servers/src/http/event.rs @@ -18,13 +18,13 @@ use api::v1::{RowInsertRequest, RowInsertRequests, Rows}; use axum::extract::{Json, Query, State}; use axum::headers::ContentType; use axum::{Extension, TypedHeader}; -use common_telemetry::error; +use common_telemetry::{error, warn}; use mime_guess::mime; use pipeline::error::{CastTypeSnafu, ExecPipelineSnafu}; use pipeline::Value as PipelineValue; use schemars::JsonSchema; use serde::{Deserialize, Serialize}; -use serde_json::Value; +use serde_json::{Deserializer, Value}; use session::context::QueryContextRef; use snafu::{OptionExt, ResultExt}; @@ -37,9 +37,10 @@ use crate::query_handler::LogHandlerRef; #[derive(Debug, Default, Serialize, Deserialize, JsonSchema)] pub struct LogIngesterQueryParams { - pub table_name: Option, + pub table: Option, pub db: Option, pub pipeline_name: Option, + pub ignore_errors: Option, } #[axum_macros::debug_handler] @@ -68,6 +69,44 @@ pub async fn add_pipeline( }) } +/// Transform NDJSON array into a single array +fn transform_ndjson_array_factory( + ignore_error: bool, +) -> impl FnMut(Result, std::result::Result) -> Result { + move |acc, item| { + acc.and_then(|acc| match acc { + Value::Array(mut acc_array) => { + if let Ok(item_value) = item { + match item_value { + Value::Array(item_array) => { + acc_array.extend(item_array); + } + Value::Object(_) => { + acc_array.push(item_value); + } + _ => { + if !ignore_error { + warn!("invalid item in array: {:?}", item_value); + return Err(InvalidParameterSnafu { + reason: format!("invalid item:{} in array", item_value), + } + .build()); + } + } + } + Ok(Value::Array(acc_array)) + } else if !ignore_error { + item.context(ParseJsonSnafu) + } else { + warn!("invalid item in array: {:?}", item); + Ok(Value::Array(acc_array)) + } + } + _ => unreachable!("invalid acc: {:?}", acc), + }) + } +} + #[axum_macros::debug_handler] pub async fn log_ingester( State(handler): State, @@ -79,14 +118,18 @@ pub async fn log_ingester( let pipeline_name = query_params.pipeline_name.context(InvalidParameterSnafu { reason: "pipeline_name is required", })?; - let table_name = query_params.table_name.context(InvalidParameterSnafu { - reason: "table_name is required", + let table_name = query_params.table.context(InvalidParameterSnafu { + reason: "table is required", })?; + let ignore_errors = query_params.ignore_errors.unwrap_or(false); + let m: mime::Mime = content_type.clone().into(); let value = match m.subtype() { - // TODO (qtang): we should decide json or jsonl - mime::JSON => serde_json::from_str(&payload).context(ParseJsonSnafu)?, + mime::JSON => Deserializer::from_str(&payload).into_iter::().fold( + Ok(Value::Array(Vec::with_capacity(100))), + transform_ndjson_array_factory(ignore_errors), + )?, // add more content type support _ => UnsupportedContentTypeSnafu { content_type }.fail()?, }; From 2e69655202e792bd17754b78933b5c5c606744cf Mon Sep 17 00:00:00 2001 From: shuiyisong Date: Wed, 5 Jun 2024 21:48:45 +0800 Subject: [PATCH 35/51] chore: introduce pipeline crate --- src/pipeline/src/etl/README.md | 13 + src/pipeline/src/etl/field.rs | 195 +++ src/pipeline/src/etl/mod.rs | 195 +++ src/pipeline/src/etl/processor/cmcd.rs | 361 ++++++ src/pipeline/src/etl/processor/csv.rs | 327 +++++ src/pipeline/src/etl/processor/date.rs | 345 +++++ src/pipeline/src/etl/processor/dissect.rs | 1118 +++++++++++++++++ src/pipeline/src/etl/processor/epoch.rs | 205 +++ src/pipeline/src/etl/processor/letter.rs | 188 +++ src/pipeline/src/etl/processor/mod.rs | 224 ++++ src/pipeline/src/etl/processor/regex.rs | 315 +++++ src/pipeline/src/etl/processor/urlencoding.rs | 177 +++ src/pipeline/src/etl/transform/index.rs | 57 + src/pipeline/src/etl/transform/mod.rs | 205 +++ .../transform/transformer/greptime/coerce.rs | 310 +++++ .../etl/transform/transformer/greptime/mod.rs | 172 +++ .../src/etl/transform/transformer/mod.rs | 16 + .../src/etl/transform/transformer/noop.rs | 36 + src/pipeline/src/etl/value/array.rs | 56 + src/pipeline/src/etl/value/map.rs | 64 + src/pipeline/src/etl/value/mod.rs | 303 +++++ src/pipeline/src/etl/value/time.rs | 187 +++ src/pipeline/tests/pipeline.rs | 461 +++++++ 23 files changed, 5530 insertions(+) create mode 100644 src/pipeline/src/etl/README.md create mode 100644 src/pipeline/src/etl/field.rs create mode 100644 src/pipeline/src/etl/mod.rs create mode 100644 src/pipeline/src/etl/processor/cmcd.rs create mode 100644 src/pipeline/src/etl/processor/csv.rs create mode 100644 src/pipeline/src/etl/processor/date.rs create mode 100644 src/pipeline/src/etl/processor/dissect.rs create mode 100644 src/pipeline/src/etl/processor/epoch.rs create mode 100644 src/pipeline/src/etl/processor/letter.rs create mode 100644 src/pipeline/src/etl/processor/mod.rs create mode 100644 src/pipeline/src/etl/processor/regex.rs create mode 100644 src/pipeline/src/etl/processor/urlencoding.rs create mode 100644 src/pipeline/src/etl/transform/index.rs create mode 100644 src/pipeline/src/etl/transform/mod.rs create mode 100644 src/pipeline/src/etl/transform/transformer/greptime/coerce.rs create mode 100644 src/pipeline/src/etl/transform/transformer/greptime/mod.rs create mode 100644 src/pipeline/src/etl/transform/transformer/mod.rs create mode 100644 src/pipeline/src/etl/transform/transformer/noop.rs create mode 100644 src/pipeline/src/etl/value/array.rs create mode 100644 src/pipeline/src/etl/value/map.rs create mode 100644 src/pipeline/src/etl/value/mod.rs create mode 100644 src/pipeline/src/etl/value/time.rs create mode 100644 src/pipeline/tests/pipeline.rs diff --git a/src/pipeline/src/etl/README.md b/src/pipeline/src/etl/README.md new file mode 100644 index 000000000000..e512ea31f7b5 --- /dev/null +++ b/src/pipeline/src/etl/README.md @@ -0,0 +1,13 @@ +# pipeline + +ETL capability + +## processors + +refer [elastic ingest processor][elastic-ingest-processor] for detail + +### Example + +Go to [pipeline](../../tests/pipeline.rs) + +[elastic-ingest-processor]: https://www.elastic.co/guide/en/elasticsearch/reference/current/processors.html diff --git a/src/pipeline/src/etl/field.rs b/src/pipeline/src/etl/field.rs new file mode 100644 index 000000000000..34181be5f4a3 --- /dev/null +++ b/src/pipeline/src/etl/field.rs @@ -0,0 +1,195 @@ +// Copyright 2023 Greptime Team +// +// Licensed 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 itertools::Itertools; + +#[derive(Debug, Default, Clone)] +pub struct Fields(Vec); + +impl Fields { + pub(crate) fn new(fields: Vec) -> Result { + let ff = Fields(fields); + ff.check() + } + + pub(crate) fn one(field: Field) -> Self { + Fields(vec![field]) + } + + pub(crate) fn get_target_fields(&self) -> Vec<&str> { + self.0.iter().map(|f| f.get_target_field()).collect() + } + + fn check(self) -> Result { + if self.0.is_empty() { + return Err("fields must not be empty".to_string()); + } + + let mut set = std::collections::HashSet::new(); + for f in self.0.iter() { + if set.contains(&f.field) { + return Err(format!( + "field name must be unique, but got duplicated: {}", + f.field + )); + } + set.insert(&f.field); + } + + Ok(self) + } +} + +impl std::fmt::Display for Fields { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + let s = self.0.iter().map(|f| f.to_string()).join(";"); + write!(f, "{s}") + } +} + +impl std::ops::Deref for Fields { + type Target = Vec; + + fn deref(&self) -> &Self::Target { + &self.0 + } +} + +#[derive(Debug, Default, Clone)] +pub struct Field { + pub field: String, + + // rename + pub target_field: Option, + + // 1-to-many mapping + // processors: + // - csv + pub target_fields: Option>, +} + +impl Field { + pub(crate) fn new(field: impl Into) -> Self { + Field { + field: field.into(), + target_field: None, + target_fields: None, + } + } + + // column_name in transform + pub(crate) fn get_target_field(&self) -> &str { + self.target_field.as_deref().unwrap_or(&self.field) + } + + pub(crate) fn get_field(&self) -> &str { + &self.field + } +} + +impl std::str::FromStr for Field { + type Err = String; + + fn from_str(s: &str) -> Result { + let mut parts = s.split(','); + let field = parts.next().ok_or("field is missing")?.trim().to_string(); + + if field.is_empty() { + return Err("field is empty".to_string()); + } + + let target_field = match parts.next() { + Some(s) if !s.trim().is_empty() => Some(s.trim().to_string()), + _ => None, + }; + + let fields: Vec<_> = parts + .filter(|s| !s.trim().is_empty()) + .map(|s| s.trim().to_string()) + .collect(); + let target_fields = if fields.is_empty() { + None + } else { + Some(fields) + }; + + Ok(Field { + field, + target_field, + target_fields, + }) + } +} + +impl std::fmt::Display for Field { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + match (&self.target_field, &self.target_fields) { + (Some(target_field), None) => write!(f, "{}, {target_field}", self.field), + (None, Some(target_fields)) => { + write!(f, "{}, {}", self.field, target_fields.iter().join(",")) + } + _ => write!(f, "{}", self.field), + } + } +} + +#[cfg(test)] +mod tests { + use crate::etl::field::Field; + + #[test] + fn test_parse_field() { + let field: Result = " ".parse(); + assert!(field.is_err()); + + let field: Result = ",".parse(); + assert!(field.is_err()); + + let field: Result = ",field".parse(); + assert!(field.is_err()); + + let cases = [ + // ("field", "field", None, None), + ( + "field, target_field", + "field", + Some("target_field".into()), + None, + ), + ( + "field, target_field1, target_field2, target_field3", + "field", + Some("target_field1".into()), + Some(vec!["target_field2".into(), "target_field3".into()]), + ), + ( + "field,, target_field1, target_field2, target_field3", + "field", + None, + Some(vec![ + "target_field1".into(), + "target_field2".into(), + "target_field3".into(), + ]), + ), + ]; + + for (s, field, target_field, target_fields) in cases.into_iter() { + let f: Field = s.parse().unwrap(); + assert_eq!(f.get_field(), field, "{s}"); + assert_eq!(f.target_field, target_field, "{s}"); + assert_eq!(f.target_fields, target_fields, "{s}"); + } + } +} diff --git a/src/pipeline/src/etl/mod.rs b/src/pipeline/src/etl/mod.rs new file mode 100644 index 000000000000..4e4595479482 --- /dev/null +++ b/src/pipeline/src/etl/mod.rs @@ -0,0 +1,195 @@ +// Copyright 2023 Greptime Team +// +// Licensed 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. + +#![allow(dead_code)] + +pub mod field; +pub mod processor; +pub mod transform; +pub mod value; + +use itertools::Itertools; +use transform::{Transformer, Transforms}; +use yaml_rust::YamlLoader; + +const DESCRIPTION: &str = "description"; +const PROCESSORS: &str = "processors"; +const TRANSFORM: &str = "transform"; + +pub enum Content { + Json(String), + Yaml(String), +} + +pub fn parse(input: &Content) -> Result, String> +where + T: Transformer, +{ + match input { + Content::Yaml(str) => { + let docs = YamlLoader::load_from_str(str).map_err(|e| e.to_string())?; + + let doc = &docs[0]; + + let description = doc[DESCRIPTION].as_str().map(|s| s.to_string()); + + let processors = if let Some(v) = doc[PROCESSORS].as_vec() { + v.try_into()? + } else { + processor::Processors::default() + }; + + let transforms = if let Some(v) = doc[TRANSFORM].as_vec() { + v.try_into()? + } else { + Transforms::default() + }; + + Ok(Pipeline { + description, + processors, + transformer: T::new(transforms)?, + }) + } + Content::Json(_) => unimplemented!(), + } +} + +#[derive(Debug, Clone)] +pub struct Pipeline +where + T: Transformer, +{ + description: Option, + processors: processor::Processors, + transformer: T, + // pub on_failure: processor::Processors, +} + +impl std::fmt::Display for Pipeline +where + T: Transformer, +{ + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + if let Some(description) = &self.description { + writeln!(f, "description: {description}")?; + } + + let processors = self.processors.iter().map(|p| p.kind()).join(","); + writeln!(f, "processors: {processors}")?; + + writeln!(f, "transformer: {}", self.transformer) + } +} + +impl Pipeline +where + T: Transformer, +{ + pub fn exec(&self, val: value::Value) -> Result { + let mut val = val; + for processor in self.processors.iter() { + val = processor.exec(val)?; + } + + self.transformer.transform(val) + } +} + +#[cfg(test)] +mod tests { + + use greptime_proto::v1::{self, ColumnDataType, SemanticType}; + + use crate::etl::transform::GreptimeTransformer; + use crate::etl::{parse, Content, Pipeline}; + + #[test] + fn test_csv_pipeline() { + let input_value_str = r#" + { + "my_field": "1,2", + "foo": "bar" + } + "#; + let input_value: serde_json::Value = serde_json::from_str(input_value_str).unwrap(); + + let pipeline_yaml = r#" +--- +description: Pipeline for Apache Tomcat + +processors: + - csv: + field: my_field, field1, field2 + +transform: + - field: field1 + type: uint32 + - field: field2 + type: uint32 +"#; + + let pipeline: Pipeline = + parse(&Content::Yaml(pipeline_yaml.into())).unwrap(); + let output = pipeline.exec(input_value.try_into().unwrap()); + assert!(output.is_ok()); + } + + #[test] + fn test_date_pipeline() { + let input_value_str = r#" + { + "my_field": "1,2", + "foo": "bar", + "test_time": "2014-5-17T04:34:56+00:00" + } + "#; + let input_value: serde_json::Value = serde_json::from_str(input_value_str).unwrap(); + + let pipeline_yaml = r#" +--- +description: Pipeline for Apache Tomcat + +processors: + - date: + field: test_time + +transform: + - field: test_time + type: time + index: timestamp +"#; + + let pipeline: Pipeline = + parse(&Content::Yaml(pipeline_yaml.into())).unwrap(); + let output = pipeline.exec(input_value.try_into().unwrap()).unwrap(); + let schemas = output.schema; + + assert_eq!(schemas.len(), 1); + let schema = schemas[0].clone(); + assert_eq!("test_time", schema.column_name); + assert_eq!(ColumnDataType::TimestampNanosecond as i32, schema.datatype); + assert_eq!(SemanticType::Timestamp as i32, schema.semantic_type); + + let row = output.rows[0].clone(); + assert_eq!(1, row.values.len()); + let value_data = row.values[0].clone().value_data; + assert_eq!( + Some(v1::value::ValueData::TimestampNanosecondValue( + 1400301296000000000 + )), + value_data + ); + } +} diff --git a/src/pipeline/src/etl/processor/cmcd.rs b/src/pipeline/src/etl/processor/cmcd.rs new file mode 100644 index 000000000000..256d6f05da11 --- /dev/null +++ b/src/pipeline/src/etl/processor/cmcd.rs @@ -0,0 +1,361 @@ +// Copyright 2023 Greptime Team +// +// Licensed 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 urlencoding::decode; + +use crate::etl::field::{Field, Fields}; +use crate::etl::processor::{ + yaml_bool, yaml_field, yaml_fields, FIELDS_NAME, FIELD_NAME, IGNORE_MISSING_NAME, +}; +use crate::etl::value::{Map, Value}; + +pub(crate) const PROCESSOR_CMCD: &str = "cmcd"; + +const CMCD_KEY_BR: &str = "br"; // Encoded bitrate, Integer kbps +const CMCD_KEY_BL: &str = "bl"; // Buffer length, Integer milliseconds +const CMCD_KEY_BS: &str = "bs"; // Buffer starvation, Boolean +const CMCD_KEY_CID: &str = "cid"; // Content ID, String +const CMCD_KEY_D: &str = "d"; // Object duration, Integer milliseconds +const CMCD_KEY_DL: &str = "dl"; // Deadline, Integer milliseconds +const CMCD_KEY_MTP: &str = "mtp"; // Measured throughput, Integer kbps +const CMCD_KEY_NOR: &str = "nor"; // Next object request, String +const CMCD_KEY_NRR: &str = "nrr"; // Next request range, String, "-" +const CMCD_KEY_OT: &str = "ot"; // Object type, Token - one of [m,a,v,av,i,c,tt,k,o] +const CMCD_KEY_PR: &str = "pr"; // Playback rate, Decimal +const CMCD_KEY_RTP: &str = "rtp"; // Requested maximum throughput, Integer kbps +const CMCD_KEY_SF: &str = "sf"; // Stall frequency, Token - one of [d,h,s,o] +const CMCD_KEY_SID: &str = "sid"; // Session ID, String +const CMCD_KEY_ST: &str = "st"; // Stream type, Token - one of [v,l] +const CMCD_KEY_SU: &str = "su"; // Startup, Boolean +const CMCD_KEY_TB: &str = "tb"; // Top bitrate, Integer kbps +const CMCD_KEY_V: &str = "v"; // Version + +/// Common Media Client Data Specification: +/// https://cdn.cta.tech/cta/media/media/resources/standards/pdfs/cta-5004-final.pdf +/// +/// +/// The data payload for Header and Query Argument transmission consists of a series of +/// key/value pairs constructed according to the following rules: +/// 1. All information in the payload MUST be represented as = pairs. +/// 2. The key and value MUST be separated by an equals sign Unicode 0x3D. If the +/// value type is BOOLEAN and the value is TRUE, then the equals sign and the value +/// MUST be omitted. +/// 3. Successive key/value pairs MUST be delimited by a comma Unicode 0x2C. +/// 4. The key names described in this specification are reserved. Custom key names +/// may be used, but they MUST carry a hyphenated prefix to ensure that there will +/// not be a namespace collision with future revisions to this specification. Clients +/// SHOULD use a reverse-DNS syntax when defining their own prefix. +/// 5. If headers are used for data transmission, then custom keys SHOULD be +/// allocated to one of the four defined header names based upon their expected +/// level of variability: +/// a. CMCD-Request: keys whose values vary with each request. +/// b. CMCD-Object: keys whose values vary with the object being requested. +/// c. CMCD-Status: keys whose values do not vary with every request or object. +/// d. CMCD-Session: keys whose values are expected to be invariant over the life of the session. +/// 6. All key names are case-sensitive. +/// 7. Any value of type String MUST be enclosed by opening and closing double +/// quotes Unicode 0x22. Double quotes and backslashes MUST be escaped using a +/// backslash "\" Unicode 0x5C character. Any value of type Token does not require +/// quoting. +/// 8. All keys are OPTIONAL. +/// 9. Key-value pairs SHOULD be sequenced in alphabetical order of the key name in +/// order to reduce the fingerprinting surface exposed by the player. +/// 10. If the data payload is transmitted as a query argument, then the entire payload +/// string MUST be URLEncoded per [5]. Data payloads transmitted via headers +/// MUST NOT be URLEncoded. +/// 11. The data payload syntax is intended to be compliant with Structured Field Values for HTTP [6]. +/// 12. Transport Layer Security SHOULD be used to protect all transmission of CMCD data. +#[derive(Debug, Default)] +pub struct CMCDProcessor { + fields: Fields, + + ignore_missing: bool, +} + +impl CMCDProcessor { + fn with_fields(&mut self, fields: Fields) { + self.fields = fields; + } + + fn with_ignore_missing(&mut self, ignore_missing: bool) { + self.ignore_missing = ignore_missing; + } + + fn parse(prefix: &str, s: &str) -> Result { + let mut map = Map::default(); + let parts = s.split(','); + for part in parts { + let mut kv = part.split('='); + let k = kv.next().ok_or(format!("{part} missing key in {s}"))?; + let v = kv.next(); + + let key = format!("{prefix}_{k}"); + match k { + CMCD_KEY_BS | CMCD_KEY_SU => { + map.insert(key, Value::Boolean(true)); + } + CMCD_KEY_BR | CMCD_KEY_BL | CMCD_KEY_D | CMCD_KEY_DL | CMCD_KEY_MTP + | CMCD_KEY_RTP | CMCD_KEY_TB => { + let v = v.ok_or(format!("{k} missing value in {s}"))?; + let val: i64 = v + .parse() + .map_err(|_| format!("failed to parse {v} as i64"))?; + map.insert(key, Value::Int64(val)); + } + CMCD_KEY_CID | CMCD_KEY_NRR | CMCD_KEY_OT | CMCD_KEY_SF | CMCD_KEY_SID + | CMCD_KEY_ST | CMCD_KEY_V => { + let v = v.ok_or(format!("{k} missing value in {s}"))?; + map.insert(key, Value::String(v.to_string())); + } + CMCD_KEY_NOR => { + let v = v.ok_or(format!("{k} missing value in {s}"))?; + let val = match decode(v) { + Ok(val) => val.to_string(), + Err(_) => v.to_string(), + }; + map.insert(key, Value::String(val)); + } + CMCD_KEY_PR => { + let v = v.ok_or(format!("{k} missing value in {s}"))?; + let val: f64 = v + .parse() + .map_err(|_| format!("failed to parse {v} as f64"))?; + map.insert(key, Value::Float64(val)); + } + _ => match v { + Some(v) => map.insert(key, Value::String(v.to_string())), + None => map.insert(k, Value::Boolean(true)), + }, + } + } + + Ok(map) + } + + fn process_field(&self, val: &str, field: &Field) -> Result { + let prefix = match field.target_field { + Some(ref target_field) => target_field, + None => field.get_field(), + }; + + Self::parse(prefix, val) + } +} + +impl TryFrom<&yaml_rust::yaml::Hash> for CMCDProcessor { + type Error = String; + + fn try_from(value: &yaml_rust::yaml::Hash) -> Result { + let mut processor = CMCDProcessor::default(); + + for (k, v) in value.iter() { + let key = k + .as_str() + .ok_or(format!("key must be a string, but got {k:?}"))?; + match key { + FIELD_NAME => { + processor.with_fields(Fields::one(yaml_field(v, FIELD_NAME)?)); + } + FIELDS_NAME => { + processor.with_fields(yaml_fields(v, FIELDS_NAME)?); + } + + IGNORE_MISSING_NAME => { + processor.with_ignore_missing(yaml_bool(v, IGNORE_MISSING_NAME)?); + } + + _ => {} + } + } + + Ok(processor) + } +} + +impl crate::etl::processor::Processor for CMCDProcessor { + fn kind(&self) -> &str { + PROCESSOR_CMCD + } + + fn ignore_missing(&self) -> bool { + self.ignore_missing + } + + fn fields(&self) -> &Fields { + &self.fields + } + + fn exec_field(&self, val: &Value, field: &Field) -> Result { + match val { + Value::String(val) => self.process_field(val, field), + _ => Err(format!( + "{} processor: expect string value, but got {val:?}", + self.kind() + )), + } + } +} + +#[cfg(test)] +mod tests { + use std::collections::HashMap; + + use urlencoding::decode; + + use super::CMCDProcessor; + use crate::etl::value::{Map, Value}; + + #[test] + fn test_cmcd() { + let ss = [ + ( + "sid%3D%226e2fb550-c457-11e9-bb97-0800200c9a66%22", + vec![( + "prefix_sid", + Value::String("\"6e2fb550-c457-11e9-bb97-0800200c9a66\"".into()), + )], + ), + ( + "br%3D3200%2Cbs%2Cd%3D4004%2Cmtp%3D25400%2Cot%3Dv%2Crtp%3D15000%2Csid%3D%226e2fb550-c457-11e9-bb97-0800200c9a66%22%2Ctb%3D6000", + vec![ + ("prefix_bs", Value::Boolean(true)), + ("prefix_ot", Value::String("v".into())), + ("prefix_rtp", Value::Int64(15000)), + ("prefix_br", Value::Int64(3200)), + ("prefix_tb", Value::Int64(6000)), + ("prefix_d", Value::Int64(4004)), + ( + "prefix_sid", + Value::String("\"6e2fb550-c457-11e9-bb97-0800200c9a66\"".into()), + ), + ("prefix_mtp", Value::Int64(25400)), + ], + ), + ( + "b%2Crtp%3D15000%2Csid%3D%226e2fb550-c457-11e9-bb97-0800200c9a66%22", + vec![ + ( + "prefix_sid", + Value::String("\"6e2fb550-c457-11e9-bb97-0800200c9a66\"".into()), + ), + ("prefix_rtp", Value::Int64(15000)), + ("b", Value::Boolean(true)), + ], + ), + ( + "bs%2Csu", + vec![ + ("prefix_su", Value::Boolean(true)), + ("prefix_bs", Value::Boolean(true)), + ], + ), + ( + "d%3D4004%2Ccom.example-myNumericKey%3D500%2Ccom.examplemyStringKey%3D%22myStringValue%22", + vec![ + ( + "prefix_com.example-myNumericKey", + Value::String("500".into()), + ), + ( + "prefix_com.examplemyStringKey", + Value::String("\"myStringValue\"".into()), + ), + ("prefix_d", Value::Int64(4004)), + ], + ), + ( + "nor%3D%22..%252F300kbps%252Fsegment35.m4v%22%2Csid%3D%226e2fb550-c457-11e9-bb97-0800200c9a66%22", + vec![ + ( + "prefix_sid", + Value::String("\"6e2fb550-c457-11e9-bb97-0800200c9a66\"".into()), + ), + ( + "prefix_nor", + Value::String("\"../300kbps/segment35.m4v\"".into()), + + ), + ], + ), + ( + "nrr%3D%2212323-48763%22%2Csid%3D%226e2fb550-c457-11e9-bb97-0800200c9a66%22", + vec![ + ("prefix_nrr", Value::String("\"12323-48763\"".into())), + ( + "prefix_sid", + Value::String("\"6e2fb550-c457-11e9-bb97-0800200c9a66\"".into()), + ), + ], + ), + ( + "nor%3D%22..%252F300kbps%252Ftrack.m4v%22%2Cnrr%3D%2212323-48763%22%2Csid%3D%226e2fb550-c457-11e9-bb97-0800200c9a66%22", + vec![ + ("prefix_nrr", Value::String("\"12323-48763\"".into())), + ( + "prefix_sid", + Value::String("\"6e2fb550-c457-11e9-bb97-0800200c9a66\"".into()), + ), + ( + "prefix_nor", + Value::String("\"../300kbps/track.m4v\"".into()), + ), + ], + ), + ( + "bl%3D21300%2Cbr%3D3200%2Cbs%2Ccid%3D%22faec5fc2-ac30-11eabb37-0242ac130002%22%2Cd%3D4004%2Cdl%3D18500%2Cmtp%3D48100%2Cnor%3D%22..%252F300kbps%252Ftrack.m4v%22%2Cnrr%3D%2212323-48763%22%2Cot%3Dv%2Cpr%3D1.08%2Crtp%3D12000%2Csf%3Dd%2Csid%3D%226e2fb550-c457-11e9-bb97-0800200c9a66%22%2Cst%3Dv%2Csu%2Ctb%3D6000", + vec![ + ("prefix_bl", Value::Int64(21300)), + ("prefix_bs", Value::Boolean(true)), + ("prefix_st", Value::String("v".into())), + ("prefix_ot", Value::String("v".into())), + ( + "prefix_sid", + Value::String("\"6e2fb550-c457-11e9-bb97-0800200c9a66\"".into()), + ), + ("prefix_tb", Value::Int64(6000)), + ("prefix_d", Value::Int64(4004)), + ( + "prefix_cid", + Value::String("\"faec5fc2-ac30-11eabb37-0242ac130002\"".into()), + ), + ("prefix_mtp", Value::Int64(48100)), + ("prefix_rtp", Value::Int64(12000)), + ( + "prefix_nor", + Value::String("\"../300kbps/track.m4v\"".into()), + ), + ("prefix_sf", Value::String("d".into())), + ("prefix_br", Value::Int64(3200)), + ("prefix_nrr", Value::String("\"12323-48763\"".into())), + ("prefix_pr", Value::Float64(1.08)), + ("prefix_su", Value::Boolean(true)), + ("prefix_dl", Value::Int64(18500)), + ], + ), + ]; + + for (s, vec) in ss.into_iter() { + let decoded = decode(s).unwrap().to_string(); + + let values = vec + .into_iter() + .map(|(k, v)| (k.to_string(), v)) + .collect::>(); + let expected = Map { values }; + + let actual = CMCDProcessor::parse("prefix", &decoded).unwrap(); + assert_eq!(actual, expected); + } + } +} diff --git a/src/pipeline/src/etl/processor/csv.rs b/src/pipeline/src/etl/processor/csv.rs new file mode 100644 index 000000000000..ae578d79e1f6 --- /dev/null +++ b/src/pipeline/src/etl/processor/csv.rs @@ -0,0 +1,327 @@ +// Copyright 2023 Greptime Team +// +// Licensed 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. + +// Reference: https://www.elastic.co/guide/en/elasticsearch/reference/current/csv-processor.html + +use std::collections::HashMap; + +use csv::{ReaderBuilder, Trim}; +use itertools::EitherOrBoth::{Both, Left, Right}; +use itertools::Itertools; + +use crate::etl::field::{Field, Fields}; +use crate::etl::processor::{ + yaml_bool, yaml_field, yaml_fields, yaml_string, Processor, FIELDS_NAME, FIELD_NAME, + IGNORE_MISSING_NAME, +}; +use crate::etl::value::{Map, Value}; + +pub(crate) const PROCESSOR_CSV: &str = "csv"; + +const SEPARATOR_NAME: &str = "separator"; +const QUOTE_NAME: &str = "quote"; +const TRIM_NAME: &str = "trim"; +const EMPTY_VALUE_NAME: &str = "empty_value"; + +/// only support string value +#[derive(Debug)] +pub struct CsvProcessor { + reader: ReaderBuilder, + + fields: Fields, + + ignore_missing: bool, + + // Value used to fill empty fields, empty fields will be skipped if this is not provided. + empty_value: Option, + // description + // if + // ignore_failure + // on_failure + // tag +} + +impl CsvProcessor { + fn new() -> Self { + let mut reader = ReaderBuilder::new(); + reader.has_headers(false); + + Self { + reader, + fields: Fields::default(), + ignore_missing: false, + empty_value: None, + } + } + + fn with_fields(&mut self, fields: Fields) { + self.fields = fields; + } + + fn try_separator(&mut self, separator: String) -> Result<(), String> { + if separator.len() != 1 { + Err(format!( + "'{}' must be a single character, but got '{}'", + SEPARATOR_NAME, separator + )) + } else { + self.reader.delimiter(separator.as_bytes()[0]); + Ok(()) + } + } + + fn try_quote(&mut self, quote: String) -> Result<(), String> { + if quote.len() != 1 { + Err(format!( + "'{}' must be a single character, but got '{}'", + QUOTE_NAME, quote + )) + } else { + self.reader.quote(quote.as_bytes()[0]); + Ok(()) + } + } + + fn with_trim(&mut self, trim: bool) { + if trim { + self.reader.trim(Trim::All); + } else { + self.reader.trim(Trim::None); + } + } + + fn with_ignore_missing(&mut self, ignore_missing: bool) { + self.ignore_missing = ignore_missing; + } + + fn with_empty_value(&mut self, empty_value: String) { + self.empty_value = Some(empty_value); + } + + // process the csv format string to a map with target_fields as keys + fn process_field(&self, val: &str, field: &Field) -> Result { + let mut reader = self.reader.from_reader(val.as_bytes()); + + if let Some(result) = reader.records().next() { + let record: csv::StringRecord = result.map_err(|e| e.to_string())?; + + let values: HashMap = field + .target_fields + .as_ref() + .ok_or(format!( + "target fields must be set after '{}'", + field.get_field() + ))? + .iter() + .map(|f| f.to_string()) + .zip_longest(record.iter()) + .filter_map(|zipped| match zipped { + Both(target_field, val) => Some((target_field, Value::String(val.into()))), + // if target fields are more than extracted fields, fill the rest with empty value + Left(target_field) => { + let value = self + .empty_value + .as_ref() + .map(|s| Value::String(s.clone())) + .unwrap_or(Value::Null); + Some((target_field, value)) + } + // if extracted fields are more than target fields, ignore the rest + Right(_) => None, + }) + .collect(); + + Ok(Map { values }) + } else { + Err("expected at least one record from csv format, but got none".into()) + } + } +} + +impl TryFrom<&yaml_rust::yaml::Hash> for CsvProcessor { + type Error = String; + + fn try_from(hash: &yaml_rust::yaml::Hash) -> Result { + let mut processor = CsvProcessor::new(); + for (k, v) in hash { + let key = k + .as_str() + .ok_or(format!("key must be a string, but got {k:?}"))?; + match key { + FIELD_NAME => { + processor.with_fields(Fields::one(yaml_field(v, FIELD_NAME)?)); + } + FIELDS_NAME => { + processor.with_fields(yaml_fields(v, FIELDS_NAME)?); + } + SEPARATOR_NAME => { + processor.try_separator(yaml_string(v, SEPARATOR_NAME)?)?; + } + QUOTE_NAME => { + processor.try_quote(yaml_string(v, QUOTE_NAME)?)?; + } + TRIM_NAME => { + processor.with_trim(yaml_bool(v, TRIM_NAME)?); + } + IGNORE_MISSING_NAME => { + processor.with_ignore_missing(yaml_bool(v, IGNORE_MISSING_NAME)?); + } + EMPTY_VALUE_NAME => { + processor.with_empty_value(yaml_string(v, EMPTY_VALUE_NAME)?); + } + + _ => {} + } + } + + Ok(processor) + } +} + +impl Processor for CsvProcessor { + fn kind(&self) -> &str { + PROCESSOR_CSV + } + + fn ignore_missing(&self) -> bool { + self.ignore_missing + } + + fn fields(&self) -> &Fields { + &self.fields + } + + fn exec_field(&self, val: &Value, field: &Field) -> Result { + match val { + Value::String(val) => self.process_field(val, field), + _ => Err(format!( + "{} processor: expect string value, but got {val:?}", + self.kind() + )), + } + } +} + +// TODO(yuanbohan): more test cases +#[cfg(test)] +mod tests { + use std::collections::HashMap; + + use super::{CsvProcessor, Value}; + use crate::etl::field::Fields; + use crate::etl::processor::Processor; + use crate::etl::value::Map; + + #[test] + fn test_equal_length() { + let mut processor = CsvProcessor::new(); + let field = "data,, a, b".parse().unwrap(); + processor.with_fields(Fields::one(field)); + + let values: HashMap = [("data".into(), Value::String("1,2".into()))] + .into_iter() + .collect(); + + let result = processor.exec(Value::Map(Map { values })).unwrap(); + + let values = [ + ("data".into(), Value::String("1,2".into())), + ("a".into(), Value::String("1".into())), + ("b".into(), Value::String("2".into())), + ] + .into_iter() + .collect(); + let expected = Value::Map(Map { values }); + + assert_eq!(expected, result); + } + + // test target_fields length larger than the record length + #[test] + fn test_target_fields_has_more_length() { + let values = [("data".into(), Value::String("1,2".into()))] + .into_iter() + .collect(); + let input = Value::Map(Map { values }); + + // with no empty value + { + let mut processor = CsvProcessor::new(); + let field = "data,, a,b,c".parse().unwrap(); + processor.with_fields(Fields::one(field)); + + let result = processor.exec(input.clone()).unwrap(); + + let values = [ + ("data".into(), Value::String("1,2".into())), + ("a".into(), Value::String("1".into())), + ("b".into(), Value::String("2".into())), + ("c".into(), Value::Null), + ] + .into_iter() + .collect(); + let expected = Value::Map(Map { values }); + + assert_eq!(expected, result); + } + + // with empty value + { + let mut processor = CsvProcessor::new(); + let field = "data,, a,b,c".parse().unwrap(); + processor.with_fields(Fields::one(field)); + processor.with_empty_value("default".into()); + + let result = processor.exec(input).unwrap(); + + let values = [ + ("data".into(), Value::String("1,2".into())), + ("a".into(), Value::String("1".into())), + ("b".into(), Value::String("2".into())), + ("c".into(), Value::String("default".into())), + ] + .into_iter() + .collect(); + let expected = Value::Map(Map { values }); + + assert_eq!(expected, result); + } + } + + // test record has larger length + #[test] + fn test_target_fields_has_less_length() { + let values = [("data".into(), Value::String("1,2,3".into()))] + .into_iter() + .collect(); + let input = Value::Map(Map { values }); + + let mut processor = CsvProcessor::new(); + let field = "data,,a,b".parse().unwrap(); + processor.with_fields(Fields::one(field)); + + let result = processor.exec(input).unwrap(); + + let values = [ + ("data".into(), Value::String("1,2,3".into())), + ("a".into(), Value::String("1".into())), + ("b".into(), Value::String("2".into())), + ] + .into_iter() + .collect(); + let expected = Value::Map(Map { values }); + + assert_eq!(expected, result); + } +} diff --git a/src/pipeline/src/etl/processor/date.rs b/src/pipeline/src/etl/processor/date.rs new file mode 100644 index 000000000000..9c4037900a7c --- /dev/null +++ b/src/pipeline/src/etl/processor/date.rs @@ -0,0 +1,345 @@ +// Copyright 2023 Greptime Team +// +// Licensed 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 chrono::{DateTime, NaiveDateTime}; +use chrono_tz::Tz; +use lazy_static::lazy_static; + +use crate::etl::field::{Field, Fields}; +use crate::etl::processor::{ + yaml_bool, yaml_field, yaml_fields, yaml_string, yaml_strings, Processor, FIELDS_NAME, + FIELD_NAME, IGNORE_MISSING_NAME, +}; +use crate::etl::value::{Map, Time, Value}; + +pub(crate) const PROCESSOR_DATE: &str = "date"; + +const FORMATS_NAME: &str = "formats"; // default RFC3339 +const TIMEZONE_NAME: &str = "timezone"; // default UTC +const LOCALE_NAME: &str = "locale"; +const OUTPUT_FORMAT_NAME: &str = "output_format"; // default with input format + +lazy_static! { + static ref DEFAULT_FORMATS: Vec = vec![ + // timezone with colon + "%Y-%m-%dT%H:%M:%S%:z", + "%Y-%m-%dT%H:%M:%S%.3f%:z", + "%Y-%m-%dT%H:%M:%S%.6f%:z", + "%Y-%m-%dT%H:%M:%S%.9f%:z", + // timezone without colon + "%Y-%m-%dT%H:%M:%S%z", + "%Y-%m-%dT%H:%M:%S%.3f%z", + "%Y-%m-%dT%H:%M:%S%.6f%z", + "%Y-%m-%dT%H:%M:%S%.9f%z", + // without timezone + "%Y-%m-%dT%H:%M:%SZ", + "%Y-%m-%dT%H:%M:%S", + "%Y-%m-%dT%H:%M:%S%.3f", + "%Y-%m-%dT%H:%M:%S%.6f", + "%Y-%m-%dT%H:%M:%S%.9f", + ] + .iter() + .map(|s| s.to_string()) + .collect(); +} + +#[derive(Debug, Default)] +struct Formats(Vec); + +impl Formats { + fn new(mut formats: Vec) -> Self { + formats.sort(); + formats.dedup(); + Formats(formats) + } +} + +impl std::ops::Deref for Formats { + type Target = Vec; + + fn deref(&self) -> &Self::Target { + &self.0 + } +} + +#[derive(Debug, Default)] +pub struct DateProcessor { + fields: Fields, + + formats: Formats, + timezone: Option, + locale: Option, // to support locale + output_format: Option, + + ignore_missing: bool, + // description + // if + // ignore_failure + // on_failure + // tag +} + +impl DateProcessor { + fn with_fields(&mut self, fields: Fields) { + self.fields = fields + } + + fn with_formats(&mut self, v: Option>) { + let v = match v { + Some(v) if !v.is_empty() => v, + _ => DEFAULT_FORMATS.clone(), + }; + + let formats = Formats::new(v); + self.formats = formats; + } + + fn with_timezone(&mut self, timezone: String) { + if !timezone.is_empty() { + self.timezone = Some(timezone); + } + } + + fn with_locale(&mut self, locale: String) { + if !locale.is_empty() { + self.locale = Some(locale); + } + } + + fn with_output_format(&mut self, output_format: String) { + if !output_format.is_empty() { + self.output_format = Some(output_format); + } + } + + fn with_ignore_missing(&mut self, ignore_missing: bool) { + self.ignore_missing = ignore_missing; + } + + fn parse(&self, val: &str) -> Result { + let mut tz = Tz::UTC; + if let Some(timezone) = &self.timezone { + tz = timezone.parse::().map_err(|e| e.to_string())?; + } + + for fmt in self.formats.iter() { + if let Ok(ns) = try_parse(val, fmt, tz) { + let mut t = Time::new(val, ns); + t.with_format(fmt); + t.with_timezone(self.timezone.clone()); + return Ok(t); + } + } + + Err(format!("{} processor: failed to parse {val}", self.kind(),)) + } + + fn process_field(&self, val: &str, field: &Field) -> Result { + let key = match field.target_field { + Some(ref target_field) => target_field, + None => field.get_field(), + }; + + Ok(Map::one(key, Value::Time(self.parse(val)?))) + } +} + +impl TryFrom<&yaml_rust::yaml::Hash> for DateProcessor { + type Error = String; + + fn try_from(hash: &yaml_rust::yaml::Hash) -> Result { + let mut processor = DateProcessor::default(); + + let mut formats_opt = None; + + for (k, v) in hash { + let key = k + .as_str() + .ok_or(format!("key must be a string, but got {k:?}"))?; + + match key { + FIELD_NAME => { + processor.with_fields(Fields::one(yaml_field(v, FIELD_NAME)?)); + } + FIELDS_NAME => { + processor.with_fields(yaml_fields(v, FIELDS_NAME)?); + } + + FORMATS_NAME => { + let formats = yaml_strings(v, FORMATS_NAME)?; + formats_opt = Some(formats); + } + TIMEZONE_NAME => { + processor.with_timezone(yaml_string(v, TIMEZONE_NAME)?); + } + LOCALE_NAME => { + processor.with_locale(yaml_string(v, LOCALE_NAME)?); + } + OUTPUT_FORMAT_NAME => { + processor.with_output_format(yaml_string(v, OUTPUT_FORMAT_NAME)?); + } + + IGNORE_MISSING_NAME => { + processor.with_ignore_missing(yaml_bool(v, IGNORE_MISSING_NAME)?); + } + + _ => {} + } + } + + processor.with_formats(formats_opt); + + Ok(processor) + } +} + +impl Processor for DateProcessor { + fn kind(&self) -> &str { + PROCESSOR_DATE + } + + fn ignore_missing(&self) -> bool { + self.ignore_missing + } + + fn fields(&self) -> &Fields { + &self.fields + } + + fn exec_field(&self, val: &Value, field: &Field) -> Result { + match val { + Value::String(s) => self.process_field(s, field), + _ => Err(format!( + "{} processor: expect string value, but got {val:?}", + self.kind() + )), + } + } +} + +/// try to parse val with timezone first, if failed, parse without timezone +fn try_parse(val: &str, fmt: &str, tz: Tz) -> Result { + if let Ok(dt) = DateTime::parse_from_str(val, fmt) { + Ok(dt.timestamp_nanos_opt().ok_or("failed to get timestamp")?) + } else { + let dt = NaiveDateTime::parse_from_str(val, fmt) + .map_err(|e| e.to_string())? + .and_local_timezone(tz) + .single() + .ok_or("failed to get local timezone")?; + Ok(dt.timestamp_nanos_opt().ok_or("failed to get timestamp")?) + } +} + +#[cfg(test)] +mod tests { + use chrono_tz::Asia::Tokyo; + + use crate::etl::processor::date::{try_parse, DateProcessor}; + + #[test] + fn test_try_parse() { + let time_with_tz = "2014-5-17T04:34:56+00:00"; + let fmt_with_tz = "%Y-%m-%dT%H:%M:%S%:z"; + + let time_without_tz = "2014-5-17T13:34:56"; + let fmt_without_tz = "%Y-%m-%dT%H:%M:%S"; + + let tz = Tokyo; + + let parsed_with_tz = try_parse(time_with_tz, fmt_with_tz, tz); + assert!(parsed_with_tz.is_ok()); + + let parsed_without_tz = try_parse(time_without_tz, fmt_without_tz, tz); + assert!(parsed_without_tz.is_ok()); + + assert_eq!(parsed_with_tz.unwrap(), parsed_without_tz.unwrap()); + } + + #[test] + fn test_parse() { + let mut processor = DateProcessor::default(); + processor.with_formats(None); + + let values: Vec<&str> = vec![ + "2014-5-17T12:34:56", + "2014-5-17T12:34:56Z", + "2014-5-17T12:34:56+09:30", + "2014-5-17T12:34:56.000+09:30", + "2014-5-17T12:34:56-0930", + "2014-5-17T12:34:56.000-0930", + ] + .into_iter() + .collect(); + + for value in values { + let parsed = processor.parse(value); + assert!(parsed.is_ok()); + } + } + + #[test] + fn test_parse_with_formats() { + let mut processor = DateProcessor::default(); + let formats = vec![ + "%Y-%m-%dT%H:%M:%S%:z", + "%Y-%m-%dT%H:%M:%S%.3f%:z", + "%Y-%m-%dT%H:%M:%S", + "%Y-%m-%dT%H:%M:%SZ", + ] + .into_iter() + .map(|s| s.to_string()) + .collect(); + processor.with_formats(Some(formats)); + + let values: Vec<&str> = vec![ + "2014-5-17T12:34:56", + "2014-5-17T12:34:56Z", + "2014-5-17T12:34:56+09:30", + "2014-5-17T12:34:56.000+09:30", + "2014-5-17T12:34:56-0930", + "2014-5-17T12:34:56.000-0930", + ] + .into_iter() + .collect(); + + for value in values { + let parsed = processor.parse(value); + assert!(parsed.is_ok()); + } + } + + #[test] + fn test_parse_with_timezone() { + let mut processor = DateProcessor::default(); + processor.with_formats(None); + processor.with_timezone("Asia/Tokyo".to_string()); + + let values: Vec<&str> = vec![ + "2014-5-17T12:34:56", + "2014-5-17T12:34:56Z", + "2014-5-17T12:34:56+09:30", + "2014-5-17T12:34:56.000+09:30", + "2014-5-17T12:34:56-0930", + "2014-5-17T12:34:56.000-0930", + ] + .into_iter() + .collect(); + + for value in values { + let parsed = processor.parse(value); + assert!(parsed.is_ok()); + } + } +} diff --git a/src/pipeline/src/etl/processor/dissect.rs b/src/pipeline/src/etl/processor/dissect.rs new file mode 100644 index 000000000000..005b104f5b14 --- /dev/null +++ b/src/pipeline/src/etl/processor/dissect.rs @@ -0,0 +1,1118 @@ +// Copyright 2023 Greptime Team +// +// Licensed 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 std::collections::{HashMap, HashSet}; + +use common_telemetry::warn; +use itertools::Itertools; + +use crate::etl::field::{Field, Fields}; +use crate::etl::processor::{ + yaml_bool, yaml_field, yaml_fields, yaml_parse_strings, yaml_string, Processor, FIELDS_NAME, + FIELD_NAME, IGNORE_MISSING_NAME, PATTERNS_NAME, +}; +use crate::etl::value::{Map, Value}; + +pub(crate) const PROCESSOR_DISSECT: &str = "dissect"; + +const APPEND_SEPARATOR_NAME: &str = "append_separator"; + +#[derive(Debug, PartialEq)] +enum StartModifier { + Append(Option), + NamedSkip, + MapKey, + MapVal, +} + +impl std::fmt::Display for StartModifier { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + match self { + StartModifier::Append(o) => match o { + Some(v) => write!(f, "+/{v}"), + None => write!(f, "+"), + }, + StartModifier::NamedSkip => write!(f, "?"), + StartModifier::MapKey => write!(f, "*"), + StartModifier::MapVal => write!(f, "&"), + } + } +} + +#[derive(Debug, PartialEq)] +struct EndModifier; + +impl std::fmt::Display for EndModifier { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + write!(f, "->",) + } +} + +#[derive(Debug, PartialEq, Default)] +struct Name { + name: String, + start_modifier: Option, + end_modifier: Option, +} + +impl Name { + fn is_name_empty(&self) -> bool { + self.name.is_empty() + } + + fn is_empty(&self) -> bool { + self.name.is_empty() && self.start_modifier.is_none() && self.end_modifier.is_none() + } + + fn try_start_modifier(&mut self, modifier: StartModifier) -> Result<(), String> { + match &self.start_modifier { + Some(m) => Err(format!("'{m}' modifier already set, but found {modifier}",)), + None => { + self.start_modifier = Some(modifier); + Ok(()) + } + } + } + + fn try_append_order(&mut self, order: u32) -> Result<(), String> { + match &mut self.start_modifier { + Some(StartModifier::Append(o)) => match o { + Some(n) => Err(format!( + "Append Order modifier is already set to '{n}', cannot be set to {order}" + )), + None => { + *o = Some(order); + Ok(()) + } + }, + Some(m) => Err(format!( + "Order can only be set to Append Modifier, current modifier is {m}" + )), + None => Err("Order can only be set to Append Modifier".to_string()), + } + } + + fn try_end_modifier(&mut self) -> Result<(), String> { + match &self.end_modifier { + Some(m) => Err(format!("End modifier already set: '{m}'")), + None => { + self.end_modifier = Some(EndModifier); + Ok(()) + } + } + } + + fn is_append_modifier_set(&self) -> bool { + matches!(self.start_modifier, Some(StartModifier::Append(_))) + } + + fn is_start_modifier_set(&self) -> bool { + self.start_modifier.is_some() + } + + fn is_end_modifier_set(&self) -> bool { + self.end_modifier.is_some() + } +} + +impl std::fmt::Display for Name { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + write!(f, "{}", self.name) + } +} + +impl From<&str> for Name { + fn from(value: &str) -> Self { + Name { + name: value.to_string(), + start_modifier: None, + end_modifier: None, + } + } +} + +#[derive(Debug, PartialEq)] +enum Part { + Split(String), + Name(Name), +} + +impl Part { + fn is_empty(&self) -> bool { + match self { + Part::Split(v) => v.is_empty(), + Part::Name(v) => v.is_empty(), + } + } + + fn empty_split() -> Self { + Part::Split(String::new()) + } + + fn empty_name() -> Self { + Part::Name(Name::default()) + } + + fn push(&mut self, ch: char) { + match self { + Part::Split(v) => v.push(ch), + Part::Name(v) => v.name.push(ch), + } + } +} + +#[derive(Debug, Default)] +struct Pattern { + origin: String, + parts: Vec, +} + +impl std::ops::Deref for Pattern { + type Target = Vec; + + fn deref(&self) -> &Self::Target { + &self.parts + } +} + +impl std::ops::DerefMut for Pattern { + fn deref_mut(&mut self) -> &mut Self::Target { + &mut self.parts + } +} + +impl std::str::FromStr for Pattern { + type Err = String; + + fn from_str(s: &str) -> Result { + let mut parts = vec![]; + let mut cursor = Part::empty_split(); + + let origin = s.to_string(); + let chars: Vec = origin.chars().collect(); + + let mut pos = 0; + while pos < chars.len() { + let ch = chars[pos]; + match (ch, &mut cursor) { + // if cursor is Split part, and found %{, then ready to start a Name part + ('%', Part::Split(_)) if matches!(chars.get(pos + 1), Some('{')) => { + if !cursor.is_empty() { + parts.push(cursor); + } + + cursor = Part::empty_name(); + pos += 1; // skip '{' + } + // if cursor is Split part, and not found % or {, then continue the Split part + (_, Part::Split(_)) => { + cursor.push(ch); + } + // if cursor is Name part, and found }, then end the Name part, start the next Split part + ('}', Part::Name(_)) => { + parts.push(cursor); + cursor = Part::empty_split(); + } + ('+', Part::Name(name)) if !name.is_start_modifier_set() => { + name.try_start_modifier(StartModifier::Append(None))?; + } + ('/', Part::Name(name)) if name.is_append_modifier_set() => { + let mut order = 0; + let mut j = pos + 1; + while j < chars.len() { + let digit = chars[j]; + if digit.is_ascii_digit() { + order = order * 10 + digit.to_digit(10).unwrap(); + j += 1; + } else { + break; + } + } + + if j == pos + 1 { + return Err(format!( + "Invalid Pattern: '{s}'. Digit order must be set after '/'", + )); + } + + name.try_append_order(order)?; + pos = j - 1; // this will change the position to the last digit of the order + } + ('?', Part::Name(name)) if !name.is_start_modifier_set() => { + name.try_start_modifier(StartModifier::NamedSkip)?; + } + ('*', Part::Name(name)) if !name.is_start_modifier_set() => { + name.try_start_modifier(StartModifier::MapKey)?; + } + ('&', Part::Name(name)) if !name.is_start_modifier_set() => { + name.try_start_modifier(StartModifier::MapVal)?; + } + ('-', Part::Name(name)) if !name.is_end_modifier_set() => { + if let Some('>') = chars.get(pos + 1) { + } else { + return Err(format!( + "Invalid Pattern: '{s}'. expected '->' but only '-'", + )); + } + + if let Some('}') = chars.get(pos + 2) { + } else { + return Err(format!("Invalid Pattern: '{s}'. expected '}}' after '->'",)); + } + + name.try_end_modifier()?; + pos += 1; // only skip '>', the next loop will skip '}' + } + (_, Part::Name(name)) if !is_valid_char(ch) => { + let tail: String = if name.is_name_empty() { + format!("Invalid '{ch}'") + } else { + format!("Invalid '{ch}' in '{name}'") + }; + return Err(format!("Invalid Pattern: '{s}'. {tail}")); + } + (_, Part::Name(_)) => { + cursor.push(ch); + } + } + + pos += 1; + } + + match cursor { + Part::Split(ref split) if !split.is_empty() => parts.push(cursor), + Part::Name(name) if !name.is_empty() => { + return Err(format!("Invalid Pattern: '{s}'. '{name}' is not closed")) + } + _ => {} + } + + let pattern = Self { parts, origin }; + pattern.check()?; + Ok(pattern) + } +} + +impl Pattern { + fn check(&self) -> Result<(), String> { + if self.len() == 0 { + return Err("Empty pattern is not allowed".to_string()); + } + + let mut map_keys = HashSet::new(); + let mut map_vals = HashSet::new(); + + for i in 0..self.len() { + let this_part = &self[i]; + let next_part = self.get(i + 1); + match (this_part, next_part) { + (Part::Split(split), _) if split.is_empty() => { + return Err(format!( + "Invalid Pattern: '{}'. Empty split is not allowed", + self.origin + )); + } + (Part::Name(name1), Some(Part::Name(name2))) => { + return Err(format!( + "Invalid Pattern: '{}'. consecutive names are not allowed: '{}' '{}'", + self.origin, name1, name2 + )); + } + (Part::Name(name), _) if name.is_name_empty() => { + if let Some(ref m) = name.start_modifier { + return Err(format!( + "Invalid Pattern: '{}'. only '{}' modifier is invalid", + self.origin, m + )); + } + } + (Part::Name(name), _) => match name.start_modifier { + Some(StartModifier::MapKey) => { + if map_keys.contains(&name.name) { + return Err(format!( + "Invalid Pattern: '{}'. Duplicate map key: '{}'", + self.origin, name.name + )); + } else { + map_keys.insert(&name.name); + } + } + Some(StartModifier::MapVal) => { + if map_vals.contains(&name.name) { + return Err(format!( + "Invalid Pattern: '{}'. Duplicate map val: '{}'", + self.origin, name.name + )); + } else { + map_vals.insert(&name.name); + } + } + _ => {} + }, + _ => {} + } + } + + if map_keys != map_vals { + return Err(format!( + "Invalid Pattern: '{}'. key and value not matched: '{}'", + self.origin, + map_keys + .symmetric_difference(&map_vals) + .map(|s| s.as_str()) + .collect::>() + .join(",") + )); + } + + Ok(()) + } +} + +impl std::fmt::Display for Pattern { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + write!(f, "{}", self.origin) + } +} + +#[derive(Debug, Default)] +pub struct DissectProcessor { + fields: Fields, + patterns: Vec, + ignore_missing: bool, + + // The character(s) that separate the appended fields. Default is an empty string. + append_separator: Option, +} + +impl DissectProcessor { + fn with_fields(&mut self, fields: Fields) { + self.fields = fields; + } + + fn with_ignore_missing(&mut self, ignore_missing: bool) { + self.ignore_missing = ignore_missing; + } + + fn with_patterns(&mut self, patterns: Vec) { + self.patterns = patterns; + } + + fn with_append_separator(&mut self, append_separator: String) { + self.append_separator = Some(append_separator); + } + + fn process_pattern(&self, chs: &[char], pattern: &Pattern) -> Result { + let mut map = Map::default(); + let mut pos = 0; + + let mut appends: HashMap> = HashMap::new(); + let mut maps: HashMap = HashMap::new(); + + let mut process_name_value = |name: &Name, value: String| { + let name_str = name.to_string(); + match name.start_modifier { + Some(StartModifier::NamedSkip) => { + // do nothing, ignore this match + } + Some(StartModifier::Append(order)) => { + appends + .entry(name_str) + .or_default() + .push((value, order.unwrap_or_default())); + } + Some(StartModifier::MapKey) => match maps.get(&name_str) { + Some(map_val) => { + map.insert(value, Value::String(map_val.to_string())); + } + None => { + maps.insert(name_str, value); + } + }, + Some(StartModifier::MapVal) => match maps.get(&name_str) { + Some(map_key) => { + map.insert(map_key, Value::String(value)); + } + None => { + maps.insert(name_str, value); + } + }, + None => { + map.insert(name.to_string(), Value::String(value)); + } + } + }; + + for i in 0..pattern.len() { + let this_part = &pattern[i]; + let next_part = pattern.get(i + 1); + match (this_part, next_part) { + // if Split part, and exactly matches, then move pos split.len() forward + (Part::Split(split), _) => { + let split_chs = split.chars().collect::>(); + let split_len = split_chs.len(); + if pos + split_len > chs.len() { + return Err(format!("'{split}' exceeds the input",)); + } + + if &chs[pos..pos + split_len] != split_chs.as_slice() { + return Err(format!( + "'{split}' does not match the input '{}'", + chs[pos..pos + split_len].iter().collect::() + )); + } + + pos += split_len; + } + + (Part::Name(name1), Some(Part::Name(name2))) => { + return Err(format!( + "consecutive names are not allowed: '{name1}' '{name2}'" + )); + } + + // if Name part is the last part, then the rest of the input is the value + (Part::Name(name), None) => { + let value = chs[pos..].iter().collect::(); + process_name_value(name, value); + } + + // if Name part, and next part is Split, then find the matched value of the name + (Part::Name(name), Some(Part::Split(split))) => { + let stop = split + .chars() + .next() + .ok_or("Empty split is not allowed".to_string())?; // this won't happen + let mut end = pos; + while end < chs.len() && chs[end] != stop { + end += 1; + } + + if !name.is_name_empty() { + let value = chs[pos..end].iter().collect::(); + process_name_value(name, value); + } + + if name.is_end_modifier_set() { + while end < chs.len() && chs[end] == stop { + end += 1; + } + end -= 1; // leave the last stop character to match the next split + } + + pos = end; + } + } + } + + if !appends.is_empty() { + let sep = match self.append_separator { + Some(ref sep) => sep, + None => " ", + }; + + for (name, mut values) in appends { + values.sort_by(|a, b| a.1.cmp(&b.1)); + let value = values.into_iter().map(|(a, _)| a).join(sep); + map.insert(name, Value::String(value)); + } + } + + Ok(map) + } + + fn process(&self, val: &str) -> Result { + let chs = val.chars().collect::>(); + + for pattern in &self.patterns { + if let Ok(map) = self.process_pattern(&chs, pattern) { + return Ok(map); + } + } + + Err("No matching pattern found".to_string()) + } +} + +impl TryFrom<&yaml_rust::yaml::Hash> for DissectProcessor { + type Error = String; + + fn try_from(value: &yaml_rust::yaml::Hash) -> Result { + let mut processor = Self::default(); + + for (k, v) in value.iter() { + let key = k + .as_str() + .ok_or(format!("key must be a string, but got '{k:?}'"))?; + + match key { + FIELD_NAME => processor.with_fields(Fields::one(yaml_field(v, FIELD_NAME)?)), + FIELDS_NAME => processor.with_fields(yaml_fields(v, FIELDS_NAME)?), + PATTERNS_NAME => { + let patterns = yaml_parse_strings(v, PATTERNS_NAME)?; + processor.with_patterns(patterns); + } + IGNORE_MISSING_NAME => { + processor.with_ignore_missing(yaml_bool(v, IGNORE_MISSING_NAME)?) + } + APPEND_SEPARATOR_NAME => { + processor.with_append_separator(yaml_string(v, APPEND_SEPARATOR_NAME)?) + } + _ => {} + } + } + + Ok(processor) + } +} + +impl Processor for DissectProcessor { + fn kind(&self) -> &str { + PROCESSOR_DISSECT + } + + fn ignore_missing(&self) -> bool { + self.ignore_missing + } + + fn fields(&self) -> &Fields { + &self.fields + } + + fn exec_field(&self, val: &Value, _field: &Field) -> Result { + match val { + Value::String(val) => match self.process(val) { + Ok(map) => Ok(map), + Err(e) => { + warn!("dissect processor: {}", e); + Ok(Map::default()) + } + }, + _ => Err(format!( + "{} processor: expect string value, but got {val:?}", + self.kind() + )), + } + } +} + +fn is_valid_char(ch: char) -> bool { + ch.is_alphanumeric() || ch == '_' +} + +#[cfg(test)] +mod tests { + use std::collections::HashMap; + + use super::{DissectProcessor, EndModifier, Name, Part, Pattern, StartModifier}; + use crate::etl::value::{Map, Value}; + + fn assert(pattern_str: &str, input: &str, expected: HashMap) { + let chs = input.chars().collect::>(); + let pattern = pattern_str.parse().unwrap(); + + let processor = DissectProcessor::default(); + let map = processor.process_pattern(&chs, &pattern).unwrap(); + + assert_eq!(map, Map::from(expected), "pattern: {}", pattern_str); + } + + #[test] + fn test_dissect_simple_pattern() { + let cases = [( + "%{clientip} %{ident} %{auth} [%{timestamp}] \"%{verb} %{request} HTTP/%{httpversion}\" %{status} %{size}", + vec![ + Part::Name("clientip".into()), + Part::Split(" ".into()), + Part::Name("ident".into()), + Part::Split(" ".into()), + Part::Name("auth".into()), + Part::Split(" [".into()), + Part::Name("timestamp".into()), + Part::Split("] \"".into()), + Part::Name("verb".into()), + Part::Split(" ".into()), + Part::Name("request".into()), + Part::Split(" HTTP/".into()), + Part::Name("httpversion".into()), + Part::Split("\" ".into()), + Part::Name("status".into()), + Part::Split(" ".into()), + Part::Name("size".into()), + ], + )]; + + for (pattern, expected) in cases.into_iter() { + let p: Pattern = pattern.parse().unwrap(); + assert_eq!(p.parts, expected); + } + } + + #[test] + fn test_dissect_modifier_pattern() { + let cases = [ + ( + "%{} %{}", + vec![ + Part::Name(Name { + name: "".into(), + start_modifier: None, + end_modifier: None, + }), + Part::Split(" ".into()), + Part::Name(Name { + name: "".into(), + start_modifier: None, + end_modifier: None, + }), + ], + ), + ( + "%{ts->} %{level}", + vec![ + Part::Name(Name { + name: "ts".into(), + start_modifier: None, + end_modifier: Some(EndModifier), + }), + Part::Split(" ".into()), + Part::Name("level".into()), + ], + ), + ( + "[%{ts}]%{->}[%{level}]", + vec![ + Part::Split("[".into()), + Part::Name(Name { + name: "ts".into(), + start_modifier: None, + end_modifier: None, + }), + Part::Split("]".into()), + Part::Name(Name { + name: "".into(), + start_modifier: None, + end_modifier: Some(EndModifier), + }), + Part::Split("[".into()), + Part::Name(Name { + name: "level".into(), + start_modifier: None, + end_modifier: None, + }), + Part::Split("]".into()), + ], + ), + ( + "%{+name} %{+name} %{+name} %{+name}", + vec![ + Part::Name(Name { + name: "name".into(), + start_modifier: Some(StartModifier::Append(None)), + end_modifier: None, + }), + Part::Split(" ".into()), + Part::Name(Name { + name: "name".into(), + start_modifier: Some(StartModifier::Append(None)), + end_modifier: None, + }), + Part::Split(" ".into()), + Part::Name(Name { + name: "name".into(), + start_modifier: Some(StartModifier::Append(None)), + end_modifier: None, + }), + Part::Split(" ".into()), + Part::Name(Name { + name: "name".into(), + start_modifier: Some(StartModifier::Append(None)), + end_modifier: None, + }), + ], + ), + ( + "%{+name/2} %{+name/4} %{+name/3} %{+name/1}", + vec![ + Part::Name(Name { + name: "name".into(), + start_modifier: Some(StartModifier::Append(Some(2))), + end_modifier: None, + }), + Part::Split(" ".into()), + Part::Name(Name { + name: "name".into(), + start_modifier: Some(StartModifier::Append(Some(4))), + end_modifier: None, + }), + Part::Split(" ".into()), + Part::Name(Name { + name: "name".into(), + start_modifier: Some(StartModifier::Append(Some(3))), + end_modifier: None, + }), + Part::Split(" ".into()), + Part::Name(Name { + name: "name".into(), + start_modifier: Some(StartModifier::Append(Some(1))), + end_modifier: None, + }), + ], + ), + ( + "%{clientip} %{?ident} %{?auth} [%{timestamp}]", + vec![ + Part::Name(Name { + name: "clientip".into(), + start_modifier: None, + end_modifier: None, + }), + Part::Split(" ".into()), + Part::Name(Name { + name: "ident".into(), + start_modifier: Some(StartModifier::NamedSkip), + end_modifier: None, + }), + Part::Split(" ".into()), + Part::Name(Name { + name: "auth".into(), + start_modifier: Some(StartModifier::NamedSkip), + end_modifier: None, + }), + Part::Split(" [".into()), + Part::Name(Name { + name: "timestamp".into(), + start_modifier: None, + end_modifier: None, + }), + Part::Split("]".into()), + ], + ), + ( + "[%{ts}] [%{level}] %{*p1}:%{&p1} %{*p2}:%{&p2}", + vec![ + Part::Split("[".into()), + Part::Name(Name { + name: "ts".into(), + start_modifier: None, + end_modifier: None, + }), + Part::Split("] [".into()), + Part::Name(Name { + name: "level".into(), + start_modifier: None, + end_modifier: None, + }), + Part::Split("] ".into()), + Part::Name(Name { + name: "p1".into(), + start_modifier: Some(StartModifier::MapKey), + end_modifier: None, + }), + Part::Split(":".into()), + Part::Name(Name { + name: "p1".into(), + start_modifier: Some(StartModifier::MapVal), + end_modifier: None, + }), + Part::Split(" ".into()), + Part::Name(Name { + name: "p2".into(), + start_modifier: Some(StartModifier::MapKey), + end_modifier: None, + }), + Part::Split(":".into()), + Part::Name(Name { + name: "p2".into(), + start_modifier: Some(StartModifier::MapVal), + end_modifier: None, + }), + ], + ), + ( + "%{&p1}:%{*p1}", + vec![ + Part::Name(Name { + name: "p1".into(), + start_modifier: Some(StartModifier::MapVal), + end_modifier: None, + }), + Part::Split(":".into()), + Part::Name(Name { + name: "p1".into(), + start_modifier: Some(StartModifier::MapKey), + end_modifier: None, + }), + ], + ), + ]; + + for (pattern, expected) in cases.into_iter() { + let p: Pattern = pattern.parse().unwrap(); + assert_eq!(p.parts, expected); + } + } + + #[test] + fn test_dissect_invalid_pattern() { + let cases = [ + ("", "Empty pattern is not allowed"), + ( + "%{name1}%{name2}", + "Invalid Pattern: '%{name1}%{name2}'. consecutive names are not allowed: 'name1' 'name2'" + ), + ( + "%{} %{ident", + "Invalid Pattern: '%{} %{ident'. 'ident' is not closed", + ), + ( + "%{->clientip} ", + "Invalid Pattern: '%{->clientip} '. expected '}' after '->'", + ), + ( + "%{/clientip} ", + "Invalid Pattern: '%{/clientip} '. Invalid '/'", + ), + ( + "%{+?clientip} ", + "Invalid Pattern: '%{+?clientip} '. Invalid '?'", + ), + ( + "%{+clientip/} ", + "Invalid Pattern: '%{+clientip/} '. Digit order must be set after '/'", + ), + ( + "%{+clientip/a} ", + "Invalid Pattern: '%{+clientip/a} '. Digit order must be set after '/'", + ), + ( + "%{clientip/1} ", + "Invalid Pattern: '%{clientip/1} '. Invalid '/' in 'clientip'", + ), + ( + "%{+clientip/1/2} ", + "Append Order modifier is already set to '1', cannot be set to 2", + ), + ( + "%{+/1} ", + "Invalid Pattern: '%{+/1} '. only '+/1' modifier is invalid", + ), + ( + "%{+} ", + "Invalid Pattern: '%{+} '. only '+' modifier is invalid", + ), + ( + "%{?} ", + "Invalid Pattern: '%{?} '. only '?' modifier is invalid", + ), + ( + "%{*} ", + "Invalid Pattern: '%{*} '. only '*' modifier is invalid", + ), + ( + "%{&} ", + "Invalid Pattern: '%{&} '. only '&' modifier is invalid", + ), + ( + "%{*ip}", + "Invalid Pattern: '%{*ip}'. key and value not matched: 'ip'" + ), + ( + "%{*ip} %{*ip}", + "Invalid Pattern: '%{*ip} %{*ip}'. Duplicate map key: 'ip'", + ), + ( + "%{*ip1} %{&ip2}", + "Invalid Pattern: '%{*ip1} %{&ip2}'. key and value not matched: 'ip1,ip2'" + ), + ]; + + for (pattern, expected) in cases.into_iter() { + let err = pattern.parse::().unwrap_err(); + assert_eq!(err, expected); + } + } + + #[test] + fn test_dissect_process() { + let expected = [ + ("timestamp", "30/Apr/1998:22:00:52 +0000"), + ("status", "200"), + ("clientip", "1.2.3.4"), + ("ident", "-"), + ("size", "3171"), + ( + "request", + "/english/venues/cities/images/montpellier/18.gif", + ), + ("auth", "-"), + ("verb", "GET"), + ("httpversion", "1.0"), + ] + .into_iter() + .map(|(k, v)| (k.to_string(), Value::String(v.to_string()))) + .collect::>(); + + { + // pattern start with Name + let pattern_str = "%{clientip} %{ident} %{auth} [%{timestamp}] \"%{verb} %{request} HTTP/%{httpversion}\" %{status} %{size}"; + let input = "1.2.3.4 - - [30/Apr/1998:22:00:52 +0000] \"GET /english/venues/cities/images/montpellier/18.gif HTTP/1.0\" 200 3171"; + + assert(pattern_str, input, expected.clone()); + } + + { + // pattern start with Split + let pattern_str = " %{clientip} %{ident} %{auth} [%{timestamp}] \"%{verb} %{request} HTTP/%{httpversion}\" %{status} %{size}"; + let input = " 1.2.3.4 - - [30/Apr/1998:22:00:52 +0000] \"GET /english/venues/cities/images/montpellier/18.gif HTTP/1.0\" 200 3171"; + + assert(pattern_str, input, expected); + } + } + + #[test] + fn test_dissect_right_padding_modifier() { + let cases = [ + ( + "%{ts->} %{level}", + "1998-08-10T17:15:42,466 WARN", + [("ts", "1998-08-10T17:15:42,466"), ("level", "WARN")], + ), + ( + "[%{ts}]%{->}[%{level}]", + "[1998-08-10T17:15:42,466] [WARN]", + [("ts", "1998-08-10T17:15:42,466"), ("level", "WARN")], + ), + ( + "[%{ts}]%{->}[%{level}]", + "[1998-08-10T17:15:42,466] [[[[WARN]", + [("ts", "1998-08-10T17:15:42,466"), ("level", "WARN")], + ), + ] + .into_iter() + .map(|(pattern, input, expected)| { + let map = expected + .into_iter() + .map(|(k, v)| (k.to_string(), Value::String(v.to_string()))); + (pattern, input, map) + }); + + for (pattern_str, input, expected) in cases { + assert( + pattern_str, + input, + expected.collect::>(), + ); + } + } + + #[test] + fn test_dissect_append_modifier() { + let cases = [ + ( + "%{+name} %{+name} %{+name} %{+name}", + "john jacob jingleheimer schmidt", + [("name", "john jacob jingleheimer schmidt")], + ), + ( + "%{+name/2} %{+name/4} %{+name/3} %{+name/1}", + "john jacob jingleheimer schmidt", + [("name", "schmidt john jingleheimer jacob")], + ), + ] + .into_iter() + .map(|(pattern, input, expected)| { + let map = expected + .into_iter() + .map(|(k, v)| (k.to_string(), Value::String(v.to_string()))); + (pattern, input, map) + }); + + for (pattern_str, input, expected) in cases { + assert( + pattern_str, + input, + expected.collect::>(), + ); + } + } + + #[test] + fn test_dissect_named_skip_modifier() { + let cases = [( + "%{clientip} %{?ident} %{?auth} [%{timestamp}]", + "1.2.3.4 - - [30/Apr/1998:22:00:52 +0000]", + [ + ("clientip", "1.2.3.4"), + ("timestamp", "30/Apr/1998:22:00:52 +0000"), + ], + )] + .into_iter() + .map(|(pattern, input, expected)| { + let map = expected + .into_iter() + .map(|(k, v)| (k.to_string(), Value::String(v.to_string()))); + (pattern, input, map) + }); + + for (pattern_str, input, expected) in cases { + assert( + pattern_str, + input, + expected.collect::>(), + ); + } + } + + #[test] + fn test_dissect_reference_keys() { + let cases = [ + ( + "[%{ts}] [%{level}] %{*p1}:%{&p1} %{*p2}:%{&p2}", + "[2018-08-10T17:15:42,466] [ERR] ip:1.2.3.4 error:REFUSED", + [ + ("ts", "2018-08-10T17:15:42,466"), + ("level", "ERR"), + ("ip", "1.2.3.4"), + ("error", "REFUSED"), + ], + ), + ( + "[%{ts}] [%{level}] %{&p1}:%{*p1} %{*p2}:%{&p2}", + "[2018-08-10T17:15:42,466] [ERR] ip:1.2.3.4 error:REFUSED", + [ + ("ts", "2018-08-10T17:15:42,466"), + ("level", "ERR"), + ("1.2.3.4", "ip"), + ("error", "REFUSED"), + ], + ), + ] + .into_iter() + .map(|(pattern, input, expected)| { + let map = expected + .into_iter() + .map(|(k, v)| (k.to_string(), Value::String(v.to_string()))); + (pattern, input, map) + }); + + for (pattern_str, input, expected) in cases { + assert( + pattern_str, + input, + expected.collect::>(), + ); + } + } +} diff --git a/src/pipeline/src/etl/processor/epoch.rs b/src/pipeline/src/etl/processor/epoch.rs new file mode 100644 index 000000000000..96a8695c9f76 --- /dev/null +++ b/src/pipeline/src/etl/processor/epoch.rs @@ -0,0 +1,205 @@ +// Copyright 2023 Greptime Team +// +// Licensed 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 crate::etl::field::{Field, Fields}; +use crate::etl::processor::{ + yaml_bool, yaml_field, yaml_fields, yaml_string, Processor, FIELDS_NAME, FIELD_NAME, + IGNORE_MISSING_NAME, +}; +use crate::etl::value::time::{ + MICROSECOND_RESOLUTION, MICRO_RESOLUTION, MILLISECOND_RESOLUTION, MILLI_RESOLUTION, + MS_RESOLUTION, NANOSECOND_RESOLUTION, NANO_RESOLUTION, NS_RESOLUTION, SECOND_RESOLUTION, + SEC_RESOLUTION, S_RESOLUTION, US_RESOLUTION, +}; +use crate::etl::value::{Epoch, Map, Value}; + +pub(crate) const PROCESSOR_EPOCH: &str = "epoch"; +const RESOLUTION_NAME: &str = "resolution"; + +#[derive(Debug, Default)] +enum Resolution { + Second, + #[default] + Milli, + Micro, + Nano, +} + +impl TryFrom<&str> for Resolution { + type Error = String; + + fn try_from(s: &str) -> Result { + match s { + SECOND_RESOLUTION | SEC_RESOLUTION | S_RESOLUTION => Ok(Resolution::Second), + MILLISECOND_RESOLUTION | MILLI_RESOLUTION | MS_RESOLUTION => Ok(Resolution::Milli), + MICROSECOND_RESOLUTION | MICRO_RESOLUTION | US_RESOLUTION => Ok(Resolution::Micro), + NANOSECOND_RESOLUTION | NANO_RESOLUTION | NS_RESOLUTION => Ok(Resolution::Nano), + _ => Err(format!("invalid resolution: {s}")), + } + } +} + +/// support string, integer, float, time, epoch +#[derive(Debug, Default)] +pub struct EpochProcessor { + fields: Fields, + resolution: Resolution, + ignore_missing: bool, + // description + // if + // ignore_failure + // on_failure + // tag +} + +impl EpochProcessor { + fn with_fields(&mut self, fields: Fields) { + self.fields = fields + } + + fn with_resolution(&mut self, resolution: Resolution) { + self.resolution = resolution; + } + + fn with_ignore_missing(&mut self, ignore_missing: bool) { + self.ignore_missing = ignore_missing; + } + + fn parse(&self, val: &Value) -> Result { + let t: i64 = match val { + Value::String(s) => s.parse::().map_err(|e| e.to_string())?, + Value::Int16(i) => *i as i64, + Value::Int32(i) => *i as i64, + Value::Int64(i) => *i, + Value::Uint8(i) => *i as i64, + Value::Uint16(i) => *i as i64, + Value::Uint32(i) => *i as i64, + Value::Uint64(i) => *i as i64, + Value::Float32(f) => *f as i64, + Value::Float64(f) => *f as i64, + + Value::Time(t) => match self.resolution { + Resolution::Second => t.timestamp(), + Resolution::Milli => t.timestamp_millis(), + Resolution::Micro => t.timestamp_micros(), + Resolution::Nano => t.timestamp_nanos(), + }, + + Value::Epoch(e) => match self.resolution { + Resolution::Second => e.timestamp(), + Resolution::Milli => e.timestamp_millis(), + Resolution::Micro => e.timestamp_micros(), + Resolution::Nano => e.timestamp_nanos(), + }, + + _ => { + return Err(format!( + "{PROCESSOR_EPOCH} processor: unsupported value {val}" + )) + } + }; + + match self.resolution { + Resolution::Second => Ok(Epoch::Second(t)), + Resolution::Milli => Ok(Epoch::Millisecond(t)), + Resolution::Micro => Ok(Epoch::Microsecond(t)), + Resolution::Nano => Ok(Epoch::Nanosecond(t)), + } + } + + fn process_field(&self, val: &Value, field: &Field) -> Result { + let key = match field.target_field { + Some(ref target_field) => target_field, + None => field.get_field(), + }; + + Ok(Map::one(key, Value::Epoch(self.parse(val)?))) + } +} + +impl TryFrom<&yaml_rust::yaml::Hash> for EpochProcessor { + type Error = String; + + fn try_from(hash: &yaml_rust::yaml::Hash) -> Result { + let mut processor = EpochProcessor::default(); + + for (k, v) in hash { + let key = k + .as_str() + .ok_or(format!("key must be a string, but got {k:?}"))?; + + match key { + FIELD_NAME => { + processor.with_fields(Fields::one(yaml_field(v, FIELD_NAME)?)); + } + FIELDS_NAME => { + processor.with_fields(yaml_fields(v, FIELDS_NAME)?); + } + RESOLUTION_NAME => { + let s = yaml_string(v, RESOLUTION_NAME)?.as_str().try_into()?; + processor.with_resolution(s); + } + IGNORE_MISSING_NAME => { + processor.with_ignore_missing(yaml_bool(v, IGNORE_MISSING_NAME)?); + } + + _ => {} + } + } + + Ok(processor) + } +} + +impl Processor for EpochProcessor { + fn kind(&self) -> &str { + PROCESSOR_EPOCH + } + + fn ignore_missing(&self) -> bool { + self.ignore_missing + } + + fn fields(&self) -> &Fields { + &self.fields + } + + fn exec_field(&self, val: &Value, field: &Field) -> Result { + self.process_field(val, field) + } +} + +#[cfg(test)] +mod tests { + use super::EpochProcessor; + use crate::etl::value::Value; + + #[test] + fn test_parse_epoch() { + let mut processor = EpochProcessor::default(); + processor.with_resolution(super::Resolution::Second); + + let values = [ + Value::String("1573840000".into()), + Value::Int32(1573840000), + Value::Uint64(1573840000), + Value::Float32(1573840000.0), + ]; + + for value in values { + let parsed = processor.parse(&value).unwrap(); + assert_eq!(parsed, super::Epoch::Second(1573840000)); + } + } +} diff --git a/src/pipeline/src/etl/processor/letter.rs b/src/pipeline/src/etl/processor/letter.rs new file mode 100644 index 000000000000..e533536769d2 --- /dev/null +++ b/src/pipeline/src/etl/processor/letter.rs @@ -0,0 +1,188 @@ +// Copyright 2023 Greptime Team +// +// Licensed 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 crate::etl::field::{Field, Fields}; +use crate::etl::processor::{ + yaml_bool, yaml_field, yaml_fields, yaml_string, Processor, FIELDS_NAME, FIELD_NAME, + IGNORE_MISSING_NAME, METHOD_NAME, +}; +use crate::etl::value::{Map, Value}; + +pub(crate) const PROCESSOR_LETTER: &str = "letter"; + +#[derive(Debug, Default)] +enum Method { + Upper, + #[default] + Lower, + Capital, +} + +impl std::fmt::Display for Method { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + match self { + Method::Upper => write!(f, "upper"), + Method::Lower => write!(f, "lower"), + Method::Capital => write!(f, "capital"), + } + } +} + +impl std::str::FromStr for Method { + type Err = String; + + fn from_str(s: &str) -> Result { + match s.to_lowercase().as_str() { + "upper" => Ok(Method::Upper), + "lower" => Ok(Method::Lower), + "capital" => Ok(Method::Capital), + _ => Err(format!("invalid method: {s}")), + } + } +} + +/// only support string value +#[derive(Debug, Default)] +pub struct LetterProcessor { + fields: Fields, + method: Method, + ignore_missing: bool, +} + +impl LetterProcessor { + fn with_fields(&mut self, fields: Fields) { + self.fields = fields; + } + + fn with_method(&mut self, method: Method) { + self.method = method; + } + + fn with_ignore_missing(&mut self, ignore_missing: bool) { + self.ignore_missing = ignore_missing; + } + + fn process_field(&self, val: &str, field: &Field) -> Result { + let processed = match self.method { + Method::Upper => val.to_uppercase(), + Method::Lower => val.to_lowercase(), + Method::Capital => capitalize(val), + }; + let val = Value::String(processed); + + let key = match field.target_field { + Some(ref target_field) => target_field, + None => field.get_field(), + }; + + Ok(Map::one(key, val)) + } +} + +impl TryFrom<&yaml_rust::yaml::Hash> for LetterProcessor { + type Error = String; + + fn try_from(value: &yaml_rust::yaml::Hash) -> Result { + let mut processor = LetterProcessor::default(); + + for (k, v) in value.iter() { + let key = k + .as_str() + .ok_or(format!("key must be a string, but got {k:?}"))?; + match key { + FIELD_NAME => { + processor.with_fields(Fields::one(yaml_field(v, FIELD_NAME)?)); + } + FIELDS_NAME => { + processor.with_fields(yaml_fields(v, FIELDS_NAME)?); + } + METHOD_NAME => { + let method = yaml_string(v, METHOD_NAME)?; + processor.with_method(method.parse()?); + } + IGNORE_MISSING_NAME => { + processor.with_ignore_missing(yaml_bool(v, IGNORE_MISSING_NAME)?); + } + _ => {} + } + } + + Ok(processor) + } +} + +impl Processor for LetterProcessor { + fn kind(&self) -> &str { + PROCESSOR_LETTER + } + + fn ignore_missing(&self) -> bool { + self.ignore_missing + } + + fn fields(&self) -> &Fields { + &self.fields + } + + fn exec_field(&self, val: &Value, field: &Field) -> Result { + match val { + Value::String(val) => self.process_field(val, field), + _ => Err(format!( + "{} processor: expect string value, but got {val:?}", + self.kind() + )), + } + } +} + +fn capitalize(s: &str) -> String { + let mut c = s.chars(); + match c.next() { + None => String::new(), + Some(f) => f.to_uppercase().collect::() + c.as_str(), + } +} + +#[cfg(test)] +mod tests { + use crate::etl::field::Fields; + use crate::etl::processor::letter::{LetterProcessor, Method}; + use crate::etl::value::{Map, Value}; + + #[test] + fn test_process() { + let field = "letter"; + let ff: crate::etl::processor::Field = field.parse().unwrap(); + let mut processor = LetterProcessor::default(); + processor.with_fields(Fields::one(ff.clone())); + + { + processor.with_method(Method::Upper); + let processed = processor.process_field("pipeline", &ff).unwrap(); + assert_eq!(Map::one(field, Value::String("PIPELINE".into())), processed) + } + + { + processor.with_method(Method::Lower); + let processed = processor.process_field("Pipeline", &ff).unwrap(); + assert_eq!(Map::one(field, Value::String("pipeline".into())), processed) + } + + { + processor.with_method(Method::Capital); + let processed = processor.process_field("pipeline", &ff).unwrap(); + assert_eq!(Map::one(field, Value::String("Pipeline".into())), processed) + } + } +} diff --git a/src/pipeline/src/etl/processor/mod.rs b/src/pipeline/src/etl/processor/mod.rs new file mode 100644 index 000000000000..96e8a629f252 --- /dev/null +++ b/src/pipeline/src/etl/processor/mod.rs @@ -0,0 +1,224 @@ +// Copyright 2023 Greptime Team +// +// Licensed 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. + +pub mod cmcd; +pub mod csv; +pub mod date; +pub mod dissect; +pub mod epoch; +pub mod letter; +pub mod regex; +pub mod urlencoding; + +use std::sync::Arc; + +use cmcd::CMCDProcessor; +use common_telemetry::warn; +use csv::CsvProcessor; +use date::DateProcessor; +use dissect::DissectProcessor; +use epoch::EpochProcessor; +use letter::LetterProcessor; +use regex::RegexProcessor; +use urlencoding::UrlEncodingProcessor; + +use crate::etl::field::{Field, Fields}; +use crate::etl::value::{Array, Map, Value}; + +const FIELD_NAME: &str = "field"; +const FIELDS_NAME: &str = "fields"; +const IGNORE_MISSING_NAME: &str = "ignore_missing"; +const METHOD_NAME: &str = "method"; +const PATTERNS_NAME: &str = "patterns"; + +// const IF_NAME: &str = "if"; +// const IGNORE_FAILURE_NAME: &str = "ignore_failure"; +// const ON_FAILURE_NAME: &str = "on_failure"; +// const TAG_NAME: &str = "tag"; + +pub trait Processor: std::fmt::Debug + Send + Sync + 'static { + fn fields(&self) -> &Fields; + fn kind(&self) -> &str; + fn ignore_missing(&self) -> bool; + + fn ignore_processor_array_failure(&self) -> bool { + true + } + + /// default behavior does nothing and returns the input value + fn exec_field(&self, val: &Value, field: &Field) -> Result { + Ok(Map::one(field.get_field(), val.clone())) + } + + fn exec_map(&self, mut map: Map) -> Result { + for ff @ Field { field, .. } in self.fields().iter() { + match map.get(field) { + Some(v) => { + map.extend(self.exec_field(v, ff)?); + } + None if self.ignore_missing() => {} + None => { + return Err(format!( + "{} processor: field '{field}' is required but missing in {map}", + self.kind(), + )) + } + } + } + + Ok(Value::Map(map)) + } + + fn exec_array(&self, arr: Array) -> Result { + let mut values = vec![]; + for val in arr.into_iter() { + match val { + Value::Map(map) => { + values.push(self.exec_map(map)?); + } + _ if self.ignore_processor_array_failure() => { + warn!("expected a map, but got {val}") + } + _ => return Err(format!("expected a map, but got {}", val)), + } + } + + Ok(Value::Array(Array { values })) + } + + fn exec(&self, val: Value) -> Result { + match val { + Value::Map(map) => self.exec_map(map), + Value::Array(arr) => self.exec_array(arr), + _ => Err(format!("expected a map or array, but got {}", val)), + } + } +} + +#[derive(Debug, Default, Clone)] +pub struct Processors { + pub processors: Vec>, +} + +impl Processors { + pub fn new() -> Self { + Processors { processors: vec![] } + } +} + +impl std::ops::Deref for Processors { + type Target = Vec>; + + fn deref(&self) -> &Self::Target { + &self.processors + } +} + +impl TryFrom<&Vec> for Processors { + type Error = String; + + fn try_from(vec: &Vec) -> Result { + let mut processors = vec![]; + + for doc in vec { + processors.push(parse_processor(doc)?); + } + + Ok(Processors { processors }) + } +} + +fn parse_processor(doc: &yaml_rust::Yaml) -> Result, String> { + let map = doc.as_hash().ok_or("processor must be a map".to_string())?; + + let key = map + .keys() + .next() + .ok_or("processor must have a string key".to_string())?; + + let value = map + .get(key) + .unwrap() + .as_hash() + .expect("processor value must be a map"); + + let str_key = key + .as_str() + .ok_or("processor key must be a string".to_string())?; + + let processor: Arc = match str_key { + cmcd::PROCESSOR_CMCD => Arc::new(CMCDProcessor::try_from(value)?), + csv::PROCESSOR_CSV => Arc::new(CsvProcessor::try_from(value)?), + date::PROCESSOR_DATE => Arc::new(DateProcessor::try_from(value)?), + dissect::PROCESSOR_DISSECT => Arc::new(DissectProcessor::try_from(value)?), + epoch::PROCESSOR_EPOCH => Arc::new(EpochProcessor::try_from(value)?), + letter::PROCESSOR_LETTER => Arc::new(LetterProcessor::try_from(value)?), + regex::PROCESSOR_REGEX => Arc::new(RegexProcessor::try_from(value)?), + urlencoding::PROCESSOR_URL_ENCODING => Arc::new(UrlEncodingProcessor::try_from(value)?), + _ => return Err(format!("unsupported {} processor", str_key)), + }; + + Ok(processor) +} + +pub(crate) fn yaml_string(v: &yaml_rust::Yaml, field: &str) -> Result { + v.as_str() + .map(|s| s.trim().to_string()) + .ok_or(format!("'{field}' must be a string")) +} + +pub(crate) fn yaml_strings(v: &yaml_rust::Yaml, field: &str) -> Result, String> { + let vec = v + .as_vec() + .ok_or(format!("'{field}' must be a list of strings",))? + .iter() + .map(|v| v.as_str().unwrap_or_default().into()) + .collect(); + Ok(vec) +} + +pub(crate) fn yaml_bool(v: &yaml_rust::Yaml, field: &str) -> Result { + v.as_bool().ok_or(format!("'{field}' must be a boolean")) +} + +pub(crate) fn yaml_parse_string(v: &yaml_rust::Yaml, field: &str) -> Result +where + T: std::str::FromStr, + T::Err: ToString, +{ + yaml_string(v, field)? + .parse::() + .map_err(|e| e.to_string()) +} + +pub(crate) fn yaml_parse_strings(v: &yaml_rust::Yaml, field: &str) -> Result, String> +where + T: std::str::FromStr, + T::Err: ToString, +{ + yaml_strings(v, field).and_then(|v| { + v.into_iter() + .map(|s| s.parse::().map_err(|e| e.to_string())) + .collect() + }) +} + +pub(crate) fn yaml_fields(v: &yaml_rust::Yaml, field: &str) -> Result { + let v = yaml_parse_strings(v, field)?; + Fields::new(v) +} + +pub(crate) fn yaml_field(v: &yaml_rust::Yaml, field: &str) -> Result { + yaml_parse_string(v, field) +} diff --git a/src/pipeline/src/etl/processor/regex.rs b/src/pipeline/src/etl/processor/regex.rs new file mode 100644 index 000000000000..8aba43436155 --- /dev/null +++ b/src/pipeline/src/etl/processor/regex.rs @@ -0,0 +1,315 @@ +// Copyright 2023 Greptime Team +// +// Licensed 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. + +// field_name and prefix with comma separated, like: +// name, new_name +const PATTERNS_NAME: &str = "patterns"; + +pub(crate) const PROCESSOR_REGEX: &str = "regex"; + +use lazy_static::lazy_static; +use regex::Regex; + +use crate::etl::field::Fields; +use crate::etl::processor::{ + yaml_bool, yaml_field, yaml_fields, yaml_strings, Field, Processor, FIELDS_NAME, FIELD_NAME, + IGNORE_MISSING_NAME, +}; +use crate::etl::value::{Map, Value}; + +lazy_static! { + static ref GROUPS_NAME_REGEX: Regex = Regex::new(r"\(\?P?<([[:word:]]+)>.+?\)").unwrap(); +} + +fn get_regex_group_names(s: &str) -> Vec { + GROUPS_NAME_REGEX + .captures_iter(s) + .filter_map(|c| c.get(1).map(|m| m.as_str().to_string())) + .collect() +} + +#[derive(Debug)] +struct GroupRegex { + origin: String, + regex: Regex, + groups: Vec, +} + +impl std::fmt::Display for GroupRegex { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + let groups = self.groups.join(", "); + write!(f, "{}, groups: [{groups}]", self.origin) + } +} + +impl std::str::FromStr for GroupRegex { + type Err = String; + + fn from_str(origin: &str) -> Result { + let groups = get_regex_group_names(origin); + if groups.is_empty() { + return Err(format!("no named group found in regex {origin}")); + } + + let regex = Regex::new(origin).map_err(|e| e.to_string())?; + Ok(GroupRegex { + origin: origin.into(), + regex, + groups, + }) + } +} + +/// only support string value +/// if no value found from a pattern, the target_field will be ignored +#[derive(Debug, Default)] +pub struct RegexProcessor { + fields: Fields, + patterns: Vec, + ignore_missing: bool, +} + +impl RegexProcessor { + fn with_fields(&mut self, fields: Fields) { + self.fields = fields; + } + + fn try_with_patterns(&mut self, patterns: Vec) -> Result<(), String> { + let mut rs = vec![]; + for pattern in patterns { + let gr = pattern.parse()?; + rs.push(gr); + } + self.patterns = rs; + Ok(()) + } + + fn with_ignore_missing(&mut self, ignore_missing: bool) { + self.ignore_missing = ignore_missing; + } + + fn check(self) -> Result { + if self.fields.is_empty() { + return Err(format!( + "no valid field found in {} processor", + PROCESSOR_REGEX + )); + } + + if self.patterns.is_empty() { + return Err(format!( + "no valid pattern found in {} processor", + PROCESSOR_REGEX + )); + } + + Ok(self) + } + + fn process_field(&self, val: &str, field: &Field, gr: &GroupRegex) -> Result { + let mut map = Map::default(); + + if let Some(captures) = gr.regex.captures(val) { + for group in &gr.groups { + if let Some(capture) = captures.name(group) { + let value = capture.as_str().to_string(); + let prefix = match &field.target_field { + Some(s) => s, + None => &field.field, + }; + + let key = format!("{prefix}_{group}"); + + map.insert(key, Value::String(value)); + } + } + } + + Ok(map) + } +} + +impl TryFrom<&yaml_rust::yaml::Hash> for RegexProcessor { + type Error = String; + + fn try_from(value: &yaml_rust::yaml::Hash) -> Result { + let mut processor = RegexProcessor::default(); + + for (k, v) in value.iter() { + let key = k + .as_str() + .ok_or(format!("key must be a string, but got {k:?}"))?; + match key { + FIELD_NAME => { + processor.with_fields(Fields::one(yaml_field(v, FIELD_NAME)?)); + } + FIELDS_NAME => { + processor.with_fields(yaml_fields(v, FIELDS_NAME)?); + } + PATTERNS_NAME => { + processor.try_with_patterns(yaml_strings(v, PATTERNS_NAME)?)?; + } + IGNORE_MISSING_NAME => { + processor.with_ignore_missing(yaml_bool(v, IGNORE_MISSING_NAME)?); + } + _ => {} + } + } + + processor.check() + } +} + +impl Processor for RegexProcessor { + fn kind(&self) -> &str { + PROCESSOR_REGEX + } + + fn ignore_missing(&self) -> bool { + self.ignore_missing + } + + fn fields(&self) -> &Fields { + &self.fields + } + + fn exec_field(&self, val: &Value, field: &Field) -> Result { + match val { + Value::String(val) => { + let mut map = Map::default(); + for gr in &self.patterns { + let m = self.process_field(val, field, gr)?; + map.extend(m); + } + Ok(map) + } + _ => Err(format!( + "{} processor: expect string value, but got {val:?}", + self.kind() + )), + } + } +} +#[cfg(test)] +mod tests { + use itertools::Itertools; + + use super::RegexProcessor; + use crate::etl::field::Fields; + use crate::etl::processor::Processor; + use crate::etl::value::{Map, Value}; + + #[test] + fn test_process() { + let mut processor = RegexProcessor::default(); + + let cc = "[c=c,n=US_CA_SANJOSE,o=55155]"; + let cg = "[a=12.34.567.89,b=12345678,c=g,n=US_CA_SANJOSE,o=20940]"; + let co = "[a=987.654.321.09,c=o]"; + let cp = "[c=p,n=US_CA_SANJOSE,o=55155]"; + let cw = "[c=w,n=US_CA_SANJOSE,o=55155]"; + let breadcrumbs = Value::String([cc, cg, co, cp, cw].iter().join(",")); + + let values = [ + ("breadcrumbs", breadcrumbs.clone()), + ("breadcrumbs_parent", Value::String(cc.to_string())), + ("breadcrumbs_edge", Value::String(cg.to_string())), + ("breadcrumbs_origin", Value::String(co.to_string())), + ("breadcrumbs_peer", Value::String(cp.to_string())), + ("breadcrumbs_wrapper", Value::String(cw.to_string())), + ] + .into_iter() + .map(|(k, v)| (k.to_string(), v)) + .collect(); + let temporary_map = Map { values }; + + { + // single field (with prefix), multiple patterns + let ff = ["breadcrumbs, breadcrumbs"] + .iter() + .map(|f| f.parse().unwrap()) + .collect(); + processor.with_fields(Fields::new(ff).unwrap()); + + let ccr = "(?\\[[^\\[]*c=c[^\\]]*\\])"; + let cgr = "(?\\[[^\\[]*c=g[^\\]]*\\])"; + let cor = "(?\\[[^\\[]*c=o[^\\]]*\\])"; + let cpr = "(?\\[[^\\[]*c=p[^\\]]*\\])"; + let cwr = "(?\\[[^\\[]*c=w[^\\]]*\\])"; + let patterns = [ccr, cgr, cor, cpr, cwr] + .iter() + .map(|p| p.to_string()) + .collect(); + processor.try_with_patterns(patterns).unwrap(); + + let mut map = Map::default(); + map.insert("breadcrumbs", breadcrumbs.clone()); + let processed_val = processor.exec_map(map).unwrap(); + + assert_eq!(processed_val, Value::Map(temporary_map.clone())); + } + + { + // multiple fields (with prefix), multiple patterns + let ff = [ + "breadcrumbs_parent, parent", + "breadcrumbs_edge, edge", + "breadcrumbs_origin, origin", + "breadcrumbs_peer, peer", + "breadcrumbs_wrapper, wrapper", + ] + .iter() + .map(|f| f.parse().unwrap()) + .collect(); + processor.with_fields(Fields::new(ff).unwrap()); + + let patterns = [ + "a=(?[^,\\]]+)", + "b=(?[^,\\]]+)", + "k=(?[^,\\]]+)", + "l=(?[^,\\]]+)", + "m=(?[^,\\]]+)", + "n=(?[^,\\]]+)", + "o=(?[^,\\]]+)", + ] + .iter() + .map(|p| p.to_string()) + .collect(); + processor.try_with_patterns(patterns).unwrap(); + + let new_values = vec![ + ("edge_ip", Value::String("12.34.567.89".to_string())), + ("edge_request_id", Value::String("12345678".to_string())), + ("edge_geo", Value::String("US_CA_SANJOSE".to_string())), + ("edge_asn", Value::String("20940".to_string())), + ("origin_ip", Value::String("987.654.321.09".to_string())), + ("peer_asn", Value::String("55155".to_string())), + ("peer_geo", Value::String("US_CA_SANJOSE".to_string())), + ("parent_asn", Value::String("55155".to_string())), + ("parent_geo", Value::String("US_CA_SANJOSE".to_string())), + ("wrapper_asn", Value::String("55155".to_string())), + ("wrapper_geo", Value::String("US_CA_SANJOSE".to_string())), + ] + .into_iter() + .map(|(k, v)| (k.to_string(), v)) + .collect(); + + let actual_val = processor.exec_map(temporary_map.clone()).unwrap(); + let mut expected_map = temporary_map.clone(); + expected_map.extend(Map { values: new_values }); + + assert_eq!(Value::Map(expected_map), actual_val); + } + } +} diff --git a/src/pipeline/src/etl/processor/urlencoding.rs b/src/pipeline/src/etl/processor/urlencoding.rs new file mode 100644 index 000000000000..c0d1669f85de --- /dev/null +++ b/src/pipeline/src/etl/processor/urlencoding.rs @@ -0,0 +1,177 @@ +// Copyright 2023 Greptime Team +// +// Licensed 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 urlencoding::{decode, encode}; + +use crate::etl::field::{Field, Fields}; +use crate::etl::processor::{ + yaml_bool, yaml_field, yaml_fields, yaml_string, FIELDS_NAME, FIELD_NAME, IGNORE_MISSING_NAME, + METHOD_NAME, +}; +use crate::etl::value::{Map, Value}; + +pub(crate) const PROCESSOR_URL_ENCODING: &str = "urlencoding"; + +#[derive(Debug, Default)] +enum Method { + #[default] + Decode, + Encode, +} + +impl std::fmt::Display for Method { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + match self { + Method::Decode => write!(f, "decode"), + Method::Encode => write!(f, "encode"), + } + } +} + +impl std::str::FromStr for Method { + type Err = String; + + fn from_str(s: &str) -> Result { + match s { + "decode" => Ok(Method::Decode), + "encode" => Ok(Method::Encode), + _ => Err(format!("invalid method: {s}")), + } + } +} + +/// only support string value +#[derive(Debug, Default)] +pub struct UrlEncodingProcessor { + fields: Fields, + method: Method, + ignore_missing: bool, +} + +impl UrlEncodingProcessor { + fn with_fields(&mut self, fields: Fields) { + self.fields = fields; + } + + fn with_ignore_missing(&mut self, ignore_missing: bool) { + self.ignore_missing = ignore_missing; + } + + fn with_method(&mut self, method: Method) { + self.method = method; + } + + fn process_field(&self, val: &str, field: &Field) -> Result { + let processed = match self.method { + Method::Encode => encode(val).to_string(), + Method::Decode => decode(val).map_err(|e| e.to_string())?.into_owned(), + }; + let val = Value::String(processed); + + let key = match field.target_field { + Some(ref target_field) => target_field, + None => field.get_field(), + }; + + Ok(Map::one(key, val)) + } +} + +impl TryFrom<&yaml_rust::yaml::Hash> for UrlEncodingProcessor { + type Error = String; + + fn try_from(value: &yaml_rust::yaml::Hash) -> Result { + let mut processor = UrlEncodingProcessor::default(); + + for (k, v) in value.iter() { + let key = k + .as_str() + .ok_or(format!("key must be a string, but got {k:?}"))?; + match key { + FIELD_NAME => { + processor.with_fields(Fields::one(yaml_field(v, FIELD_NAME)?)); + } + FIELDS_NAME => { + processor.with_fields(yaml_fields(v, FIELDS_NAME)?); + } + + IGNORE_MISSING_NAME => { + processor.with_ignore_missing(yaml_bool(v, IGNORE_MISSING_NAME)?); + } + + METHOD_NAME => { + let method = yaml_string(v, METHOD_NAME)?; + processor.with_method(method.parse()?); + } + + _ => {} + } + } + + Ok(processor) + } +} + +impl crate::etl::processor::Processor for UrlEncodingProcessor { + fn kind(&self) -> &str { + PROCESSOR_URL_ENCODING + } + + fn ignore_missing(&self) -> bool { + self.ignore_missing + } + + fn fields(&self) -> &Fields { + &self.fields + } + + fn exec_field(&self, val: &Value, field: &Field) -> Result { + match val { + Value::String(val) => self.process_field(val, field), + _ => Err(format!( + "{} processor: expect string value, but got {val:?}", + self.kind() + )), + } + } +} + +#[cfg(test)] +mod tests { + use crate::etl::field::{Field, Fields}; + use crate::etl::processor::urlencoding::UrlEncodingProcessor; + use crate::etl::value::{Map, Value}; + + #[test] + fn test_decode_url() { + let field = "url"; + let ff: Field = field.parse().unwrap(); + + let decoded = "//BC/[a=6.7.8.9,c=g,k=0,l=1]"; + let encoded = "%2F%2FBC%2F%5Ba%3D6.7.8.9%2Cc%3Dg%2Ck%3D0%2Cl%3D1%5D"; + + let mut processor = UrlEncodingProcessor::default(); + processor.with_fields(Fields::one(ff.clone())); + + { + let result = processor.process_field(encoded, &ff).unwrap(); + assert_eq!(Map::one(field, Value::String(decoded.into())), result) + } + { + processor.with_method(super::Method::Encode); + let result = processor.process_field(decoded, &ff).unwrap(); + assert_eq!(Map::one(field, Value::String(encoded.into())), result) + } + } +} diff --git a/src/pipeline/src/etl/transform/index.rs b/src/pipeline/src/etl/transform/index.rs new file mode 100644 index 000000000000..674df720f8c3 --- /dev/null +++ b/src/pipeline/src/etl/transform/index.rs @@ -0,0 +1,57 @@ +// Copyright 2023 Greptime Team +// +// Licensed 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. + +const INDEX_TIMESTAMP: &str = "timestamp"; +const INDEX_TAG: &str = "tag"; +const INDEX_FULLTEXT: &str = "fulltext"; + +#[derive(Debug, PartialEq, Eq, Clone, Copy)] +pub enum Index { + Timestamp, + Tag, + Fulltext, +} + +impl std::fmt::Display for Index { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + let index = match self { + Index::Timestamp => INDEX_TIMESTAMP, + Index::Tag => INDEX_TAG, + Index::Fulltext => INDEX_FULLTEXT, + }; + + write!(f, "{}", index) + } +} + +impl TryFrom for Index { + type Error = String; + + fn try_from(value: String) -> Result { + Index::try_from(value.as_str()) + } +} + +impl TryFrom<&str> for Index { + type Error = String; + + fn try_from(value: &str) -> Result { + match value { + INDEX_TIMESTAMP => Ok(Index::Timestamp), + INDEX_TAG => Ok(Index::Tag), + INDEX_FULLTEXT => Ok(Index::Fulltext), + _ => Err(format!("unsupported index type: {}", value)), + } + } +} diff --git a/src/pipeline/src/etl/transform/mod.rs b/src/pipeline/src/etl/transform/mod.rs new file mode 100644 index 000000000000..991aa05df644 --- /dev/null +++ b/src/pipeline/src/etl/transform/mod.rs @@ -0,0 +1,205 @@ +// Copyright 2023 Greptime Team +// +// Licensed 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. + +pub mod index; +pub mod transformer; + +use itertools::Itertools; + +use crate::etl::field::Fields; +use crate::etl::processor::{yaml_field, yaml_fields, yaml_string}; +use crate::etl::transform::index::Index; +use crate::etl::value::Value; + +const TRANSFORM_FIELD: &str = "field"; +const TRANSFORM_FIELDS: &str = "fields"; +const TRANSFORM_TYPE: &str = "type"; +const TRANSFORM_INDEX: &str = "index"; +const TRANSFORM_DEFAULT: &str = "default"; + +pub use transformer::greptime::GreptimeTransformer; +// pub use transformer::noop::NoopTransformer; + +pub trait Transformer: std::fmt::Display + Sized + Send + Sync + 'static { + type Output; + + fn new(transforms: Transforms) -> Result; + fn transform(&self, val: crate::etl::value::Value) -> Result; +} + +#[derive(Debug, Default, Clone)] +pub struct Transforms { + transforms: Vec, +} + +impl std::fmt::Display for Transforms { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + let transforms = self + .transforms + .iter() + .map(|field| field.to_string()) + .join(", "); + + write!(f, "{}", transforms) + } +} + +impl std::ops::Deref for Transforms { + type Target = Vec; + + fn deref(&self) -> &Self::Target { + &self.transforms + } +} + +impl std::ops::DerefMut for Transforms { + fn deref_mut(&mut self) -> &mut Self::Target { + &mut self.transforms + } +} + +impl TryFrom<&Vec> for Transforms { + type Error = String; + + fn try_from(docs: &Vec) -> Result { + let mut transforms = vec![]; + + for doc in docs { + let transform: Transform = doc + .as_hash() + .ok_or("transform element must be a map".to_string())? + .try_into()?; + transforms.push(transform); + } + + Ok(Transforms { transforms }) + } +} + +/// only field is required +#[derive(Debug, Clone)] +pub struct Transform { + pub fields: Fields, + + pub type_: Value, + + pub default: Option, + + pub index: Option, +} + +impl std::fmt::Display for Transform { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + let index = if let Some(index) = &self.index { + format!(", index: {}", index) + } else { + "".to_string() + }; + + let fields = format!("field(s): {}", self.fields); + let type_ = format!("type: {}", self.type_); + + write!(f, "{type_}{index}, {fields}") + } +} + +impl Default for Transform { + fn default() -> Self { + Transform { + fields: Fields::default(), + type_: Value::Null, + default: None, + index: None, + } + } +} + +impl Transform { + fn with_fields(&mut self, fields: Fields) { + self.fields = fields; + } + + fn with_type(&mut self, type_: Value) { + self.type_ = type_; + } + + fn try_default(&mut self, default: Value) -> Result<(), String> { + match (&self.type_, &default) { + (Value::Null, _) => Err(format!( + "transform {} type MUST BE set before default {}", + self.fields, &default, + )), + (_, Value::Null) => Ok(()), // if default is not set, then it will be regarded as default null + (_, _) => { + let target = self + .type_ + .parse_str_value(default.to_str_value().as_str())?; + self.default = Some(target); + Ok(()) + } + } + } + + fn with_index(&mut self, index: Index) { + self.index = Some(index); + } + + pub(crate) fn get_default(&self) -> Option<&Value> { + self.default.as_ref() + } +} + +impl TryFrom<&yaml_rust::yaml::Hash> for Transform { + type Error = String; + + fn try_from(hash: &yaml_rust::yaml::Hash) -> Result { + let mut transform = Transform::default(); + + let mut default_opt = None; + + for (k, v) in hash { + let key = k.as_str().ok_or("key must be a string")?; + match key { + TRANSFORM_FIELD => { + transform.with_fields(Fields::one(yaml_field(v, TRANSFORM_FIELD)?)); + } + + TRANSFORM_FIELDS => { + transform.with_fields(yaml_fields(v, TRANSFORM_FIELDS)?); + } + + TRANSFORM_TYPE => { + let t = yaml_string(v, TRANSFORM_TYPE)?; + transform.with_type(Value::parse_str_type(&t)?); + } + + TRANSFORM_INDEX => { + let index = yaml_string(v, TRANSFORM_INDEX)?; + transform.with_index(index.try_into()?); + } + + TRANSFORM_DEFAULT => { + default_opt = Some(Value::try_from(v)?); + } + _ => {} + } + } + + if let Some(default) = default_opt { + transform.try_default(default)?; + } + + Ok(transform) + } +} diff --git a/src/pipeline/src/etl/transform/transformer/greptime/coerce.rs b/src/pipeline/src/etl/transform/transformer/greptime/coerce.rs new file mode 100644 index 000000000000..6b077a22dca5 --- /dev/null +++ b/src/pipeline/src/etl/transform/transformer/greptime/coerce.rs @@ -0,0 +1,310 @@ +// Copyright 2023 Greptime Team +// +// Licensed 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 greptime_proto::v1::value::ValueData; +use greptime_proto::v1::{ColumnDataType, ColumnSchema, SemanticType}; + +use crate::etl::transform::index::Index; +use crate::etl::transform::Transform; +use crate::etl::value::{Epoch, Time, Value}; + +impl TryFrom for ValueData { + type Error = String; + + fn try_from(value: Value) -> Result { + match value { + Value::Null => Err("Null type not supported".to_string()), + + Value::Int8(v) => Ok(ValueData::I32Value(v as i32)), + Value::Int16(v) => Ok(ValueData::I32Value(v as i32)), + Value::Int32(v) => Ok(ValueData::I32Value(v)), + Value::Int64(v) => Ok(ValueData::I64Value(v)), + + Value::Uint8(v) => Ok(ValueData::U32Value(v as u32)), + Value::Uint16(v) => Ok(ValueData::U32Value(v as u32)), + Value::Uint32(v) => Ok(ValueData::U32Value(v)), + Value::Uint64(v) => Ok(ValueData::U64Value(v)), + + Value::Float32(v) => Ok(ValueData::F32Value(v)), + Value::Float64(v) => Ok(ValueData::F64Value(v)), + + Value::Boolean(v) => Ok(ValueData::BoolValue(v)), + Value::String(v) => Ok(ValueData::StringValue(v.clone())), + + Value::Time(Time { nanosecond, .. }) => Ok(ValueData::TimeNanosecondValue(nanosecond)), + + Value::Epoch(Epoch::Nanosecond(ns)) => Ok(ValueData::TimestampNanosecondValue(ns)), + Value::Epoch(Epoch::Microsecond(us)) => Ok(ValueData::TimestampMicrosecondValue(us)), + Value::Epoch(Epoch::Millisecond(ms)) => Ok(ValueData::TimestampMillisecondValue(ms)), + Value::Epoch(Epoch::Second(s)) => Ok(ValueData::TimestampSecondValue(s)), + + Value::Array(_) => unimplemented!("Array type not supported"), + Value::Map(_) => unimplemented!("Object type not supported"), + } + } +} + +// TODO(yuanbohan): add fulltext support in datatype_extension +pub(crate) fn coerce_columns(transform: &Transform) -> Result, String> { + let mut columns = Vec::new(); + + for field in transform.fields.iter() { + let column_name = field.get_target_field().to_string(); + + let datatype = coerce_type(transform)? as i32; + + let semantic_type = coerce_semantic_type(transform) as i32; + + let column = ColumnSchema { + column_name, + datatype, + semantic_type, + datatype_extension: None, + }; + columns.push(column); + } + + Ok(columns) +} + +fn coerce_semantic_type(transform: &Transform) -> SemanticType { + match transform.index { + Some(Index::Tag) => SemanticType::Tag, + Some(Index::Timestamp) => SemanticType::Timestamp, + Some(Index::Fulltext) => unimplemented!("Fulltext"), + None => SemanticType::Field, + } +} + +fn coerce_type(transform: &Transform) -> Result { + match transform.type_ { + Value::Int8(_) => Ok(ColumnDataType::Int8), + Value::Int16(_) => Ok(ColumnDataType::Int16), + Value::Int32(_) => Ok(ColumnDataType::Int32), + Value::Int64(_) => Ok(ColumnDataType::Int64), + + Value::Uint8(_) => Ok(ColumnDataType::Uint8), + Value::Uint16(_) => Ok(ColumnDataType::Uint16), + Value::Uint32(_) => Ok(ColumnDataType::Uint32), + Value::Uint64(_) => Ok(ColumnDataType::Uint64), + + Value::Float32(_) => Ok(ColumnDataType::Float32), + Value::Float64(_) => Ok(ColumnDataType::Float64), + + Value::Boolean(_) => Ok(ColumnDataType::Boolean), + Value::String(_) => Ok(ColumnDataType::String), + + Value::Time(_) => Ok(ColumnDataType::TimestampNanosecond), + + Value::Epoch(Epoch::Nanosecond(_)) => Ok(ColumnDataType::TimestampNanosecond), + Value::Epoch(Epoch::Microsecond(_)) => Ok(ColumnDataType::TimestampMicrosecond), + Value::Epoch(Epoch::Millisecond(_)) => Ok(ColumnDataType::TimestampMillisecond), + Value::Epoch(Epoch::Second(_)) => Ok(ColumnDataType::TimestampSecond), + + Value::Array(_) => unimplemented!("Array"), + Value::Map(_) => unimplemented!("Object"), + + Value::Null => Err(format!( + "Null type not supported when to coerce '{}' type", + transform.fields + )), + } +} + +pub(crate) fn coerce_value( + val: &Value, + transform: &Transform, +) -> Result, String> { + match val { + Value::Null => Ok(None), + + Value::Int8(n) => coerce_i64_value(*n as i64, transform), + Value::Int16(n) => coerce_i64_value(*n as i64, transform), + Value::Int32(n) => coerce_i64_value(*n as i64, transform), + Value::Int64(n) => coerce_i64_value(*n, transform), + + Value::Uint8(n) => coerce_u64_value(*n as u64, transform), + Value::Uint16(n) => coerce_u64_value(*n as u64, transform), + Value::Uint32(n) => coerce_u64_value(*n as u64, transform), + Value::Uint64(n) => coerce_u64_value(*n, transform), + + Value::Float32(n) => coerce_f64_value(*n as f64, transform), + Value::Float64(n) => coerce_f64_value(*n, transform), + + Value::Boolean(b) => coerce_bool_value(*b, transform), + Value::String(s) => coerce_string_value(s, transform), + + Value::Time(Time { nanosecond, .. }) => { + Ok(Some(ValueData::TimestampNanosecondValue(*nanosecond))) + } + + Value::Epoch(Epoch::Nanosecond(ns)) => Ok(Some(ValueData::TimestampNanosecondValue(*ns))), + Value::Epoch(Epoch::Microsecond(us)) => Ok(Some(ValueData::TimestampMicrosecondValue(*us))), + Value::Epoch(Epoch::Millisecond(ms)) => Ok(Some(ValueData::TimestampMillisecondValue(*ms))), + Value::Epoch(Epoch::Second(s)) => Ok(Some(ValueData::TimestampSecondValue(*s))), + + Value::Array(_) => unimplemented!("Array type not supported"), + Value::Map(_) => unimplemented!("Object type not supported"), + } +} + +fn coerce_bool_value(b: bool, transform: &Transform) -> Result, String> { + let val = match transform.type_ { + Value::Int8(_) => ValueData::I8Value(b as i32), + Value::Int16(_) => ValueData::I16Value(b as i32), + Value::Int32(_) => ValueData::I32Value(b as i32), + Value::Int64(_) => ValueData::I64Value(b as i64), + + Value::Uint8(_) => ValueData::U8Value(b as u32), + Value::Uint16(_) => ValueData::U16Value(b as u32), + Value::Uint32(_) => ValueData::U32Value(b as u32), + Value::Uint64(_) => ValueData::U64Value(b as u64), + + Value::Float32(_) => ValueData::F32Value(if b { 1.0 } else { 0.0 }), + Value::Float64(_) => ValueData::F64Value(if b { 1.0 } else { 0.0 }), + + Value::Boolean(_) => ValueData::BoolValue(b), + Value::String(_) => ValueData::StringValue(b.to_string()), + + Value::Time(_) => return Err("Boolean type not supported for Time".to_string()), + Value::Epoch(_) => return Err("Boolean type not supported for Epoch".to_string()), + + Value::Array(_) => unimplemented!("Array type not supported"), + Value::Map(_) => unimplemented!("Object type not supported"), + + Value::Null => return Ok(None), + }; + + Ok(Some(val)) +} + +fn coerce_i64_value(n: i64, transform: &Transform) -> Result, String> { + let val = match transform.type_ { + Value::Int8(_) => ValueData::I8Value(n as i32), + Value::Int16(_) => ValueData::I16Value(n as i32), + Value::Int32(_) => ValueData::I32Value(n as i32), + Value::Int64(_) => ValueData::I64Value(n), + + Value::Uint8(_) => ValueData::U8Value(n as u32), + Value::Uint16(_) => ValueData::U16Value(n as u32), + Value::Uint32(_) => ValueData::U32Value(n as u32), + Value::Uint64(_) => ValueData::U64Value(n as u64), + + Value::Float32(_) => ValueData::F32Value(n as f32), + Value::Float64(_) => ValueData::F64Value(n as f64), + + Value::Boolean(_) => ValueData::BoolValue(n != 0), + Value::String(_) => ValueData::StringValue(n.to_string()), + + Value::Time(_) => return Err("Integer type not supported for Time".to_string()), + Value::Epoch(_) => return Err("Integer type not supported for Epoch".to_string()), + + Value::Array(_) => unimplemented!("Array type not supported"), + Value::Map(_) => unimplemented!("Object type not supported"), + + Value::Null => return Ok(None), + }; + + Ok(Some(val)) +} + +fn coerce_u64_value(n: u64, transform: &Transform) -> Result, String> { + let val = match transform.type_ { + Value::Int8(_) => ValueData::I8Value(n as i32), + Value::Int16(_) => ValueData::I16Value(n as i32), + Value::Int32(_) => ValueData::I32Value(n as i32), + Value::Int64(_) => ValueData::I64Value(n as i64), + + Value::Uint8(_) => ValueData::U8Value(n as u32), + Value::Uint16(_) => ValueData::U16Value(n as u32), + Value::Uint32(_) => ValueData::U32Value(n as u32), + Value::Uint64(_) => ValueData::U64Value(n), + + Value::Float32(_) => ValueData::F32Value(n as f32), + Value::Float64(_) => ValueData::F64Value(n as f64), + + Value::Boolean(_) => ValueData::BoolValue(n != 0), + Value::String(_) => ValueData::StringValue(n.to_string()), + + Value::Time(_) => return Err("Integer type not supported for Time".to_string()), + Value::Epoch(_) => return Err("Integer type not supported for Epoch".to_string()), + + Value::Array(_) => unimplemented!("Array type not supported"), + Value::Map(_) => unimplemented!("Object type not supported"), + + Value::Null => return Ok(None), + }; + + Ok(Some(val)) +} + +fn coerce_f64_value(n: f64, transform: &Transform) -> Result, String> { + let val = match transform.type_ { + Value::Int8(_) => ValueData::I8Value(n as i32), + Value::Int16(_) => ValueData::I16Value(n as i32), + Value::Int32(_) => ValueData::I32Value(n as i32), + Value::Int64(_) => ValueData::I64Value(n as i64), + + Value::Uint8(_) => ValueData::U8Value(n as u32), + Value::Uint16(_) => ValueData::U16Value(n as u32), + Value::Uint32(_) => ValueData::U32Value(n as u32), + Value::Uint64(_) => ValueData::U64Value(n as u64), + + Value::Float32(_) => ValueData::F32Value(n as f32), + Value::Float64(_) => ValueData::F64Value(n), + + Value::Boolean(_) => ValueData::BoolValue(n != 0.0), + Value::String(_) => ValueData::StringValue(n.to_string()), + + Value::Time(_) => return Err("Float type not supported for Time".to_string()), + Value::Epoch(_) => return Err("Float type not supported for Epoch".to_string()), + + Value::Array(_) => unimplemented!("Array type not supported"), + Value::Map(_) => unimplemented!("Object type not supported"), + + Value::Null => return Ok(None), + }; + + Ok(Some(val)) +} + +fn coerce_string_value(s: &str, transform: &Transform) -> Result, String> { + let val = match transform.type_ { + Value::Int8(_) => ValueData::I8Value(s.parse::().map_err(|e| e.to_string())?), + Value::Int16(_) => ValueData::I16Value(s.parse::().map_err(|e| e.to_string())?), + Value::Int32(_) => ValueData::I32Value(s.parse::().map_err(|e| e.to_string())?), + Value::Int64(_) => ValueData::I64Value(s.parse::().map_err(|e| e.to_string())?), + + Value::Uint8(_) => ValueData::U8Value(s.parse::().map_err(|e| e.to_string())?), + Value::Uint16(_) => ValueData::U16Value(s.parse::().map_err(|e| e.to_string())?), + Value::Uint32(_) => ValueData::U32Value(s.parse::().map_err(|e| e.to_string())?), + Value::Uint64(_) => ValueData::U64Value(s.parse::().map_err(|e| e.to_string())?), + + Value::Float32(_) => ValueData::F32Value(s.parse::().map_err(|e| e.to_string())?), + Value::Float64(_) => ValueData::F64Value(s.parse::().map_err(|e| e.to_string())?), + + Value::Boolean(_) => ValueData::BoolValue(s.parse::().map_err(|e| e.to_string())?), + Value::String(_) => ValueData::StringValue(s.to_string()), + + Value::Time(_) => return Err("String type not supported for Time".to_string()), + Value::Epoch(_) => return Err("String type not supported for Epoch".to_string()), + + Value::Array(_) => unimplemented!("Array type not supported"), + Value::Map(_) => unimplemented!("Object type not supported"), + + Value::Null => return Ok(None), + }; + + Ok(Some(val)) +} diff --git a/src/pipeline/src/etl/transform/transformer/greptime/mod.rs b/src/pipeline/src/etl/transform/transformer/greptime/mod.rs new file mode 100644 index 000000000000..bbbfa0e9104b --- /dev/null +++ b/src/pipeline/src/etl/transform/transformer/greptime/mod.rs @@ -0,0 +1,172 @@ +// Copyright 2023 Greptime Team +// +// Licensed 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. + +pub mod coerce; + +use std::collections::HashSet; + +use coerce::{coerce_columns, coerce_value}; +use greptime_proto::v1::{ColumnSchema, Row, Rows, Value as GreptimeValue}; +use itertools::Itertools; + +use crate::etl::field::{Field, Fields}; +use crate::etl::transform::index::Index; +use crate::etl::transform::{Transform, Transformer, Transforms}; +use crate::etl::value::{Array, Epoch, Map, Value}; + +const DEFAULT_GREPTIME_TIMESTAMP_COLUMN: &str = "greptime_timestamp"; + +/// fields not in the columns will be discarded +/// to prevent automatic column creation in GreptimeDB +#[derive(Debug, Clone)] +pub struct GreptimeTransformer { + transforms: Transforms, +} + +impl GreptimeTransformer { + fn default_greptime_timestamp_column() -> Transform { + let ns = chrono::Utc::now().timestamp_nanos_opt().unwrap_or(0); + let type_ = Value::Epoch(Epoch::Nanosecond(ns)); + let default = Some(type_.clone()); + let field = Field::new(DEFAULT_GREPTIME_TIMESTAMP_COLUMN); + let fields = Fields::new(vec![field]).unwrap(); + + Transform { + fields, + type_, + default, + index: Some(Index::Timestamp), + } + } + + fn schemas(&self) -> Result, String> { + let mut schema = vec![]; + for transform in self.transforms.iter() { + schema.extend(coerce_columns(transform)?); + } + Ok(schema) + } + + fn transform_map(&self, map: &Map) -> Result { + let mut values = vec![]; + + for transform in self.transforms.iter() { + for field in transform.fields.iter() { + let value_data = match map.get(field.get_field()) { + Some(val) => coerce_value(val, transform)?, + None if transform.get_default().is_some() => { + coerce_value(transform.get_default().unwrap(), transform)? + } + None => None, + }; + values.push(GreptimeValue { value_data }); + } + } + + Ok(Row { values }) + } + + fn transform_array(&self, arr: &Array) -> Result, String> { + let mut rows = vec![]; + for v in arr.iter() { + match v { + Value::Map(map) => { + let row = self.transform_map(map)?; + rows.push(row); + } + _ => return Err(format!("Expected map, found: {v:?}")), + } + } + Ok(rows) + } +} + +impl std::fmt::Display for GreptimeTransformer { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + writeln!(f, "GreptimeTransformer.\nColumns: {}", self.transforms) + } +} + +impl Transformer for GreptimeTransformer { + type Output = Rows; + + fn new(mut transforms: Transforms) -> Result { + if transforms.is_empty() { + return Err("transform cannot be empty".to_string()); + } + + let mut column_names_set = HashSet::new(); + let mut timestamp_columns = vec![]; + + for transform in transforms.iter() { + let target_fields_set = transform + .fields + .iter() + .map(|f| f.get_target_field()) + .collect::>(); + + let intersections: Vec<_> = column_names_set.intersection(&target_fields_set).collect(); + if !intersections.is_empty() { + let duplicates = intersections.iter().join(","); + return Err(format!( + "column name must be unique, but got duplicated: {duplicates}" + )); + } + + column_names_set.extend(target_fields_set); + + if let Some(idx) = transform.index { + if idx == Index::Timestamp { + match transform.fields.len() { + 1 => timestamp_columns.push(transform.fields.first().unwrap().get_field()), + _ => return Err(format!( + "Illegal to set multiple timestamp Index columns, please set only one: {}", + transform.fields.get_target_fields().join(", ") + )), + } + } + } + } + + match timestamp_columns.len() { + 0 => { + transforms.push(GreptimeTransformer::default_greptime_timestamp_column()); + Ok(GreptimeTransformer { transforms }) + } + 1 => Ok(GreptimeTransformer { transforms }), + _ => { + let columns: String = timestamp_columns.iter().map(|s| s.to_string()).join(", "); + let count = timestamp_columns.len(); + Err( + format!("transform must have exactly one field specified as timestamp Index, but got {count}: {columns}") + ) + } + } + } + + fn transform(&self, value: Value) -> Result { + let schema = self.schemas()?; + match value { + Value::Map(map) => { + let rows = vec![self.transform_map(&map)?]; + Ok(Rows { schema, rows }) + } + Value::Array(arr) => { + let rows = self.transform_array(&arr)?; + Ok(Rows { schema, rows }) + } + _ => Err(format!("Expected map or array, found: {}", value)), + } + } +} diff --git a/src/pipeline/src/etl/transform/transformer/mod.rs b/src/pipeline/src/etl/transform/transformer/mod.rs new file mode 100644 index 000000000000..87bd16b4feca --- /dev/null +++ b/src/pipeline/src/etl/transform/transformer/mod.rs @@ -0,0 +1,16 @@ +// Copyright 2023 Greptime Team +// +// Licensed 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. + +pub mod greptime; +pub mod noop; diff --git a/src/pipeline/src/etl/transform/transformer/noop.rs b/src/pipeline/src/etl/transform/transformer/noop.rs new file mode 100644 index 000000000000..40b82d5ee760 --- /dev/null +++ b/src/pipeline/src/etl/transform/transformer/noop.rs @@ -0,0 +1,36 @@ +// Copyright 2023 Greptime Team +// +// Licensed 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 crate::etl::transform::{Transformer, Transforms}; +use crate::etl::value::Value; + +pub struct NoopTransformer; + +impl std::fmt::Display for NoopTransformer { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + write!(f, "NoopTransformer") + } +} + +impl Transformer for NoopTransformer { + type Output = Value; + + fn new(_transforms: Transforms) -> Result { + Ok(NoopTransformer) + } + + fn transform(&self, val: Value) -> Result { + Ok(val) + } +} diff --git a/src/pipeline/src/etl/value/array.rs b/src/pipeline/src/etl/value/array.rs new file mode 100644 index 000000000000..a401cf00ab67 --- /dev/null +++ b/src/pipeline/src/etl/value/array.rs @@ -0,0 +1,56 @@ +// Copyright 2023 Greptime Team +// +// Licensed 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 crate::etl::value::Value; + +#[derive(Debug, Clone, PartialEq, Default)] +pub struct Array { + pub values: Vec, +} + +impl Array { + pub fn new() -> Self { + Array { values: vec![] } + } +} + +impl std::fmt::Display for Array { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + let values = self + .values + .iter() + .map(|v| v.to_string()) + .collect::>() + .join(", "); + write!(f, "[{}]", values) + } +} + +impl std::ops::Deref for Array { + type Target = Vec; + + fn deref(&self) -> &Self::Target { + &self.values + } +} + +impl IntoIterator for Array { + type Item = Value; + + type IntoIter = std::vec::IntoIter; + + fn into_iter(self) -> Self::IntoIter { + self.values.into_iter() + } +} diff --git a/src/pipeline/src/etl/value/map.rs b/src/pipeline/src/etl/value/map.rs new file mode 100644 index 000000000000..47041f15350d --- /dev/null +++ b/src/pipeline/src/etl/value/map.rs @@ -0,0 +1,64 @@ +// Copyright 2023 Greptime Team +// +// Licensed 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 std::collections::HashMap; + +use crate::etl::value::Value; + +#[derive(Debug, Clone, PartialEq, Default)] +pub struct Map { + pub values: HashMap, +} + +impl Map { + pub fn one(key: impl Into, value: Value) -> Map { + let mut map = Map::default(); + map.insert(key, value); + map + } + + pub fn insert(&mut self, key: impl Into, value: Value) { + self.values.insert(key.into(), value); + } + + pub fn extend(&mut self, Map { values }: Map) { + self.values.extend(values); + } +} + +impl From> for Map { + fn from(values: HashMap) -> Self { + Map { values } + } +} + +impl std::ops::Deref for Map { + type Target = HashMap; + + fn deref(&self) -> &Self::Target { + &self.values + } +} + +impl std::fmt::Display for Map { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + let values = self + .values + .iter() + .map(|(k, v)| format!("{}: {}", k, v)) + .collect::>() + .join(", "); + write!(f, "{{{}}}", values) + } +} diff --git a/src/pipeline/src/etl/value/mod.rs b/src/pipeline/src/etl/value/mod.rs new file mode 100644 index 000000000000..a8daa5fa6149 --- /dev/null +++ b/src/pipeline/src/etl/value/mod.rs @@ -0,0 +1,303 @@ +// Copyright 2023 Greptime Team +// +// Licensed 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. + +pub mod array; +pub mod map; +pub mod time; + +pub use std::collections::HashMap; + +pub use array::Array; +pub use map::Map; +pub use time::{Epoch, Time}; + +/// Value can be used as type +/// acts as value: the enclosed value is the actual value +/// acts as type: the enclosed value is the default value +#[derive(Debug, Clone, PartialEq)] +pub enum Value { + // as value: null + // as type: no type specified + Null, + + Int8(i8), + Int16(i16), + Int32(i32), + Int64(i64), + + Uint8(u8), + Uint16(u16), + Uint32(u32), + Uint64(u64), + + Float32(f32), + Float64(f64), + + Boolean(bool), + String(String), + + Time(Time), + Epoch(Epoch), + + Array(Array), + Map(Map), +} + +impl Value { + pub fn is_null(&self) -> bool { + matches!(self, Value::Null) + } + + pub fn parse_str_type(t: &str) -> Result { + let mut parts = t.splitn(2, ','); + let head = parts.next().unwrap_or_default(); + let tail = parts.next().map(|s| s.trim().to_string()); + match head.to_lowercase().as_str() { + "int8" => Ok(Value::Int8(0)), + "int16" => Ok(Value::Int16(0)), + "int32" => Ok(Value::Int32(0)), + "int64" => Ok(Value::Int64(0)), + + "uint8" => Ok(Value::Uint8(0)), + "uint16" => Ok(Value::Uint16(0)), + "uint32" => Ok(Value::Uint32(0)), + "uint64" => Ok(Value::Uint64(0)), + + "float32" => Ok(Value::Float32(0.0)), + "float64" => Ok(Value::Float64(0.0)), + + "boolean" => Ok(Value::Boolean(false)), + "string" => Ok(Value::String("".to_string())), + + "time" => Ok(Value::Time(Time::default())), + "epoch" => match tail { + Some(resolution) if !resolution.is_empty() => match resolution.as_str() { + time::NANOSECOND_RESOLUTION | time::NANO_RESOLUTION | time::NS_RESOLUTION => { + Ok(Value::Epoch(Epoch::Nanosecond(0))) + } + time::MICROSECOND_RESOLUTION | time::MICRO_RESOLUTION | time::US_RESOLUTION => { + Ok(Value::Epoch(Epoch::Microsecond(0))) + } + time::MILLISECOND_RESOLUTION | time::MILLI_RESOLUTION | time::MS_RESOLUTION => { + Ok(Value::Epoch(Epoch::Millisecond(0))) + } + time::SECOND_RESOLUTION | time::SEC_RESOLUTION | time::S_RESOLUTION => { + Ok(Value::Epoch(Epoch::Second(0))) + } + _ => Err(format!( + "invalid resolution: '{resolution}'. Available resolutions: {}", + time::VALID_RESOLUTIONS.join(",") + )), + }, + _ => Err(format!( + "resolution MUST BE set for epoch type: '{t}'. Available resolutions: {}", + time::VALID_RESOLUTIONS.join(", ") + )), + }, + + "array" => Ok(Value::Array(Array::default())), + "map" => Ok(Value::Map(Map::default())), + + _ => Err(format!("failed to parse type: '{t}'")), + } + } + + /// only support string, bool, number, null + pub fn parse_str_value(&self, v: &str) -> Result { + match self { + Value::Int8(_) => v + .parse::() + .map(Value::Int8) + .map_err(|e| format!("failed to parse int8: {}", e)), + Value::Int16(_) => v + .parse::() + .map(Value::Int16) + .map_err(|e| format!("failed to parse int16: {}", e)), + Value::Int32(_) => v + .parse::() + .map(Value::Int32) + .map_err(|e| format!("failed to parse int32: {}", e)), + Value::Int64(_) => v + .parse::() + .map(Value::Int64) + .map_err(|e| format!("failed to parse int64: {}", e)), + + Value::Uint8(_) => v + .parse::() + .map(Value::Uint8) + .map_err(|e| format!("failed to parse uint8: {}", e)), + Value::Uint16(_) => v + .parse::() + .map(Value::Uint16) + .map_err(|e| format!("failed to parse uint16: {}", e)), + Value::Uint32(_) => v + .parse::() + .map(Value::Uint32) + .map_err(|e| format!("failed to parse uint32: {}", e)), + Value::Uint64(_) => v + .parse::() + .map(Value::Uint64) + .map_err(|e| format!("failed to parse uint64: {}", e)), + + Value::Float32(_) => v + .parse::() + .map(Value::Float32) + .map_err(|e| format!("failed to parse float32: {}", e)), + Value::Float64(_) => v + .parse::() + .map(Value::Float64) + .map_err(|e| format!("failed to parse float64: {}", e)), + + Value::Boolean(_) => v + .parse::() + .map(Value::Boolean) + .map_err(|e| format!("failed to parse bool: {}", e)), + Value::String(_) => Ok(Value::String(v.to_string())), + + Value::Null => Ok(Value::Null), + + _ => Err(format!("default value not unsupported for type {}", self)), + } + } + + /// only support string, bool, number, null + pub fn to_str_value(&self) -> String { + match self { + Value::Int8(v) => format!("{}", v), + Value::Int16(v) => format!("{}", v), + Value::Int32(v) => format!("{}", v), + Value::Int64(v) => format!("{}", v), + + Value::Uint8(v) => format!("{}", v), + Value::Uint16(v) => format!("{}", v), + Value::Uint32(v) => format!("{}", v), + Value::Uint64(v) => format!("{}", v), + + Value::Float32(v) => format!("{}", v), + Value::Float64(v) => format!("{}", v), + + Value::Boolean(v) => format!("{}", v), + Value::String(v) => v.to_string(), + + v => v.to_string(), + } + } +} + +impl std::fmt::Display for Value { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + let str = match self { + Value::Null => "null".to_string(), + + Value::Int8(v) => format!("int8({})", v), + Value::Int16(v) => format!("int16({})", v), + Value::Int32(v) => format!("int32({})", v), + Value::Int64(v) => format!("int64({})", v), + + Value::Uint8(v) => format!("uint8({})", v), + Value::Uint16(v) => format!("uint16({})", v), + Value::Uint32(v) => format!("uint32({})", v), + Value::Uint64(v) => format!("uint64({})", v), + + Value::Float32(v) => format!("float32({})", v), + Value::Float64(v) => format!("float64({})", v), + + Value::Boolean(v) => format!("boolean({})", v), + Value::String(v) => format!("string({})", v), + + Value::Time(v) => format!("time({})", v), + Value::Epoch(v) => format!("epoch({})", v), + + Value::Array(v) => format!("{}", v), + Value::Map(v) => format!("{}", v), + }; + + write!(f, "{}", str) + } +} + +impl TryFrom for Value { + type Error = String; + + fn try_from(v: serde_json::Value) -> Result { + match v { + serde_json::Value::Null => Ok(Value::Null), + serde_json::Value::Bool(v) => Ok(Value::Boolean(v)), + serde_json::Value::Number(v) => { + if let Some(v) = v.as_i64() { + Ok(Value::Int64(v)) + } else if let Some(v) = v.as_u64() { + Ok(Value::Uint64(v)) + } else if let Some(v) = v.as_f64() { + Ok(Value::Float64(v)) + } else { + Err(format!("unsupported number type: {}", v)) + } + } + serde_json::Value::String(v) => Ok(Value::String(v)), + serde_json::Value::Array(v) => { + let mut values = vec![]; + for v in v { + values.push(Value::try_from(v)?); + } + Ok(Value::Array(Array { values })) + } + serde_json::Value::Object(v) => { + let mut values = HashMap::new(); + for (k, v) in v { + values.insert(k, Value::try_from(v)?); + } + Ok(Value::Map(Map { values })) + } + } + } +} + +impl TryFrom<&yaml_rust::Yaml> for Value { + type Error = String; + + fn try_from(v: &yaml_rust::Yaml) -> Result { + match v { + yaml_rust::Yaml::Null => Ok(Value::Null), + yaml_rust::Yaml::Boolean(v) => Ok(Value::Boolean(*v)), + yaml_rust::Yaml::Integer(v) => Ok(Value::Int64(*v)), + yaml_rust::Yaml::Real(v) => { + if let Ok(v) = v.parse() { + Ok(Value::Float64(v)) + } else { + Err(format!("failed to parse float64: {}", v)) + } + } + yaml_rust::Yaml::String(v) => Ok(Value::String(v.to_string())), + yaml_rust::Yaml::Array(arr) => { + let mut values = vec![]; + for v in arr { + values.push(Value::try_from(v)?); + } + Ok(Value::Array(Array { values })) + } + yaml_rust::Yaml::Hash(v) => { + let mut values = HashMap::new(); + for (k, v) in v { + let key = k + .as_str() + .ok_or(format!("key in Hash must be a string, but got {v:?}"))?; + values.insert(key.to_string(), Value::try_from(v)?); + } + Ok(Value::Map(Map { values })) + } + _ => Err(format!("unsupported yaml type: {v:?}")), + } + } +} diff --git a/src/pipeline/src/etl/value/time.rs b/src/pipeline/src/etl/value/time.rs new file mode 100644 index 000000000000..cca883f33231 --- /dev/null +++ b/src/pipeline/src/etl/value/time.rs @@ -0,0 +1,187 @@ +// Copyright 2023 Greptime Team +// +// Licensed 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 common_telemetry::error; + +#[derive(Debug, Clone, PartialEq)] +pub struct Time { + pub value: String, + pub nanosecond: i64, + pub format: Option, + pub timezone: Option, + // TODO(yuanbohan): support locale + // pub locale: Option, +} + +impl Time { + pub(crate) fn new(v: impl Into, nanosecond: i64) -> Self { + let value = v.into(); + Time { + value, + nanosecond, + format: None, + timezone: None, + } + } + + pub(crate) fn with_format(&mut self, format: impl Into) { + self.format = Some(format.into()); + } + + pub(crate) fn with_timezone(&mut self, timezone: Option) { + self.timezone = timezone; + } + + pub(crate) fn timestamp_nanos(&self) -> i64 { + self.nanosecond + } + + pub(crate) fn timestamp_micros(&self) -> i64 { + self.nanosecond / 1_000 + } + + pub(crate) fn timestamp_millis(&self) -> i64 { + self.nanosecond / 1_000_000 + } + + pub(crate) fn timestamp(&self) -> i64 { + self.nanosecond / 1_000_000_000 + } +} + +impl Default for Time { + fn default() -> Self { + let dt = chrono::Utc::now(); + let v = dt.to_rfc3339(); + let ns = match dt.timestamp_nanos_opt() { + Some(ns) => ns, + None => { + error!("failed to get nanosecond from timestamp, use 0 instead"); + 0 + } + }; + Time::new(v, ns) + } +} + +impl std::fmt::Display for Time { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + let format = if let Some(format) = &self.format { + format!(", format: {}", format) + } else { + "".to_string() + }; + + let timezone = if let Some(timezone) = &self.timezone { + format!(", timezone: {}", timezone) + } else { + "".to_string() + }; + + write!(f, "{}, format: {}{}", self.value, format, timezone) + } +} + +#[derive(Debug, Clone, PartialEq)] +pub enum Epoch { + Nanosecond(i64), + Microsecond(i64), + Millisecond(i64), + Second(i64), +} + +pub(crate) const NANOSECOND_RESOLUTION: &str = "nanosecond"; +pub(crate) const NANO_RESOLUTION: &str = "nano"; +pub(crate) const NS_RESOLUTION: &str = "ns"; +pub(crate) const MICROSECOND_RESOLUTION: &str = "microsecond"; +pub(crate) const MICRO_RESOLUTION: &str = "micro"; +pub(crate) const US_RESOLUTION: &str = "us"; +pub(crate) const MILLISECOND_RESOLUTION: &str = "millisecond"; +pub(crate) const MILLI_RESOLUTION: &str = "milli"; +pub(crate) const MS_RESOLUTION: &str = "ms"; +pub(crate) const SECOND_RESOLUTION: &str = "second"; +pub(crate) const SEC_RESOLUTION: &str = "sec"; +pub(crate) const S_RESOLUTION: &str = "s"; + +pub(crate) const VALID_RESOLUTIONS: [&str; 12] = [ + NANOSECOND_RESOLUTION, + NANO_RESOLUTION, + NS_RESOLUTION, + MICROSECOND_RESOLUTION, + MICRO_RESOLUTION, + US_RESOLUTION, + MILLISECOND_RESOLUTION, + MILLI_RESOLUTION, + MS_RESOLUTION, + SECOND_RESOLUTION, + SEC_RESOLUTION, + S_RESOLUTION, +]; + +impl Epoch { + pub(crate) fn timestamp_nanos(&self) -> i64 { + match self { + Epoch::Nanosecond(v) => *v, + Epoch::Microsecond(v) => *v * 1_000, + Epoch::Millisecond(v) => *v * 1_000_000, + Epoch::Second(v) => *v * 1_000_000_000, + } + } + + pub(crate) fn timestamp_micros(&self) -> i64 { + match self { + Epoch::Nanosecond(v) => *v / 1_000, + Epoch::Microsecond(v) => *v, + Epoch::Millisecond(v) => *v * 1_000, + Epoch::Second(v) => *v * 1_000_000, + } + } + + pub(crate) fn timestamp_millis(&self) -> i64 { + match self { + Epoch::Nanosecond(v) => *v / 1_000_000, + Epoch::Microsecond(v) => *v / 1_000, + Epoch::Millisecond(v) => *v, + Epoch::Second(v) => *v * 1_000, + } + } + + pub(crate) fn timestamp(&self) -> i64 { + match self { + Epoch::Nanosecond(v) => *v / 1_000_000_000, + Epoch::Microsecond(v) => *v / 1_000_000, + Epoch::Millisecond(v) => *v / 1_000, + Epoch::Second(v) => *v, + } + } +} + +impl Default for Epoch { + fn default() -> Self { + Epoch::Nanosecond(chrono::Utc::now().timestamp_nanos_opt().unwrap_or(0)) + } +} + +impl std::fmt::Display for Epoch { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + let (value, resolution) = match self { + Epoch::Nanosecond(v) => (v, NANOSECOND_RESOLUTION), + Epoch::Microsecond(v) => (v, MICROSECOND_RESOLUTION), + Epoch::Millisecond(v) => (v, MILLISECOND_RESOLUTION), + Epoch::Second(v) => (v, SECOND_RESOLUTION), + }; + + write!(f, "{}, resolution: {}", value, resolution) + } +} diff --git a/src/pipeline/tests/pipeline.rs b/src/pipeline/tests/pipeline.rs new file mode 100644 index 000000000000..869bd13c78f3 --- /dev/null +++ b/src/pipeline/tests/pipeline.rs @@ -0,0 +1,461 @@ +// Copyright 2023 Greptime Team +// +// Licensed 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 common_telemetry::tracing::info; +use greptime_proto::v1::value::ValueData::{ + BoolValue, F64Value, StringValue, TimestampSecondValue, U32Value, U64Value, U8Value, +}; +use greptime_proto::v1::Value as GreptimeValue; +use pipeline::{parse, Content, GreptimeTransformer, Pipeline, Value}; + +// use pipeline::transform::GreptimeTransformer; +// use pipeline::value::Value; +// use pipeline::{parse, Content, Pipeline}; + +#[test] +fn main() { + let input_value_str = r#" + [ + { + "version": 1, + "streamId": "12345", + "cp": "123456", + "reqId": "1239f220", + "reqTimeSec": "1573840000", + "bytes": "4995", + "cliIP": "128.147.28.68", + "statusCode": "206", + "proto": "HTTPS", + "reqHost": "test.hostname.net", + "reqMethod": "GET", + "reqPath": "/path1/path2/file.ext", + "reqPort": "443", + "rspContentLen": "5000", + "rspContentType": "text/html", + "UA": "Mozilla%2F5.0+%28Macintosh%3B+Intel+Mac+OS+X+10_14_3%29", + "tlsOverheadTimeMSec": "0", + "tlsVersion": "TLSv1", + "objSize": "484", + "uncompressedSize": "484", + "overheadBytes": "232", + "totalBytes": "0", + "queryStr": "cmcd=//1.0@V/bl=21600,br=1426,cid=%22akam-email%22,d=6006,mtp=11100,ot=m,sf=h,sid=%229f36f5c9-d6a2-497b-8c73-4b8f694eab749f36f5c9-d6a2-497b-8c73%22,tb=1426,dl=18500,nor=%22../300kbps/track.m4v%22,nrr=%2212323-48763%22,su,bs,rtp=12000,pr=1.08,sf=d,st=v%22", + "breadcrumbs": "//BC/%5Ba=23.33.41.20,c=g,k=0,l=1%5D", + "accLang": "en-US", + "cookie": "cookie-content", + "range": "37334-42356", + "referer": "https%3A%2F%2Ftest.referrer.net%2Fen-US%2Fdocs%2FWeb%2Ftest", + "xForwardedFor": "8.47.28.38", + "maxAgeSec": "3600", + "reqEndTimeMSec": "3", + "errorCode": "ERR_ACCESS_DENIED|fwd_acl", + "turnAroundTimeMSec": "11", + "transferTimeMSec": "125", + "dnsLookupTimeMSec": "50", + "lastByte": "1", + "edgeIP": "23.50.51.173", + "country": "IN", + "state": "Virginia", + "city": "HERNDON", + "serverCountry": "SG", + "billingRegion": "8", + "cacheStatus": "1", + "securityRules": "ULnR_28976|3900000:3900001:3900005:3900006:BOT-ANOMALY-HEADER|", + "ewUsageInfo": "//4380/4.0/1/-/0/4/#1,2\\//4380/4.0/4/-/0/4/#0,0\\//4380/4.0/5/-/1/1/#0,0", + "ewExecutionInfo": "c:4380:7:161:162:161:n:::12473:200|C:4380:3:0:4:0:n:::6967:200|R:4380:20:99:99:1:n:::35982:200", + "customField": "any-custom-value" + } + ] +"#; + let input_value: Value = serde_json::from_str::(input_value_str) + .expect("failed to parse input value") + .try_into() + .expect("failed to convert input value"); + + let pipeline_yaml = r#" +--- +description: Pipeline for Akamai DataStream2 Log + +processors: + - urlencoding: + fields: + - breadcrumbs + - UA + - referer + - queryStr + method: decode + ignore_missing: true + - epoch: + field: reqTimeSec + resolution: second + ignore_missing: true + - regex: + field: breadcrumbs + patterns: + - "(?\\[[^\\[]*c=c[^\\]]*\\])" + - "(?\\[[^\\[]*c=g[^\\]]*\\])" + - "(?\\[[^\\[]*c=o[^\\]]*\\])" + - "(?\\[[^\\[]*c=p[^\\]]*\\])" + - "(?\\[[^\\[]*c=w[^\\]]*\\])" + ignore_missing: true + - regex: + fields: + - breadcrumbs_parent + - breadcrumbs_edge + - breadcrumbs_origin + - breadcrumbs_peer + - breadcrumbs_cloud_wrapper + ignore_missing: true + patterns: + - "a=(?[^,\\]]+)" + - "b=(?[^,\\]]+)" + - "k=(?[^,\\]]+)" + - "l=(?[^,\\]]+)" + - "m=(?[^,\\]]+)" + - "n=(?[^,\\]]+)" + - "o=(?[^,\\]]+)" + - regex: + field: queryStr, cmcd + patterns: + - "(?i)CMCD=//(?[\\d\\.]+)@V/(?.+$)" + ignore_missing: true + - cmcd: + field: cmcd_data, cmcd + ignore_missing: true + +transform: + - fields: + - breadcrumbs + - referer + - queryStr, query_str + - customField, custom_field + - reqId, req_id + - city + - state + - country + - securityRules, security_rules + - ewUsageInfo, ew_usage_info + - ewExecutionInfo, ew_execution_info + - errorCode, error_code + - xForwardedFor, x_forwarded_for + - range + - accLang, acc_lang + - reqMethod, req_method + - reqHost, req_host + - proto + - cliIP, cli_ip + - rspContentType, rsp_content_type + - tlsVersion, tls_version + type: string + - fields: + - version + - cacheStatus, cache_status + - lastByte, last_byte + type: uint8 + - fields: + - streamId, stream_id + - billingRegion, billing_region + - dnsLookupTimeMSec, dns_lookup_time_msec + - transferTimeMSec, transfer_time_msec + - turnAroundTimeMSec, turn_around_time_msec + - reqEndTimeMSec, req_end_time_msec + - maxAgeSec, max_age_sec + - reqPort, req_port + - statusCode, status_code + - cp + - tlsOverheadTimeMSec, tls_overhead_time_msec + type: uint32 + - fields: + - bytes + - rspContentLen, rsp_content_len + - objSize, obj_size + - uncompressedSize, uncompressed_size + - overheadBytes, overhead_bytes + - totalBytes, total_bytes + type: uint64 + - fields: + - UA, user_agent + - cookie + - reqPath, req_path + type: string + # index: fulltext + - field: reqTimeSec, req_time_sec + # epoch time is special, the resolution MUST BE specified + type: epoch, second + index: timestamp + + # the following is from cmcd + - fields: + - cmcd_version + - cmcd_cid, cmcd_content_id + - cmcd_nor, cmcd_next_object_requests + - cmcd_nrr, cmcd_next_range_request + - cmcd_ot, cmcd_object_type + - cmcd_sf, cmcd_streaming_format + - cmcd_sid, cmcd_session_id + - cmcd_st, cmcd_stream_type + - cmcd_v + type: string + - fields: + - cmcd_br, cmcd_encoded_bitrate + - cmcd_bl, cmcd_buffer_length + - cmcd_d, cmcd_object_duration + - cmcd_dl, cmcd_deadline + - cmcd_mtp, cmcd_measured_throughput + - cmcd_rtp, cmcd_requested_max_throughput + - cmcd_tb, cmcd_top_bitrate + type: uint64 + - fields: + - cmcd_pr, cmcd_playback_rate + type: float64 + - fields: + - cmcd_bs, cmcd_buffer_starvation + - cmcd_su, cmcd_startup + type: boolean + + # the following is from breadcrumbs + - fields: + - breadcrumbs_parent_ip + - breadcrumbs_parent_request_id + - breadcrumbs_parent_geo + - breadcrumbs_edge_ip + - breadcrumbs_edge_request_id + - breadcrumbs_edge_geo + - breadcrumbs_origin_ip + - breadcrumbs_origin_request_id + - breadcrumbs_origin_geo + - breadcrumbs_peer_ip + - breadcrumbs_peer_request_id + - breadcrumbs_peer_geo + - breadcrumbs_cloud_wrapper_ip + - breadcrumbs_cloud_wrapper_request_id + - breadcrumbs_cloud_wrapper_geo + type: string + - fields: + - breadcrumbs_parent_request_end_time + - breadcrumbs_parent_turn_around_time + - breadcrumbs_parent_dns_lookup_time + - breadcrumbs_parent_asn + - breadcrumbs_edge_request_end_time + - breadcrumbs_edge_turn_around_time + - breadcrumbs_edge_dns_lookup_time + - breadcrumbs_edge_asn + - breadcrumbs_origin_request_end_time + - breadcrumbs_origin_turn_around_time + - breadcrumbs_origin_dns_lookup_time + - breadcrumbs_origin_asn + - breadcrumbs_peer_request_end_time + - breadcrumbs_peer_turn_around_time + - breadcrumbs_peer_dns_lookup_time + - breadcrumbs_peer_asn + - breadcrumbs_cloud_wrapper_request_end_time + - breadcrumbs_cloud_wrapper_turn_around_time + - breadcrumbs_cloud_wrapper_dns_lookup_time + - breadcrumbs_cloud_wrapper_asn + type: uint32 +"#; + + let expected_values = vec![ + ( + "breadcrumbs", + Some(StringValue("//BC/[a=23.33.41.20,c=g,k=0,l=1]".into())), + ), + ( + "referer", + Some(StringValue( + "https://test.referrer.net/en-US/docs/Web/test".into(), + )), + ), + ( + "query_str", + Some(StringValue("cmcd=//1.0@V/bl=21600,br=1426,cid=\"akam-email\",d=6006,mtp=11100,ot=m,sf=h,sid=\"9f36f5c9-d6a2-497b-8c73-4b8f694eab749f36f5c9-d6a2-497b-8c73\",tb=1426,dl=18500,nor=\"../300kbps/track.m4v\",nrr=\"12323-48763\",su,bs,rtp=12000,pr=1.08,sf=d,st=v\"".into())), + ), + ("custom_field", Some(StringValue("any-custom-value".into()))), + ("req_id", Some(StringValue("1239f220".into()))), + ("city", Some(StringValue("HERNDON".into()))), + ("state", Some(StringValue("Virginia".into()))), + ("country", Some(StringValue("IN".into()))), + ( + "security_rules", + Some(StringValue( + "ULnR_28976|3900000:3900001:3900005:3900006:BOT-ANOMALY-HEADER|".into(), + )), + ), + ( + "ew_usage_info", + Some(StringValue( + "//4380/4.0/1/-/0/4/#1,2\\//4380/4.0/4/-/0/4/#0,0\\//4380/4.0/5/-/1/1/#0,0".into(), + )), + ), + ( + "ew_execution_info", + Some(StringValue("c:4380:7:161:162:161:n:::12473:200|C:4380:3:0:4:0:n:::6967:200|R:4380:20:99:99:1:n:::35982:200".into()))), + ( + "error_code", + Some(StringValue("ERR_ACCESS_DENIED|fwd_acl".into())), + ), + ("x_forwarded_for", Some(StringValue("8.47.28.38".into()))), + ("range", Some(StringValue("37334-42356".into()))), + ("acc_lang", Some(StringValue("en-US".into()))), + ("req_method", Some(StringValue("GET".into()))), + ("req_host", Some(StringValue("test.hostname.net".into()))), + ("proto", Some(StringValue("HTTPS".into()))), + ("cli_ip", Some(StringValue("128.147.28.68".into()))), + ("rsp_content_type", Some(StringValue("text/html".into()))), + ("tls_version", Some(StringValue("TLSv1".into()))), + ("version", Some(U8Value(1))), + ("cache_status", Some(U8Value(1))), + ("last_byte", Some(U8Value(1))), + ("stream_id", Some(U32Value(12345))), + ("billing_region", Some(U32Value(8))), + ("dns_lookup_time_msec", Some(U32Value(50))), + ("transfer_time_msec", Some(U32Value(125))), + ("turn_around_time_msec", Some(U32Value(11))), + ("req_end_time_msec", Some(U32Value(3))), + ("max_age_sec", Some(U32Value(3600))), + ("req_port", Some(U32Value(443))), + ("status_code", Some(U32Value(206))), + ("cp", Some(U32Value(123456))), + ("tls_overhead_time_msec", Some(U32Value(0))), + ("bytes", Some(U64Value(4995))), + ("rsp_content_len", Some(U64Value(5000))), + ("obj_size", Some(U64Value(484))), + ("uncompressed_size", Some(U64Value(484))), + ("overhead_bytes", Some(U64Value(232))), + ("total_bytes", Some(U64Value(0))), + ( + "user_agent", + Some(StringValue( + "Mozilla/5.0+(Macintosh;+Intel+Mac+OS+X+10_14_3)".into(), + )), + ), + ("cookie", Some(StringValue("cookie-content".into()))), + ( + "req_path", + Some(StringValue("/path1/path2/file.ext".into())), + ), + ("req_time_sec", Some(TimestampSecondValue(1573840000))), + ("cmcd_version", Some(StringValue("1.0".into()))), + ( + "cmcd_content_id", + Some(StringValue("\"akam-email\"".into())), + ), + ( + "cmcd_next_object_requests", + Some(StringValue("\"../300kbps/track.m4v\"".into())), + ), + ( + "cmcd_next_range_request", + Some(StringValue("\"12323-48763\"".into())), + ), + ("cmcd_object_type", Some(StringValue("m".into()))), + ("cmcd_streaming_format", Some(StringValue("d".into()))), + ( + "cmcd_session_id", + Some(StringValue( + "\"9f36f5c9-d6a2-497b-8c73-4b8f694eab749f36f5c9-d6a2-497b-8c73\"".into(), + )), + ), + ("cmcd_stream_type", Some(StringValue("v\"".into()))), + ("cmcd_v", None), + ("cmcd_encoded_bitrate", Some(U64Value(1426))), + ("cmcd_buffer_length", Some(U64Value(21600))), + ("cmcd_object_duration", Some(U64Value(6006))), + ("cmcd_deadline", Some(U64Value(18500))), + ("cmcd_measured_throughput", Some(U64Value(11100))), + ("cmcd_requested_max_throughput", Some(U64Value(12000))), + ("cmcd_top_bitrate", Some(U64Value(1426))), + ("cmcd_playback_rate", Some(F64Value(1.08))), + ("cmcd_buffer_starvation", Some(BoolValue(true))), + ("cmcd_startup", Some(BoolValue(true))), + ("breadcrumbs_parent_ip", None), + ("breadcrumbs_parent_request_id", None), + ("breadcrumbs_parent_geo", None), + ( + "breadcrumbs_edge_ip", + Some(StringValue("23.33.41.20".into())), + ), + ("breadcrumbs_edge_request_id", None), + ("breadcrumbs_edge_geo", None), + ("breadcrumbs_origin_ip", None), + ("breadcrumbs_origin_request_id", None), + ("breadcrumbs_origin_geo", None), + ("breadcrumbs_peer_ip", None), + ("breadcrumbs_peer_request_id", None), + ("breadcrumbs_peer_geo", None), + ("breadcrumbs_cloud_wrapper_ip", None), + ("breadcrumbs_cloud_wrapper_request_id", None), + ("breadcrumbs_cloud_wrapper_geo", None), + ("breadcrumbs_parent_request_end_time", None), + ("breadcrumbs_parent_turn_around_time", None), + ("breadcrumbs_parent_dns_lookup_time", None), + ("breadcrumbs_parent_asn", None), + ("breadcrumbs_edge_request_end_time", Some(U32Value(0))), + ("breadcrumbs_edge_turn_around_time", Some(U32Value(1))), + ("breadcrumbs_edge_dns_lookup_time", None), + ("breadcrumbs_edge_asn", None), + ("breadcrumbs_origin_request_end_time", None), + ("breadcrumbs_origin_turn_around_time", None), + ("breadcrumbs_origin_dns_lookup_time", None), + ("breadcrumbs_origin_asn", None), + ("breadcrumbs_peer_request_end_time", None), + ("breadcrumbs_peer_turn_around_time", None), + ("breadcrumbs_peer_dns_lookup_time", None), + ("breadcrumbs_peer_asn", None), + ("breadcrumbs_cloud_wrapper_request_end_time", None), + ("breadcrumbs_cloud_wrapper_turn_around_time", None), + ("breadcrumbs_cloud_wrapper_dns_lookup_time", None), + ("breadcrumbs_cloud_wrapper_asn", None), + ] + .into_iter() + .map(|(_, d)| GreptimeValue { value_data: d }) + .collect::>(); + + let yaml_content = Content::Yaml(pipeline_yaml.into()); + let pipeline: Pipeline = + parse(&yaml_content).expect("failed to parse pipeline"); + let output = pipeline.exec(input_value).expect("failed to exec pipeline"); + + assert_eq!(output.rows.len(), 1); + let values = output.rows.first().unwrap().values.clone(); + assert_eq!(expected_values, values); + + for s in output.schema.iter() { + info!( + "{}({}): {}", + s.column_name, + s.datatype().as_str_name(), + s.semantic_type().as_str_name() + ); + } + info!("\n"); + + let get_schema_name = |ss: &Vec, i: usize| { + let s = ss.get(i).unwrap(); + s.column_name.clone() + }; + + for row in output.rows.iter() { + let values = &row.values; + for i in 0..values.len() { + let val = values.get(i).unwrap(); + info!( + "{}: {:?}, ", + get_schema_name(&output.schema, i), + val.value_data + ); + } + info!("\n"); + } +} From 77ef015c022b3676a8b243add61a6154f54220d6 Mon Sep 17 00:00:00 2001 From: paomian Date: Thu, 6 Jun 2024 14:17:14 +0800 Subject: [PATCH 36/51] chore: update upload pipeline api --- Cargo.lock | 28 ++++++++++++ src/servers/Cargo.toml | 2 +- src/servers/src/http.rs | 2 +- src/servers/src/http/event.rs | 80 +++++++++++++++++++++++++++++------ 4 files changed, 98 insertions(+), 14 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 15eefa5b40c8..246a09fb8366 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -771,6 +771,7 @@ dependencies = [ "matchit", "memchr", "mime", + "multer", "percent-encoding", "pin-project-lite", "rustversion", @@ -3562,6 +3563,15 @@ version = "0.3.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a357d28ed41a50f9c765dbfe56cbc04a64e53e5fc58ba79fbc34c10ef3df831f" +[[package]] +name = "encoding_rs" +version = "0.8.34" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b45de904aa0b010bce2ab45264d0631681847fa7b6f2eaa7dab7619943bc4f59" +dependencies = [ + "cfg-if", +] + [[package]] name = "endian-type" version = "0.1.2" @@ -5962,6 +5972,24 @@ dependencies = [ "rand_core", ] +[[package]] +name = "multer" +version = "2.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "01acbdc23469fd8fe07ab135923371d5f5a422fbf9c522158677c8eb15bc51c2" +dependencies = [ + "bytes", + "encoding_rs", + "futures-util", + "http 0.2.12", + "httparse", + "log", + "memchr", + "mime", + "spin 0.9.8", + "version_check", +] + [[package]] name = "multimap" version = "0.8.3" diff --git a/src/servers/Cargo.toml b/src/servers/Cargo.toml index 31321a84010c..e02c8cd6dc7e 100644 --- a/src/servers/Cargo.toml +++ b/src/servers/Cargo.toml @@ -22,7 +22,7 @@ arrow-ipc.workspace = true arrow-schema.workspace = true async-trait = "0.1" auth.workspace = true -axum.workspace = true +axum = {workspace=true,features = ["multipart"]} axum-macros = "0.3.8" base64.workspace = true bytes.workspace = true diff --git a/src/servers/src/http.rs b/src/servers/src/http.rs index d9d8b6ec2731..efce02fec638 100644 --- a/src/servers/src/http.rs +++ b/src/servers/src/http.rs @@ -713,7 +713,7 @@ impl HttpServer { fn route_log(log_handler: LogHandlerRef) -> Router { Router::new() .route("/logs", routing::post(event::log_ingester)) - .route("/pipelines", routing::post(event::add_pipeline)) + .route("/pipelines/:pipeline_name", routing::post(event::add_pipeline)) .layer( ServiceBuilder::new() .layer(HandleErrorLayer::new(handle_error)) diff --git a/src/servers/src/http/event.rs b/src/servers/src/http/event.rs index 7c1e01f0ff2c..7e1a1f48d8e0 100644 --- a/src/servers/src/http/event.rs +++ b/src/servers/src/http/event.rs @@ -13,11 +13,16 @@ // limitations under the License. use std::collections::HashMap; +use std::result::Result as StdResult; use api::v1::{RowInsertRequest, RowInsertRequests, Rows}; -use axum::extract::{Json, Query, State}; +use axum::body::HttpBody; +use axum::extract::{FromRequest, Multipart, Path, Query, State}; use axum::headers::ContentType; -use axum::{Extension, TypedHeader}; +use axum::http::header::CONTENT_TYPE; +use axum::http::{Request, StatusCode}; +use axum::response::{IntoResponse, Response}; +use axum::{async_trait, BoxError, Extension, TypedHeader}; use common_telemetry::{error, warn}; use mime_guess::mime; use pipeline::error::{CastTypeSnafu, ExecPipelineSnafu}; @@ -43,24 +48,75 @@ pub struct LogIngesterQueryParams { pub ignore_errors: Option, } +pub struct PipelineContent(String); + +#[async_trait] +impl FromRequest for PipelineContent +where + B: HttpBody + Send + 'static, + B::Data: Send, + bytes::Bytes: std::convert::From<::Data>, + B::Error: Into, + S: Send + Sync, +{ + type Rejection = Response; + + async fn from_request(req: Request, state: &S) -> StdResult { + let content_type_header = req.headers().get(CONTENT_TYPE); + let content_type = content_type_header.and_then(|value| value.to_str().ok()); + if let Some(content_type) = content_type { + if content_type.ends_with("yaml") { + let payload = String::from_request(req, state) + .await + .map_err(IntoResponse::into_response)?; + return Ok(Self(payload)); + } + + if content_type.starts_with("multipart/form-data") { + let mut payload: Multipart = Multipart::from_request(req, state) + .await + .map_err(IntoResponse::into_response)?; + let file = payload + .next_field() + .await + .map_err(IntoResponse::into_response)?; + let payload = file + .ok_or(StatusCode::UNSUPPORTED_MEDIA_TYPE.into_response())? + .text() + .await + .map_err(IntoResponse::into_response)?; + return Ok(Self(payload)); + } + } + + Err(StatusCode::UNSUPPORTED_MEDIA_TYPE.into_response()) + } +} + #[axum_macros::debug_handler] pub async fn add_pipeline( State(handler): State, + Path(pipeline_name): Path, Extension(query_ctx): Extension, - Json(payload): Json, + PipelineContent(payload): PipelineContent, ) -> Result { - let name = payload["name"].as_str().context(InvalidParameterSnafu { - reason: "name is required in payload", - })?; - let pipeline = payload["pipeline"] - .as_str() - .context(InvalidParameterSnafu { - reason: "pipeline is required in payload", - })?; + if pipeline_name.is_empty() { + return Err(InvalidParameterSnafu { + reason: "pipeline_name is required in path", + } + .build()); + } + + if payload.is_empty() { + return Err(InvalidParameterSnafu { + reason: "pipeline is required in body", + } + .build()); + } let content_type = "yaml"; let result = handler - .insert_pipeline(name, content_type, pipeline, query_ctx) + .insert_pipeline(&pipeline_name, content_type, &payload, query_ctx) .await; result.map(|_| "ok".to_string()).map_err(|e| { From 43a57a7dd86095fded3a0cae94cb72438bf623f9 Mon Sep 17 00:00:00 2001 From: paomian Date: Thu, 6 Jun 2024 16:27:05 +0800 Subject: [PATCH 37/51] chore: fix by pr commit --- src/frontend/src/instance/log_handler.rs | 4 +- src/operator/src/insert.rs | 20 +++---- src/pipeline/src/lib.rs | 4 +- src/pipeline/src/{mng => manager}/error.rs | 14 ++--- src/pipeline/src/{mng => manager}/mod.rs | 0 .../src/{mng => manager}/pipeline_operator.rs | 2 +- src/pipeline/src/{mng => manager}/table.rs | 57 ++++++++++--------- src/servers/Cargo.toml | 1 + src/servers/src/error.rs | 8 --- src/servers/src/http.rs | 2 +- src/servers/src/http/event.rs | 23 ++++---- src/servers/src/query_handler.rs | 6 +- 12 files changed, 70 insertions(+), 71 deletions(-) rename src/pipeline/src/{mng => manager}/error.rs (89%) rename src/pipeline/src/{mng => manager}/mod.rs (100%) rename src/pipeline/src/{mng => manager}/pipeline_operator.rs (99%) rename src/pipeline/src/{mng => manager}/table.rs (84%) diff --git a/src/frontend/src/instance/log_handler.rs b/src/frontend/src/instance/log_handler.rs index ecc22b4935ec..a28310476bfc 100644 --- a/src/frontend/src/instance/log_handler.rs +++ b/src/frontend/src/instance/log_handler.rs @@ -27,7 +27,7 @@ use super::Instance; #[async_trait] impl LogHandler for Instance { - async fn insert_log( + async fn insert_logs( &self, log: RowInsertRequests, ctx: QueryContextRef, @@ -66,7 +66,7 @@ impl LogHandler for Instance { } async fn delete_pipeline(&self, _name: &str, _query_ctx: QueryContextRef) -> ServerResult<()> { - // todo(qtang): impl delete + // TODO(qtang): impl delete todo!("delete_pipeline") } } diff --git a/src/operator/src/insert.rs b/src/operator/src/insert.rs index 990435243b0d..df8d26ca626a 100644 --- a/src/operator/src/insert.rs +++ b/src/operator/src/insert.rs @@ -66,7 +66,7 @@ pub struct Inserter { pub type InserterRef = Arc; -enum TableType { +enum AutoCreateTableType { Logical(String), Physical, Log, @@ -117,7 +117,7 @@ impl Inserter { .create_or_alter_tables_on_demand( &requests, &ctx, - TableType::Physical, + AutoCreateTableType::Physical, statement_executor, ) .await?; @@ -144,7 +144,7 @@ impl Inserter { validate_column_count_match(&requests)?; let table_name_to_ids = self - .create_or_alter_tables_on_demand(&requests, &ctx, TableType::Log, statement_executor) + .create_or_alter_tables_on_demand(&requests, &ctx, AutoCreateTableType::Log, statement_executor) .await?; let inserts = RowToRegion::new(table_name_to_ids, self.partition_manager.as_ref()) .convert(requests) @@ -179,7 +179,7 @@ impl Inserter { .create_or_alter_tables_on_demand( &requests, &ctx, - TableType::Logical(physical_table.to_string()), + AutoCreateTableType::Logical(physical_table.to_string()), statement_executor, ) .await?; @@ -416,7 +416,7 @@ impl Inserter { &self, requests: &RowInsertRequests, ctx: &QueryContextRef, - table_type: TableType, + auto_create_table_type: AutoCreateTableType, statement_executor: &StatementExecutor, ) -> Result> { let mut table_name_to_ids = HashMap::with_capacity(requests.inserts.len()); @@ -444,8 +444,8 @@ impl Inserter { } } - match table_type { - TableType::Logical(on_physical_table) => { + match auto_create_table_type { + AutoCreateTableType::Logical(on_physical_table) => { if !create_tables.is_empty() { // Creates logical tables in batch. let tables = self @@ -469,7 +469,7 @@ impl Inserter { .await?; } } - TableType::Physical => { + AutoCreateTableType::Physical => { for req in create_tables { let table = self.create_table(req, ctx, statement_executor).await?; let table_info = table.table_info(); @@ -481,7 +481,7 @@ impl Inserter { .await?; } } - TableType::Log => { + AutoCreateTableType::Log => { for req in create_tables { let table = self.create_log_table(req, ctx, statement_executor).await?; let table_info = table.table_info(); @@ -643,7 +643,7 @@ impl Inserter { ) -> Result { let table_ref = TableReference::full(ctx.current_catalog(), ctx.current_schema(), &req.table_name); - + // SAFETY: `req.rows` is guaranteed to be `Some` by `handle_log_inserts`. let request_schema = req.rows.as_ref().unwrap().schema.as_slice(); let create_table_expr = &mut build_create_table_expr(&table_ref, request_schema)?; diff --git a/src/pipeline/src/lib.rs b/src/pipeline/src/lib.rs index b0f16bf8a62d..86ed9c7ea79b 100644 --- a/src/pipeline/src/lib.rs +++ b/src/pipeline/src/lib.rs @@ -13,9 +13,9 @@ // limitations under the License. mod etl; -mod mng; +mod manager; pub use etl::transform::GreptimeTransformer; pub use etl::value::Value; pub use etl::{parse, Content, Pipeline}; -pub use mng::{error, pipeline_operator, table}; +pub use manager::{error, pipeline_operator, table}; diff --git a/src/pipeline/src/mng/error.rs b/src/pipeline/src/manager/error.rs similarity index 89% rename from src/pipeline/src/mng/error.rs rename to src/pipeline/src/manager/error.rs index 1b67bd5055ba..a56f193d3b5e 100644 --- a/src/pipeline/src/mng/error.rs +++ b/src/pipeline/src/manager/error.rs @@ -44,7 +44,7 @@ pub enum Error { }, #[snafu(display("Failed to parse pipeline: {}", reason))] - ParsePipeline { + CompilePipeline { reason: String, #[snafu(implicit)] location: Location, @@ -101,7 +101,7 @@ pub enum Error { }, #[snafu(display("Failed to execute pipeline, reason: {}", reason))] - ExecPipeline { + PipelineTransform { reason: String, #[snafu(implicit)] location: Location, @@ -118,12 +118,10 @@ impl ErrorExt for Error { PipelineTableNotFound { .. } => StatusCode::TableNotFound, InsertPipeline { source, .. } => source.status_code(), CollectRecords { source, .. } => source.status_code(), - PipelineNotFound { .. } => StatusCode::InvalidArguments, - ParsePipeline { .. } => StatusCode::InvalidArguments, - // should we put `ExecPipeline` in internal? - // since pipeline is already compiled - // it's most likely an user input error - BuildDfLogicalPlan { .. } | ExecPipeline { .. } => StatusCode::Internal, + PipelineNotFound { .. } | CompilePipeline { .. } | PipelineTransform { .. } => { + StatusCode::InvalidArguments + } + BuildDfLogicalPlan { .. } => StatusCode::Internal, ExecuteInternalStatement { source, .. } => source.status_code(), Catalog { source, .. } => source.status_code(), CreateTable { source, .. } => source.status_code(), diff --git a/src/pipeline/src/mng/mod.rs b/src/pipeline/src/manager/mod.rs similarity index 100% rename from src/pipeline/src/mng/mod.rs rename to src/pipeline/src/manager/mod.rs diff --git a/src/pipeline/src/mng/pipeline_operator.rs b/src/pipeline/src/manager/pipeline_operator.rs similarity index 99% rename from src/pipeline/src/mng/pipeline_operator.rs rename to src/pipeline/src/manager/pipeline_operator.rs index 8540fa797be6..38e8d1bd9f2d 100644 --- a/src/pipeline/src/mng/pipeline_operator.rs +++ b/src/pipeline/src/manager/pipeline_operator.rs @@ -123,7 +123,7 @@ impl PipelineOperator { .context(PipelineTableNotFoundSnafu)?; info!( - "Created scripts table {}.", + "Created pipelines table {}.", table.table_info().full_table_name() ); diff --git a/src/pipeline/src/mng/table.rs b/src/pipeline/src/manager/table.rs similarity index 84% rename from src/pipeline/src/mng/table.rs rename to src/pipeline/src/manager/table.rs index 040924f799de..a6049d335d9f 100644 --- a/src/pipeline/src/mng/table.rs +++ b/src/pipeline/src/manager/table.rs @@ -41,8 +41,8 @@ use table::table::adapter::DfTableProviderAdapter; use table::TableRef; use crate::error::{ - BuildDfLogicalPlanSnafu, CastTypeSnafu, CollectRecordsSnafu, ExecuteInternalStatementSnafu, - InsertPipelineSnafu, ParsePipelineSnafu, PipelineNotFoundSnafu, Result, + BuildDfLogicalPlanSnafu, CastTypeSnafu, CollectRecordsSnafu, CompilePipeline, + ExecuteInternalStatementSnafu, InsertPipelineSnafu, PipelineNotFoundSnafu, Result, }; use crate::etl::transform::GreptimeTransformer; use crate::etl::{parse, Content, Pipeline}; @@ -50,6 +50,12 @@ use crate::etl::{parse, Content, Pipeline}; pub type PipelineTableRef = Arc; pub const PIPELINE_TABLE_NAME: &str = "pipelines"; + +pub const PIPELINE_TABLE_PIPELINE_NAME_COLUMN_NAME: &str = "name"; +pub const PIPELINE_TABLE_PIPELINE_SCHEMA_COLUMN_NAME: &str = "schema"; +pub const PIPELINE_TABLE_PIPELINE_CONTENT_TYPE_COLUMN_NAME: &str = "content_type"; +pub const PIPELINE_TABLE_PIPELINE_CONTENT_COLUMN_NAME: &str = "pipeline"; +pub const PIPELINE_TABLE_CREATED_AT_COLUMN_NAME: &str = "created_at"; pub struct PipelineTable { inserter: InserterRef, statement_executor: StatementExecutorRef, @@ -75,22 +81,16 @@ impl PipelineTable { } pub fn build_pipeline_schema() -> (String, Vec, Vec) { - let pipeline_name = "name"; - let schema_name = "schema"; - let content_type = "content_type"; - let pipeline_content = "pipeline"; - let created_at = "created_at"; - ( - created_at.to_string(), + PIPELINE_TABLE_CREATED_AT_COLUMN_NAME.to_string(), vec![ - schema_name.to_string(), - pipeline_name.to_string(), - content_type.to_string(), + PIPELINE_TABLE_PIPELINE_SCHEMA_COLUMN_NAME.to_string(), + PIPELINE_TABLE_PIPELINE_NAME_COLUMN_NAME.to_string(), + PIPELINE_TABLE_PIPELINE_CONTENT_TYPE_COLUMN_NAME.to_string(), ], vec![ ColumnDef { - name: pipeline_name.to_string(), + name: PIPELINE_TABLE_PIPELINE_NAME_COLUMN_NAME.to_string(), data_type: ColumnDataType::String as i32, is_nullable: false, default_constraint: vec![], @@ -99,7 +99,7 @@ impl PipelineTable { datatype_extension: None, }, ColumnDef { - name: schema_name.to_string(), + name: PIPELINE_TABLE_PIPELINE_SCHEMA_COLUMN_NAME.to_string(), data_type: ColumnDataType::String as i32, is_nullable: false, default_constraint: vec![], @@ -108,7 +108,7 @@ impl PipelineTable { datatype_extension: None, }, ColumnDef { - name: content_type.to_string(), + name: PIPELINE_TABLE_PIPELINE_CONTENT_TYPE_COLUMN_NAME.to_string(), data_type: ColumnDataType::String as i32, is_nullable: false, default_constraint: vec![], @@ -117,7 +117,7 @@ impl PipelineTable { datatype_extension: None, }, ColumnDef { - name: pipeline_content.to_string(), + name: PIPELINE_TABLE_PIPELINE_CONTENT_COLUMN_NAME.to_string(), data_type: ColumnDataType::String as i32, is_nullable: false, default_constraint: vec![], @@ -126,7 +126,7 @@ impl PipelineTable { datatype_extension: None, }, ColumnDef { - name: created_at.to_string(), + name: PIPELINE_TABLE_CREATED_AT_COLUMN_NAME.to_string(), data_type: ColumnDataType::TimestampMillisecond as i32, is_nullable: false, default_constraint: vec![], @@ -141,31 +141,31 @@ impl PipelineTable { fn build_insert_column_schemas() -> Vec { vec![ PbColumnSchema { - column_name: "name".to_string(), + column_name: PIPELINE_TABLE_PIPELINE_NAME_COLUMN_NAME.to_string(), datatype: ColumnDataType::String.into(), semantic_type: SemanticType::Tag.into(), ..Default::default() }, PbColumnSchema { - column_name: "schema".to_string(), + column_name: PIPELINE_TABLE_PIPELINE_SCHEMA_COLUMN_NAME.to_string(), datatype: ColumnDataType::String.into(), semantic_type: SemanticType::Tag.into(), ..Default::default() }, PbColumnSchema { - column_name: "content_type".to_string(), + column_name: PIPELINE_TABLE_PIPELINE_CONTENT_TYPE_COLUMN_NAME.to_string(), datatype: ColumnDataType::String.into(), semantic_type: SemanticType::Tag.into(), ..Default::default() }, PbColumnSchema { - column_name: "pipeline".to_string(), + column_name: PIPELINE_TABLE_PIPELINE_CONTENT_COLUMN_NAME.to_string(), datatype: ColumnDataType::String.into(), semantic_type: SemanticType::Field.into(), ..Default::default() }, PbColumnSchema { - column_name: "created_at".to_string(), + column_name: PIPELINE_TABLE_CREATED_AT_COLUMN_NAME.to_string(), datatype: ColumnDataType::TimestampMillisecond.into(), semantic_type: SemanticType::Timestamp.into(), ..Default::default() @@ -184,7 +184,7 @@ impl PipelineTable { pub fn compile_pipeline(pipeline: &str) -> Result> { let yaml_content = Content::Yaml(pipeline.into()); parse::(&yaml_content) - .map_err(|e| ParsePipelineSnafu { reason: e }.build()) + .map_err(|e| CompilePipeline { reason: e }.build()) } fn generate_pipeline_cache_key(schema: &str, name: &str) -> String { @@ -308,11 +308,16 @@ impl PipelineTable { let plan = LogicalPlanBuilder::scan(table_name, table_source, None) .context(BuildDfLogicalPlanSnafu)? .filter(and( - col("schema").eq(lit(schema)), - col("name").eq(lit(name)), + col(PIPELINE_TABLE_PIPELINE_SCHEMA_COLUMN_NAME).eq(lit(schema)), + col(PIPELINE_TABLE_PIPELINE_NAME_COLUMN_NAME).eq(lit(name)), )) .context(BuildDfLogicalPlanSnafu)? - .project(vec![col("pipeline")]) + .project(vec![col(PIPELINE_TABLE_PIPELINE_CONTENT_COLUMN_NAME)]) + .context(BuildDfLogicalPlanSnafu)? + .sort(vec![ + col(PIPELINE_TABLE_CREATED_AT_COLUMN_NAME), + lit("DESC"), + ]) .context(BuildDfLogicalPlanSnafu)? .build() .context(BuildDfLogicalPlanSnafu)?; diff --git a/src/servers/Cargo.toml b/src/servers/Cargo.toml index e02c8cd6dc7e..1c0837c7ebd7 100644 --- a/src/servers/Cargo.toml +++ b/src/servers/Cargo.toml @@ -105,6 +105,7 @@ tower = { workspace = true, features = ["full"] } tower-http = { version = "0.4", features = ["full"] } urlencoding = "2.1" zstd.workspace = true + [target.'cfg(not(windows))'.dependencies] tikv-jemalloc-ctl = { version = "0.5", features = ["use_std"] } diff --git a/src/servers/src/error.rs b/src/servers/src/error.rs index f3b28548d883..80c93444158f 100644 --- a/src/servers/src/error.rs +++ b/src/servers/src/error.rs @@ -562,13 +562,6 @@ pub enum Error { location: Location, }, - #[snafu(display("Failed to insert log. msg: {}", msg))] - InsertLog { - msg: String, - #[snafu(implicit)] - location: Location, - }, - #[snafu(display("Failed to decode url"))] UrlDecode { #[snafu(source)] @@ -678,7 +671,6 @@ impl ErrorExt for Error { | ParseJson { .. } | ToStructuredLog { .. } | UnsupportedContentType { .. } - | InsertLog { .. } | TimestampOverflow { .. } => StatusCode::InvalidArguments, RowWriter { source, .. } diff --git a/src/servers/src/http.rs b/src/servers/src/http.rs index efce02fec638..003b9a45db7b 100644 --- a/src/servers/src/http.rs +++ b/src/servers/src/http.rs @@ -591,7 +591,7 @@ impl HttpServerBuilder { pub fn with_log_ingest_handler(self, handler: LogHandlerRef) -> Self { Self { router: self.router.nest( - &format!("/{HTTP_API_VERSION}/event"), + &format!("/{HTTP_API_VERSION}/events"), HttpServer::route_log(handler), ), ..self diff --git a/src/servers/src/http/event.rs b/src/servers/src/http/event.rs index 7e1a1f48d8e0..927ca089296e 100644 --- a/src/servers/src/http/event.rs +++ b/src/servers/src/http/event.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashMap; use std::result::Result as StdResult; use api::v1::{RowInsertRequest, RowInsertRequests, Rows}; @@ -25,7 +24,7 @@ use axum::response::{IntoResponse, Response}; use axum::{async_trait, BoxError, Extension, TypedHeader}; use common_telemetry::{error, warn}; use mime_guess::mime; -use pipeline::error::{CastTypeSnafu, ExecPipelineSnafu}; +use pipeline::error::{CastTypeSnafu, PipelineTransformSnafu}; use pipeline::Value as PipelineValue; use schemars::JsonSchema; use serde::{Deserialize, Serialize}; @@ -190,16 +189,17 @@ pub async fn log_ingester( _ => UnsupportedContentTypeSnafu { content_type }.fail()?, }; - log_ingester_inner(handler, pipeline_name, table_name, value, query_ctx).await + ingest_logs_inner(handler, pipeline_name, table_name, value, query_ctx).await } -async fn log_ingester_inner( +async fn ingest_logs_inner( state: LogHandlerRef, pipeline_name: String, table_name: String, payload: Value, query_ctx: QueryContextRef, ) -> Result { + let start = std::time::Instant::now(); let pipeline_data = PipelineValue::try_from(payload) .map_err(|reason| CastTypeSnafu { msg: reason }.build()) .context(PipelineSnafu)?; @@ -209,7 +209,7 @@ async fn log_ingester_inner( .await?; let transformed_data: Rows = pipeline .exec(pipeline_data) - .map_err(|reason| ExecPipelineSnafu { reason }.build()) + .map_err(|reason| PipelineTransformSnafu { reason }.build()) .context(PipelineSnafu)?; let insert_request = RowInsertRequest { @@ -219,11 +219,10 @@ async fn log_ingester_inner( let insert_requests = RowInsertRequests { inserts: vec![insert_request], }; - state.insert_log(insert_requests, query_ctx).await.map(|_| { - HttpResponse::GreptimedbV1(GreptimedbV1Response { - output: vec![], - execution_time_ms: 0, - resp_metrics: HashMap::new(), - }) - }) + let output = state.insert_logs(insert_requests, query_ctx).await; + + let response = GreptimedbV1Response::from_output(vec![output]) + .await + .with_execution_time(start.elapsed().as_millis() as u64); + Ok(response) } diff --git a/src/servers/src/query_handler.rs b/src/servers/src/query_handler.rs index d6a280f4a685..9ddadcb2d494 100644 --- a/src/servers/src/query_handler.rs +++ b/src/servers/src/query_handler.rs @@ -121,9 +121,13 @@ pub trait OpenTelemetryProtocolHandler { ) -> Result; } +/// LogHandler is responsible for handling log related requests. +/// It should be able to insert logs and manage pipelines. +/// The pipeline is a series of transformations that can be applied to logs. +/// The pipeline is stored in the database and can be retrieved by name. #[async_trait] pub trait LogHandler { - async fn insert_log(&self, log: RowInsertRequests, ctx: QueryContextRef) -> Result; + async fn insert_logs(&self, log: RowInsertRequests, ctx: QueryContextRef) -> Result; async fn get_pipeline( &self, From 35602851788b55bb9e76c0c3fccfb376fae1a54d Mon Sep 17 00:00:00 2001 From: paomian Date: Thu, 6 Jun 2024 17:54:51 +0800 Subject: [PATCH 38/51] chore: add some doc for pub fn/struct --- src/pipeline/src/manager/pipeline_operator.rs | 26 ++++++++++++++----- src/pipeline/src/manager/table.rs | 16 ++++++++++-- 2 files changed, 33 insertions(+), 9 deletions(-) diff --git a/src/pipeline/src/manager/pipeline_operator.rs b/src/pipeline/src/manager/pipeline_operator.rs index 38e8d1bd9f2d..060bcd131cdf 100644 --- a/src/pipeline/src/manager/pipeline_operator.rs +++ b/src/pipeline/src/manager/pipeline_operator.rs @@ -32,6 +32,14 @@ use crate::{GreptimeTransformer, Pipeline}; pub const PIPELINE_TABLE_NAME: &str = "pipelines"; +/// PipelineOperator is responsible for managing pipelines. +/// It provides the ability to: +/// - Create a pipeline table if it does not exist +/// - Get a pipeline from the pipeline table +/// - Insert a pipeline into the pipeline table +/// - Compile a pipeline +/// - Add a pipeline table to the cache +/// - Get a pipeline table from the cache pub struct PipelineOperator { inserter: InserterRef, statement_executor: StatementExecutorRef, @@ -41,6 +49,7 @@ pub struct PipelineOperator { } impl PipelineOperator { + /// Create a table request for the pipeline table. pub fn create_table_request(&self, catalog: &str) -> RegisterSystemTableRequest { let (time_index, primary_keys, column_defs) = PipelineTable::build_pipeline_schema(); @@ -81,8 +90,7 @@ impl PipelineOperator { } async fn create_pipeline_table_if_not_exists(&self, ctx: QueryContextRef) -> Result<()> { - let catalog_str = ctx.current_catalog().to_owned(); - let catalog = catalog_str.as_str(); + let catalog = ctx.current_catalog(); // exist in cache if self.get_pipeline_table_from_cache(catalog).is_some() { @@ -105,19 +113,19 @@ impl PipelineOperator { return Ok(()); } - let schema = expr.schema_name.clone(); - let table_name = expr.table_name.clone(); - // create table self.statement_executor - .create_table_inner(&mut expr, None, ctx) + .create_table_inner(&mut expr, None, ctx.clone()) .await .context(CreateTableSnafu)?; + let schema = &expr.schema_name; + let table_name = &expr.table_name; + // get from catalog let table = self .catalog_manager - .table(catalog, &schema, &table_name) + .table(catalog, schema, table_name) .await .context(CatalogSnafu)? .context(PipelineTableNotFoundSnafu)?; @@ -133,6 +141,7 @@ impl PipelineOperator { Ok(()) } + /// Get a pipeline table from the cache. pub fn get_pipeline_table_from_cache(&self, catalog: &str) -> Option { self.tables.read().unwrap().get(catalog).cloned() } @@ -152,6 +161,7 @@ impl PipelineOperator { } impl PipelineOperator { + /// Create a new PipelineOperator. pub fn new( inserter: InserterRef, statement_executor: StatementExecutorRef, @@ -167,6 +177,7 @@ impl PipelineOperator { } } + /// Get a pipeline from the pipeline table. pub async fn get_pipeline( &self, query_ctx: QueryContextRef, @@ -180,6 +191,7 @@ impl PipelineOperator { .await } + /// Insert a pipeline into the pipeline table. pub async fn insert_pipeline( &self, name: &str, diff --git a/src/pipeline/src/manager/table.rs b/src/pipeline/src/manager/table.rs index a6049d335d9f..33e086d4db2b 100644 --- a/src/pipeline/src/manager/table.rs +++ b/src/pipeline/src/manager/table.rs @@ -41,7 +41,7 @@ use table::table::adapter::DfTableProviderAdapter; use table::TableRef; use crate::error::{ - BuildDfLogicalPlanSnafu, CastTypeSnafu, CollectRecordsSnafu, CompilePipeline, + BuildDfLogicalPlanSnafu, CastTypeSnafu, CollectRecordsSnafu, CompilePipelineSnafu, ExecuteInternalStatementSnafu, InsertPipelineSnafu, PipelineNotFoundSnafu, Result, }; use crate::etl::transform::GreptimeTransformer; @@ -56,6 +56,9 @@ pub const PIPELINE_TABLE_PIPELINE_SCHEMA_COLUMN_NAME: &str = "schema"; pub const PIPELINE_TABLE_PIPELINE_CONTENT_TYPE_COLUMN_NAME: &str = "content_type"; pub const PIPELINE_TABLE_PIPELINE_CONTENT_COLUMN_NAME: &str = "pipeline"; pub const PIPELINE_TABLE_CREATED_AT_COLUMN_NAME: &str = "created_at"; + +/// PipelineTable is a table that stores the pipeline schema and content. +/// Every catalog has its own pipeline table. pub struct PipelineTable { inserter: InserterRef, statement_executor: StatementExecutorRef, @@ -65,6 +68,7 @@ pub struct PipelineTable { } impl PipelineTable { + /// Create a new PipelineTable. pub fn new( inserter: InserterRef, statement_executor: StatementExecutorRef, @@ -80,6 +84,7 @@ impl PipelineTable { } } + /// Build the schema for the pipeline table. pub fn build_pipeline_schema() -> (String, Vec, Vec) { ( PIPELINE_TABLE_CREATED_AT_COLUMN_NAME.to_string(), @@ -138,6 +143,7 @@ impl PipelineTable { ) } + /// Build the column schemas for inserting a row into the pipeline table. fn build_insert_column_schemas() -> Vec { vec![ PbColumnSchema { @@ -181,10 +187,11 @@ impl PipelineTable { .into() } + /// Compile a pipeline from a string. pub fn compile_pipeline(pipeline: &str) -> Result> { let yaml_content = Content::Yaml(pipeline.into()); parse::(&yaml_content) - .map_err(|e| CompilePipeline { reason: e }.build()) + .map_err(|e| CompilePipelineSnafu { reason: e }.build()) } fn generate_pipeline_cache_key(schema: &str, name: &str) -> String { @@ -203,6 +210,7 @@ impl PipelineTable { .cloned() } + /// Insert a pipeline into the pipeline table. async fn insert_pipeline_to_pipeline_table( &self, schema: &str, @@ -255,6 +263,8 @@ impl PipelineTable { Ok(()) } + /// Get a pipeline by name. + /// If the pipeline is not in the cache, it will be get from table and compiled and inserted into the cache. pub async fn get_pipeline( &self, schema: &str, @@ -273,6 +283,8 @@ impl PipelineTable { Ok(compiled_pipeline) } + /// Insert a pipeline into the pipeline table and compile it. + /// The compiled pipeline will be inserted into the cache. pub async fn insert_and_compile( &self, schema: &str, From 4872c8aed73110e9ccc55a1f298744693ed4091c Mon Sep 17 00:00:00 2001 From: paomian Date: Thu, 6 Jun 2024 20:00:08 +0800 Subject: [PATCH 39/51] chore: some minro fix --- src/frontend/src/instance/log_handler.rs | 7 ++++-- src/operator/src/insert.rs | 28 ++++++++++++++++++++---- src/operator/src/metrics.rs | 6 +++++ src/servers/src/error.rs | 7 ++++++ 4 files changed, 42 insertions(+), 6 deletions(-) diff --git a/src/frontend/src/instance/log_handler.rs b/src/frontend/src/instance/log_handler.rs index a28310476bfc..0e6065ae869d 100644 --- a/src/frontend/src/instance/log_handler.rs +++ b/src/frontend/src/instance/log_handler.rs @@ -18,7 +18,10 @@ use auth::{PermissionChecker, PermissionCheckerRef, PermissionReq}; use client::Output; use common_error::ext::BoxedError; use pipeline::{GreptimeTransformer, Pipeline}; -use servers::error::{AuthSnafu, ExecuteGrpcRequestSnafu, PipelineSnafu, Result as ServerResult}; +use servers::error::{ + AuthSnafu, ExecuteGrpcRequestSnafu, PipelineSnafu, Result as ServerResult, + UnsupportedDeletePipelineSnafu, +}; use servers::query_handler::LogHandler; use session::context::QueryContextRef; use snafu::ResultExt; @@ -67,7 +70,7 @@ impl LogHandler for Instance { async fn delete_pipeline(&self, _name: &str, _query_ctx: QueryContextRef) -> ServerResult<()> { // TODO(qtang): impl delete - todo!("delete_pipeline") + Err(UnsupportedDeletePipelineSnafu {}.build()) } } diff --git a/src/operator/src/insert.rs b/src/operator/src/insert.rs index df8d26ca626a..adbfce278f8d 100644 --- a/src/operator/src/insert.rs +++ b/src/operator/src/insert.rs @@ -72,6 +72,16 @@ enum AutoCreateTableType { Log, } +impl AutoCreateTableType { + fn as_str(&self) -> &'static str { + match self { + AutoCreateTableType::Logical(_) => "logical", + AutoCreateTableType::Physical => "physical", + AutoCreateTableType::Log => "log", + } + } +} + impl Inserter { pub fn new( catalog_manager: CatalogManagerRef, @@ -144,7 +154,12 @@ impl Inserter { validate_column_count_match(&requests)?; let table_name_to_ids = self - .create_or_alter_tables_on_demand(&requests, &ctx, AutoCreateTableType::Log, statement_executor) + .create_or_alter_tables_on_demand( + &requests, + &ctx, + AutoCreateTableType::Log, + statement_executor, + ) .await?; let inserts = RowToRegion::new(table_name_to_ids, self.partition_manager.as_ref()) .convert(requests) @@ -422,6 +437,9 @@ impl Inserter { let mut table_name_to_ids = HashMap::with_capacity(requests.inserts.len()); let mut create_tables = vec![]; let mut alter_tables = vec![]; + let _timer = crate::metrics::CREATE_OR_ALTER_TABLES + .with_label_values(&[auto_create_table_type.as_str()]) + .start_timer(); for req in &requests.inserts { let catalog = ctx.current_catalog(); let schema = ctx.current_schema(); @@ -647,7 +665,9 @@ impl Inserter { let request_schema = req.rows.as_ref().unwrap().schema.as_slice(); let create_table_expr = &mut build_create_table_expr(&table_ref, request_schema)?; - info!("Table `{table_ref}` does not exist, try creating table"); + info!("Table `{table_ref}` does not exist, try creating the log table"); + // Set append_mode to true for log table. + // because log table does not need to esure the ts and tags unique. create_table_expr .table_options .insert("append_mode".to_string(), "true".to_string()); @@ -658,14 +678,14 @@ impl Inserter { match res { Ok(table) => { info!( - "Successfully created table {}.{}.{}", + "Successfully created a log table {}.{}.{}", table_ref.catalog, table_ref.schema, table_ref.table, ); Ok(table) } Err(err) => { error!( - "Failed to create table {}.{}.{}: {}", + "Failed to create a log table {}.{}.{}: {}", table_ref.catalog, table_ref.schema, table_ref.table, err ); Err(err) diff --git a/src/operator/src/metrics.rs b/src/operator/src/metrics.rs index 97c5e0015a55..f761956f7054 100644 --- a/src/operator/src/metrics.rs +++ b/src/operator/src/metrics.rs @@ -51,4 +51,10 @@ lazy_static! { "DDL operator create view" ) .unwrap(); + pub static ref CREATE_OR_ALTER_TABLES: HistogramVec = register_histogram_vec!( + "greptime_grpc_region_request", + "grpc region request", + &["table_type"] + ) + .unwrap(); } diff --git a/src/servers/src/error.rs b/src/servers/src/error.rs index 80c93444158f..04b6fa196ca2 100644 --- a/src/servers/src/error.rs +++ b/src/servers/src/error.rs @@ -156,6 +156,12 @@ pub enum Error { location: Location, }, + #[snafu(display("Unsupported delete pipeline."))] + UnsupportedDeletePipeline { + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("Failed to execute script by name: {}", name))] ExecuteScript { name: String, @@ -629,6 +635,7 @@ impl ErrorExt for Error { | FileWatch { .. } => StatusCode::Internal, UnsupportedDataType { .. } => StatusCode::Unsupported, + UnsupportedDeletePipeline { .. } => StatusCode::Unsupported, #[cfg(not(windows))] UpdateJemallocMetrics { .. } => StatusCode::Internal, From 11933b018f7dba1a66552f421319e1241ceee519 Mon Sep 17 00:00:00 2001 From: paomian Date: Thu, 6 Jun 2024 21:40:06 +0800 Subject: [PATCH 40/51] chore: add pipeline version support --- src/frontend/src/instance/log_handler.rs | 3 +- src/pipeline/src/manager/pipeline_operator.rs | 3 +- src/pipeline/src/manager/table.rs | 94 +++++++++++++------ src/servers/src/http/event.rs | 17 +++- src/servers/src/query_handler.rs | 1 + 5 files changed, 86 insertions(+), 32 deletions(-) diff --git a/src/frontend/src/instance/log_handler.rs b/src/frontend/src/instance/log_handler.rs index 0e6065ae869d..a5bb9c5e9c0f 100644 --- a/src/frontend/src/instance/log_handler.rs +++ b/src/frontend/src/instance/log_handler.rs @@ -47,10 +47,11 @@ impl LogHandler for Instance { async fn get_pipeline( &self, name: &str, + version: Option, query_ctx: QueryContextRef, ) -> ServerResult> { self.pipeline_operator - .get_pipeline(query_ctx, name) + .get_pipeline(query_ctx, name, version) .await .context(PipelineSnafu) } diff --git a/src/pipeline/src/manager/pipeline_operator.rs b/src/pipeline/src/manager/pipeline_operator.rs index 060bcd131cdf..854163bf25c4 100644 --- a/src/pipeline/src/manager/pipeline_operator.rs +++ b/src/pipeline/src/manager/pipeline_operator.rs @@ -182,12 +182,13 @@ impl PipelineOperator { &self, query_ctx: QueryContextRef, name: &str, + version: Option, ) -> Result> { self.create_pipeline_table_if_not_exists(query_ctx.clone()) .await?; self.get_pipeline_table_from_cache(query_ctx.current_catalog()) .context(PipelineTableNotFoundSnafu)? - .get_pipeline(query_ctx.current_schema(), name) + .get_pipeline(query_ctx.current_schema(), name, version) .await } diff --git a/src/pipeline/src/manager/table.rs b/src/pipeline/src/manager/table.rs index 33e086d4db2b..1156fb6a08cd 100644 --- a/src/pipeline/src/manager/table.rs +++ b/src/pipeline/src/manager/table.rs @@ -29,7 +29,8 @@ use datafusion::logical_expr::{and, col, lit}; use datafusion_common::TableReference; use datafusion_expr::LogicalPlanBuilder; use datatypes::prelude::ScalarVector; -use datatypes::vectors::{StringVector, Vector}; +use datatypes::timestamp::TimestampMillisecond; +use datatypes::vectors::{StringVector, TimestampMillisecondVector, Vector}; use operator::insert::InserterRef; use operator::statement::StatementExecutorRef; use query::plan::LogicalPlan; @@ -194,19 +195,20 @@ impl PipelineTable { .map_err(|e| CompilePipelineSnafu { reason: e }.build()) } - fn generate_pipeline_cache_key(schema: &str, name: &str) -> String { - format!("{}.{}", schema, name) + fn generate_pipeline_cache_key(schema: &str, name: &str, _version: Option<&str>) -> String { + format!("{}-{}", schema, name) } fn get_compiled_pipeline_from_cache( &self, schema: &str, name: &str, + version: Option<&str>, ) -> Option> { self.pipelines .read() .unwrap() - .get(&Self::generate_pipeline_cache_key(schema, name)) + .get(&Self::generate_pipeline_cache_key(schema, name, version)) .cloned() } @@ -217,7 +219,7 @@ impl PipelineTable { name: &str, content_type: &str, pipeline: &str, - ) -> Result<()> { + ) -> Result { let now = util::current_time_millis(); let table_info = self.table.table_info(); @@ -260,7 +262,7 @@ impl PipelineTable { output ); - Ok(()) + Ok(now) } /// Get a pipeline by name. @@ -269,15 +271,18 @@ impl PipelineTable { &self, schema: &str, name: &str, + version: Option, ) -> Result> { - if let Some(pipeline) = self.get_compiled_pipeline_from_cache(schema, name) { + if let Some(pipeline) = + self.get_compiled_pipeline_from_cache(schema, name, version.as_deref()) + { return Ok(pipeline); } - let pipeline = self.find_pipeline_by_name(schema, name).await?; - let compiled_pipeline = Self::compile_pipeline(&pipeline)?; + let pipeline = self.find_pipeline_by_name(schema, name, version).await?; + let compiled_pipeline = Self::compile_pipeline(&pipeline.0)?; self.pipelines.write().unwrap().insert( - Self::generate_pipeline_cache_key(schema, name), + Self::generate_pipeline_cache_key(schema, name, None), compiled_pipeline.clone(), ); Ok(compiled_pipeline) @@ -293,19 +298,25 @@ impl PipelineTable { pipeline: &str, ) -> Result> { let compiled_pipeline = Self::compile_pipeline(pipeline)?; - - self.insert_pipeline_to_pipeline_table(schema, name, content_type, pipeline) + // we will use the version in the future + let _version = self + .insert_pipeline_to_pipeline_table(schema, name, content_type, pipeline) .await?; self.pipelines.write().unwrap().insert( - Self::generate_pipeline_cache_key(schema, name), + Self::generate_pipeline_cache_key(schema, name, None), compiled_pipeline.clone(), ); Ok(compiled_pipeline) } - async fn find_pipeline_by_name(&self, schema: &str, name: &str) -> Result { + async fn find_pipeline_by_name( + &self, + schema: &str, + name: &str, + version: Option, + ) -> Result<(String, TimestampMillisecond)> { let table_info = self.table.table_info(); let table_name = TableReference::full( @@ -316,21 +327,34 @@ impl PipelineTable { let table_provider = Arc::new(DfTableProviderAdapter::new(self.table.clone())); let table_source = Arc::new(DefaultTableSource::new(table_provider)); + let schema_and_name_filter = and( + col(PIPELINE_TABLE_PIPELINE_SCHEMA_COLUMN_NAME).eq(lit(schema)), + col(PIPELINE_TABLE_PIPELINE_NAME_COLUMN_NAME).eq(lit(name)), + ); + let filter = if let Some(v) = version { + and( + schema_and_name_filter, + col(PIPELINE_TABLE_CREATED_AT_COLUMN_NAME).lt_eq(lit(v)), + ) + } else { + schema_and_name_filter + }; let plan = LogicalPlanBuilder::scan(table_name, table_source, None) .context(BuildDfLogicalPlanSnafu)? - .filter(and( - col(PIPELINE_TABLE_PIPELINE_SCHEMA_COLUMN_NAME).eq(lit(schema)), - col(PIPELINE_TABLE_PIPELINE_NAME_COLUMN_NAME).eq(lit(name)), - )) + .filter(filter) .context(BuildDfLogicalPlanSnafu)? - .project(vec![col(PIPELINE_TABLE_PIPELINE_CONTENT_COLUMN_NAME)]) + .project(vec![ + col(PIPELINE_TABLE_PIPELINE_CONTENT_COLUMN_NAME), + col(PIPELINE_TABLE_CREATED_AT_COLUMN_NAME), + ]) .context(BuildDfLogicalPlanSnafu)? .sort(vec![ - col(PIPELINE_TABLE_CREATED_AT_COLUMN_NAME), - lit("DESC"), + col(PIPELINE_TABLE_CREATED_AT_COLUMN_NAME).sort(true, true) ]) .context(BuildDfLogicalPlanSnafu)? + .limit(0, Some(1)) + .context(BuildDfLogicalPlanSnafu)? .build() .context(BuildDfLogicalPlanSnafu)?; @@ -350,26 +374,40 @@ impl PipelineTable { .context(CollectRecordsSnafu)?; ensure!(!records.is_empty(), PipelineNotFoundSnafu { name }); - // assume schema + name is unique for now + ensure!( - records.len() == 1 && records[0].num_columns() == 1, + records.len() == 1 && records[0].num_columns() == 2, PipelineNotFoundSnafu { name } ); - let pipeline_column = records[0].column(0); - let pipeline_column = pipeline_column + let pipeline_content_column = records[0].column(0); + let pipeline_content = pipeline_content_column .as_any() .downcast_ref::() .with_context(|| CastTypeSnafu { msg: format!( "can't downcast {:?} array into string vector", - pipeline_column.data_type() + pipeline_content_column.data_type() + ), + })?; + + let pipeline_created_at_column = records[0].column(1); + let pipeline_created_at = pipeline_created_at_column + .as_any() + .downcast_ref::() + .with_context(|| CastTypeSnafu { + msg: format!( + "can't downcast {:?} array into scalar vector", + pipeline_created_at_column.data_type() ), })?; - ensure!(pipeline_column.len() == 1, PipelineNotFoundSnafu { name }); + ensure!(pipeline_content.len() == 1, PipelineNotFoundSnafu { name }); // Safety: asserted above - Ok(pipeline_column.get_data(0).unwrap().to_string()) + Ok(( + pipeline_content.get_data(0).unwrap().to_string(), + pipeline_created_at.get_data(0).unwrap(), + )) } } diff --git a/src/servers/src/http/event.rs b/src/servers/src/http/event.rs index 927ca089296e..852f9933814a 100644 --- a/src/servers/src/http/event.rs +++ b/src/servers/src/http/event.rs @@ -45,6 +45,8 @@ pub struct LogIngesterQueryParams { pub db: Option, pub pipeline_name: Option, pub ignore_errors: Option, + // we can't resolve the version correctly now. + // pub version: Option, } pub struct PipelineContent(String); @@ -177,6 +179,8 @@ pub async fn log_ingester( reason: "table is required", })?; + let version = None; + let ignore_errors = query_params.ignore_errors.unwrap_or(false); let m: mime::Mime = content_type.clone().into(); @@ -189,12 +193,21 @@ pub async fn log_ingester( _ => UnsupportedContentTypeSnafu { content_type }.fail()?, }; - ingest_logs_inner(handler, pipeline_name, table_name, value, query_ctx).await + ingest_logs_inner( + handler, + pipeline_name, + version, + table_name, + value, + query_ctx, + ) + .await } async fn ingest_logs_inner( state: LogHandlerRef, pipeline_name: String, + version: Option, table_name: String, payload: Value, query_ctx: QueryContextRef, @@ -205,7 +218,7 @@ async fn ingest_logs_inner( .context(PipelineSnafu)?; let pipeline = state - .get_pipeline(&pipeline_name, query_ctx.clone()) + .get_pipeline(&pipeline_name, version, query_ctx.clone()) .await?; let transformed_data: Rows = pipeline .exec(pipeline_data) diff --git a/src/servers/src/query_handler.rs b/src/servers/src/query_handler.rs index 9ddadcb2d494..fd775e6e1f1e 100644 --- a/src/servers/src/query_handler.rs +++ b/src/servers/src/query_handler.rs @@ -132,6 +132,7 @@ pub trait LogHandler { async fn get_pipeline( &self, name: &str, + version: Option, query_ctx: QueryContextRef, ) -> Result>; From 92a2bda7c2fbf6bbd95389b9eb224ced47786fd1 Mon Sep 17 00:00:00 2001 From: paomian Date: Fri, 7 Jun 2024 12:20:18 +0800 Subject: [PATCH 41/51] chore: impl log pipeline version --- src/operator/src/metrics.rs | 2 +- src/pipeline/src/manager/error.rs | 3 +- src/pipeline/src/manager/table.rs | 81 ++++++++++++++++++++++--------- src/servers/src/http/event.rs | 6 +-- 4 files changed, 63 insertions(+), 29 deletions(-) diff --git a/src/operator/src/metrics.rs b/src/operator/src/metrics.rs index f761956f7054..f03ec99008a5 100644 --- a/src/operator/src/metrics.rs +++ b/src/operator/src/metrics.rs @@ -52,7 +52,7 @@ lazy_static! { ) .unwrap(); pub static ref CREATE_OR_ALTER_TABLES: HistogramVec = register_histogram_vec!( - "greptime_grpc_region_request", + "greptime_create_or_alter_tables", "grpc region request", &["table_type"] ) diff --git a/src/pipeline/src/manager/error.rs b/src/pipeline/src/manager/error.rs index a56f193d3b5e..60f891fd4774 100644 --- a/src/pipeline/src/manager/error.rs +++ b/src/pipeline/src/manager/error.rs @@ -50,9 +50,10 @@ pub enum Error { location: Location, }, - #[snafu(display("Pipeline not found, name: {}", name))] + #[snafu(display("Pipeline not found, name: {}, version: {}", name, version.clone().unwrap_or("latest".to_string())))] PipelineNotFound { name: String, + version: Option, #[snafu(implicit)] location: Location, }, diff --git a/src/pipeline/src/manager/table.rs b/src/pipeline/src/manager/table.rs index 1156fb6a08cd..a8709f22b096 100644 --- a/src/pipeline/src/manager/table.rs +++ b/src/pipeline/src/manager/table.rs @@ -23,14 +23,14 @@ use api::v1::{ use common_query::OutputData; use common_recordbatch::util as record_util; use common_telemetry::info; -use common_time::util; +use common_time::timestamp::{TimeUnit, Timestamp}; use datafusion::datasource::DefaultTableSource; use datafusion::logical_expr::{and, col, lit}; use datafusion_common::TableReference; use datafusion_expr::LogicalPlanBuilder; use datatypes::prelude::ScalarVector; -use datatypes::timestamp::TimestampMillisecond; -use datatypes::vectors::{StringVector, TimestampMillisecondVector, Vector}; +use datatypes::timestamp::TimestampNanosecond; +use datatypes::vectors::{StringVector, TimestampNanosecondVector, Vector}; use operator::insert::InserterRef; use operator::statement::StatementExecutorRef; use query::plan::LogicalPlan; @@ -133,7 +133,7 @@ impl PipelineTable { }, ColumnDef { name: PIPELINE_TABLE_CREATED_AT_COLUMN_NAME.to_string(), - data_type: ColumnDataType::TimestampMillisecond as i32, + data_type: ColumnDataType::TimestampNanosecond as i32, is_nullable: false, default_constraint: vec![], semantic_type: SemanticType::Timestamp as i32, @@ -173,7 +173,7 @@ impl PipelineTable { }, PbColumnSchema { column_name: PIPELINE_TABLE_CREATED_AT_COLUMN_NAME.to_string(), - datatype: ColumnDataType::TimestampMillisecond.into(), + datatype: ColumnDataType::TimestampNanosecond.into(), semantic_type: SemanticType::Timestamp.into(), ..Default::default() }, @@ -195,8 +195,8 @@ impl PipelineTable { .map_err(|e| CompilePipelineSnafu { reason: e }.build()) } - fn generate_pipeline_cache_key(schema: &str, name: &str, _version: Option<&str>) -> String { - format!("{}-{}", schema, name) + fn generate_pipeline_cache_key(schema: &str, name: &str, version: Option<&str>) -> String { + format!("{}-{}-{}", schema, name, version.unwrap_or("latest")) } fn get_compiled_pipeline_from_cache( @@ -219,8 +219,8 @@ impl PipelineTable { name: &str, content_type: &str, pipeline: &str, - ) -> Result { - let now = util::current_time_millis(); + ) -> Result { + let now = Timestamp::current_time(TimeUnit::Nanosecond); let table_info = self.table.table_info(); @@ -234,7 +234,7 @@ impl PipelineTable { ValueData::StringValue(schema.to_string()).into(), ValueData::StringValue(content_type.to_string()).into(), ValueData::StringValue(pipeline.to_string()).into(), - ValueData::TimestampMillisecondValue(now).into(), + ValueData::TimestampNanosecondValue(now.value()).into(), ], }], }), @@ -273,16 +273,20 @@ impl PipelineTable { name: &str, version: Option, ) -> Result> { + info!("pipeline version: {:?}", version); if let Some(pipeline) = self.get_compiled_pipeline_from_cache(schema, name, version.as_deref()) { return Ok(pipeline); } - let pipeline = self.find_pipeline_by_name(schema, name, version).await?; + let pipeline = self + .find_pipeline_by_name(schema, name, version.as_ref()) + .await?; let compiled_pipeline = Self::compile_pipeline(&pipeline.0)?; + self.pipelines.write().unwrap().insert( - Self::generate_pipeline_cache_key(schema, name, None), + Self::generate_pipeline_cache_key(schema, name, version.as_deref()), compiled_pipeline.clone(), ); Ok(compiled_pipeline) @@ -299,14 +303,21 @@ impl PipelineTable { ) -> Result> { let compiled_pipeline = Self::compile_pipeline(pipeline)?; // we will use the version in the future - let _version = self + let version = self .insert_pipeline_to_pipeline_table(schema, name, content_type, pipeline) .await?; - self.pipelines.write().unwrap().insert( - Self::generate_pipeline_cache_key(schema, name, None), - compiled_pipeline.clone(), - ); + { + let mut g = self.pipelines.write().unwrap(); + g.insert( + Self::generate_pipeline_cache_key(schema, name, None), + compiled_pipeline.clone(), + ); + g.insert( + Self::generate_pipeline_cache_key(schema, name, Some(&version.to_iso8601_string())), + compiled_pipeline.clone(), + ); + } Ok(compiled_pipeline) } @@ -315,8 +326,8 @@ impl PipelineTable { &self, schema: &str, name: &str, - version: Option, - ) -> Result<(String, TimestampMillisecond)> { + version: Option<&String>, + ) -> Result<(String, TimestampNanosecond)> { let table_info = self.table.table_info(); let table_name = TableReference::full( @@ -334,7 +345,7 @@ impl PipelineTable { let filter = if let Some(v) = version { and( schema_and_name_filter, - col(PIPELINE_TABLE_CREATED_AT_COLUMN_NAME).lt_eq(lit(v)), + col(PIPELINE_TABLE_CREATED_AT_COLUMN_NAME).eq(lit(v)), ) } else { schema_and_name_filter @@ -358,6 +369,8 @@ impl PipelineTable { .build() .context(BuildDfLogicalPlanSnafu)?; + info!("find_pipeline_by_name: plan: {:?}", plan); + let output = self .query_engine .execute(LogicalPlan::DfPlan(plan), Self::query_ctx(&table_info)) @@ -373,11 +386,20 @@ impl PipelineTable { .await .context(CollectRecordsSnafu)?; - ensure!(!records.is_empty(), PipelineNotFoundSnafu { name }); + ensure!( + !records.is_empty(), + PipelineNotFoundSnafu { + name, + version: version.cloned() + } + ); ensure!( records.len() == 1 && records[0].num_columns() == 2, - PipelineNotFoundSnafu { name } + PipelineNotFoundSnafu { + name, + version: version.cloned() + } ); let pipeline_content_column = records[0].column(0); @@ -394,7 +416,7 @@ impl PipelineTable { let pipeline_created_at_column = records[0].column(1); let pipeline_created_at = pipeline_created_at_column .as_any() - .downcast_ref::() + .downcast_ref::() .with_context(|| CastTypeSnafu { msg: format!( "can't downcast {:?} array into scalar vector", @@ -402,7 +424,18 @@ impl PipelineTable { ), })?; - ensure!(pipeline_content.len() == 1, PipelineNotFoundSnafu { name }); + info!( + "find_pipeline_by_name: pipeline_content: {:?}, pipeline_created_at: {:?}", + pipeline_content, pipeline_created_at + ); + + ensure!( + pipeline_content.len() == 1, + PipelineNotFoundSnafu { + name, + version: version.cloned() + } + ); // Safety: asserted above Ok(( diff --git a/src/servers/src/http/event.rs b/src/servers/src/http/event.rs index 852f9933814a..f49cf63f8c42 100644 --- a/src/servers/src/http/event.rs +++ b/src/servers/src/http/event.rs @@ -45,8 +45,8 @@ pub struct LogIngesterQueryParams { pub db: Option, pub pipeline_name: Option, pub ignore_errors: Option, - // we can't resolve the version correctly now. - // pub version: Option, + + pub version: Option, } pub struct PipelineContent(String); @@ -179,7 +179,7 @@ pub async fn log_ingester( reason: "table is required", })?; - let version = None; + let version = query_params.version; let ignore_errors = query_params.ignore_errors.unwrap_or(false); From 8216854ce1d78dda94566c320e3066c122ea312d Mon Sep 17 00:00:00 2001 From: paomian Date: Wed, 12 Jun 2024 11:52:30 +0800 Subject: [PATCH 42/51] chore: fix format issue --- src/servers/Cargo.toml | 2 +- src/servers/src/http.rs | 5 ++++- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/src/servers/Cargo.toml b/src/servers/Cargo.toml index 1c0837c7ebd7..bb30a1533145 100644 --- a/src/servers/Cargo.toml +++ b/src/servers/Cargo.toml @@ -22,7 +22,7 @@ arrow-ipc.workspace = true arrow-schema.workspace = true async-trait = "0.1" auth.workspace = true -axum = {workspace=true,features = ["multipart"]} +axum = { workspace = true, features = ["multipart"] } axum-macros = "0.3.8" base64.workspace = true bytes.workspace = true diff --git a/src/servers/src/http.rs b/src/servers/src/http.rs index 003b9a45db7b..3f7f71653f73 100644 --- a/src/servers/src/http.rs +++ b/src/servers/src/http.rs @@ -713,7 +713,10 @@ impl HttpServer { fn route_log(log_handler: LogHandlerRef) -> Router { Router::new() .route("/logs", routing::post(event::log_ingester)) - .route("/pipelines/:pipeline_name", routing::post(event::add_pipeline)) + .route( + "/pipelines/:pipeline_name", + routing::post(event::add_pipeline), + ) .layer( ServiceBuilder::new() .layer(HandleErrorLayer::new(handle_error)) From 6aed1314ab2b29fb410b12460ded24adb912ff34 Mon Sep 17 00:00:00 2001 From: paomian Date: Wed, 12 Jun 2024 14:25:55 +0800 Subject: [PATCH 43/51] fix: make the LogicalPlan of a query pipeline sorted in desc order --- src/pipeline/src/manager/table.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/pipeline/src/manager/table.rs b/src/pipeline/src/manager/table.rs index a8709f22b096..cc24c449dc1e 100644 --- a/src/pipeline/src/manager/table.rs +++ b/src/pipeline/src/manager/table.rs @@ -361,7 +361,7 @@ impl PipelineTable { ]) .context(BuildDfLogicalPlanSnafu)? .sort(vec![ - col(PIPELINE_TABLE_CREATED_AT_COLUMN_NAME).sort(true, true) + col(PIPELINE_TABLE_CREATED_AT_COLUMN_NAME).sort(false, true) ]) .context(BuildDfLogicalPlanSnafu)? .limit(0, Some(1)) From db827df6cac53779139fcb9eb5e5d395321c63ad Mon Sep 17 00:00:00 2001 From: paomian Date: Thu, 13 Jun 2024 14:23:20 +0800 Subject: [PATCH 44/51] chore: remove some debug log --- src/pipeline/src/manager/table.rs | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/pipeline/src/manager/table.rs b/src/pipeline/src/manager/table.rs index cc24c449dc1e..04ce982643f0 100644 --- a/src/pipeline/src/manager/table.rs +++ b/src/pipeline/src/manager/table.rs @@ -22,7 +22,7 @@ use api::v1::{ }; use common_query::OutputData; use common_recordbatch::util as record_util; -use common_telemetry::info; +use common_telemetry::{debug, info}; use common_time::timestamp::{TimeUnit, Timestamp}; use datafusion::datasource::DefaultTableSource; use datafusion::logical_expr::{and, col, lit}; @@ -273,7 +273,6 @@ impl PipelineTable { name: &str, version: Option, ) -> Result> { - info!("pipeline version: {:?}", version); if let Some(pipeline) = self.get_compiled_pipeline_from_cache(schema, name, version.as_deref()) { @@ -369,7 +368,7 @@ impl PipelineTable { .build() .context(BuildDfLogicalPlanSnafu)?; - info!("find_pipeline_by_name: plan: {:?}", plan); + debug!("find_pipeline_by_name: plan: {:?}", plan); let output = self .query_engine @@ -424,7 +423,7 @@ impl PipelineTable { ), })?; - info!( + debug!( "find_pipeline_by_name: pipeline_content: {:?}, pipeline_created_at: {:?}", pipeline_content, pipeline_created_at ); From 998b243f8a3a27a1795b2e4afcbd428c8b8a4575 Mon Sep 17 00:00:00 2001 From: paomian Date: Fri, 14 Jun 2024 14:40:28 +0800 Subject: [PATCH 45/51] chore: replacing hashmap cache with moak --- Cargo.lock | 1 + src/frontend/src/instance/log_handler.rs | 4 +- src/pipeline/Cargo.toml | 1 + src/pipeline/src/manager/pipeline_operator.rs | 4 +- src/pipeline/src/manager/table.rs | 37 +++++++++++-------- src/servers/src/query_handler.rs | 2 +- 6 files changed, 29 insertions(+), 20 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 246a09fb8366..bfc51e8240ec 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7310,6 +7310,7 @@ dependencies = [ "greptime-proto", "itertools 0.10.5", "lazy_static", + "moka", "once_cell", "operator", "paste", diff --git a/src/frontend/src/instance/log_handler.rs b/src/frontend/src/instance/log_handler.rs index a5bb9c5e9c0f..0eef1a816aa8 100644 --- a/src/frontend/src/instance/log_handler.rs +++ b/src/frontend/src/instance/log_handler.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::sync::Arc; + use api::v1::RowInsertRequests; use async_trait::async_trait; use auth::{PermissionChecker, PermissionCheckerRef, PermissionReq}; @@ -49,7 +51,7 @@ impl LogHandler for Instance { name: &str, version: Option, query_ctx: QueryContextRef, - ) -> ServerResult> { + ) -> ServerResult>> { self.pipeline_operator .get_pipeline(query_ctx, name, version) .await diff --git a/src/pipeline/Cargo.toml b/src/pipeline/Cargo.toml index f44c9825e63f..b07e31fdb7f1 100644 --- a/src/pipeline/Cargo.toml +++ b/src/pipeline/Cargo.toml @@ -52,6 +52,7 @@ table.workspace = true tokio.workspace = true urlencoding = "2.1" yaml-rust = "0.4" +moka = { workspace = true, features = ["sync"] } [dev-dependencies] catalog = { workspace = true, features = ["testing"] } diff --git a/src/pipeline/src/manager/pipeline_operator.rs b/src/pipeline/src/manager/pipeline_operator.rs index 854163bf25c4..9229e9f32e94 100644 --- a/src/pipeline/src/manager/pipeline_operator.rs +++ b/src/pipeline/src/manager/pipeline_operator.rs @@ -152,7 +152,7 @@ impl PipelineOperator { name: &str, content_type: &str, pipeline: &str, - ) -> Result> { + ) -> Result>> { self.get_pipeline_table_from_cache(ctx.current_catalog()) .context(PipelineTableNotFoundSnafu)? .insert_and_compile(ctx.current_schema(), name, content_type, pipeline) @@ -183,7 +183,7 @@ impl PipelineOperator { query_ctx: QueryContextRef, name: &str, version: Option, - ) -> Result> { + ) -> Result>> { self.create_pipeline_table_if_not_exists(query_ctx.clone()) .await?; self.get_pipeline_table_from_cache(query_ctx.current_catalog()) diff --git a/src/pipeline/src/manager/table.rs b/src/pipeline/src/manager/table.rs index 04ce982643f0..0bca6ef69f85 100644 --- a/src/pipeline/src/manager/table.rs +++ b/src/pipeline/src/manager/table.rs @@ -12,8 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashMap; -use std::sync::{Arc, RwLock}; +use std::sync::Arc; +use std::time::Duration; use api::v1::value::ValueData; use api::v1::{ @@ -31,6 +31,7 @@ use datafusion_expr::LogicalPlanBuilder; use datatypes::prelude::ScalarVector; use datatypes::timestamp::TimestampNanosecond; use datatypes::vectors::{StringVector, TimestampNanosecondVector, Vector}; +use moka::sync::Cache; use operator::insert::InserterRef; use operator::statement::StatementExecutorRef; use query::plan::LogicalPlan; @@ -58,6 +59,11 @@ pub const PIPELINE_TABLE_PIPELINE_CONTENT_TYPE_COLUMN_NAME: &str = "content_type pub const PIPELINE_TABLE_PIPELINE_CONTENT_COLUMN_NAME: &str = "pipeline"; pub const PIPELINE_TABLE_CREATED_AT_COLUMN_NAME: &str = "created_at"; +/// Pipeline table cache size. +pub const PIPELINES_CACHE_SIZE: u64 = 10000; +/// Pipeline table cache time to live. unit: second +pub const PIPELINES_CACHE_TTL: u64 = 10; + /// PipelineTable is a table that stores the pipeline schema and content. /// Every catalog has its own pipeline table. pub struct PipelineTable { @@ -65,7 +71,7 @@ pub struct PipelineTable { statement_executor: StatementExecutorRef, table: TableRef, query_engine: QueryEngineRef, - pipelines: RwLock>>, + pipelines: Cache>>, } impl PipelineTable { @@ -81,7 +87,10 @@ impl PipelineTable { statement_executor, table, query_engine, - pipelines: RwLock::new(HashMap::default()), + pipelines: Cache::builder() + .max_capacity(PIPELINES_CACHE_SIZE) + .time_to_live(Duration::from_secs(10)) + .build(), } } @@ -204,12 +213,9 @@ impl PipelineTable { schema: &str, name: &str, version: Option<&str>, - ) -> Option> { + ) -> Option>> { self.pipelines - .read() - .unwrap() .get(&Self::generate_pipeline_cache_key(schema, name, version)) - .cloned() } /// Insert a pipeline into the pipeline table. @@ -272,7 +278,7 @@ impl PipelineTable { schema: &str, name: &str, version: Option, - ) -> Result> { + ) -> Result>> { if let Some(pipeline) = self.get_compiled_pipeline_from_cache(schema, name, version.as_deref()) { @@ -282,9 +288,9 @@ impl PipelineTable { let pipeline = self .find_pipeline_by_name(schema, name, version.as_ref()) .await?; - let compiled_pipeline = Self::compile_pipeline(&pipeline.0)?; + let compiled_pipeline = Arc::new(Self::compile_pipeline(&pipeline.0)?); - self.pipelines.write().unwrap().insert( + self.pipelines.insert( Self::generate_pipeline_cache_key(schema, name, version.as_deref()), compiled_pipeline.clone(), ); @@ -299,20 +305,19 @@ impl PipelineTable { name: &str, content_type: &str, pipeline: &str, - ) -> Result> { - let compiled_pipeline = Self::compile_pipeline(pipeline)?; + ) -> Result>> { + let compiled_pipeline = Arc::new(Self::compile_pipeline(pipeline)?); // we will use the version in the future let version = self .insert_pipeline_to_pipeline_table(schema, name, content_type, pipeline) .await?; { - let mut g = self.pipelines.write().unwrap(); - g.insert( + self.pipelines.insert( Self::generate_pipeline_cache_key(schema, name, None), compiled_pipeline.clone(), ); - g.insert( + self.pipelines.insert( Self::generate_pipeline_cache_key(schema, name, Some(&version.to_iso8601_string())), compiled_pipeline.clone(), ); diff --git a/src/servers/src/query_handler.rs b/src/servers/src/query_handler.rs index fd775e6e1f1e..2e860b7f9299 100644 --- a/src/servers/src/query_handler.rs +++ b/src/servers/src/query_handler.rs @@ -134,7 +134,7 @@ pub trait LogHandler { name: &str, version: Option, query_ctx: QueryContextRef, - ) -> Result>; + ) -> Result>>; async fn insert_pipeline( &self, From 12c3e04fd98e7d34c7ca61072d9915a0091fdb90 Mon Sep 17 00:00:00 2001 From: paomian Date: Fri, 14 Jun 2024 14:43:26 +0800 Subject: [PATCH 46/51] chore: fix by pr commit --- src/frontend/src/instance/log_handler.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/frontend/src/instance/log_handler.rs b/src/frontend/src/instance/log_handler.rs index 0eef1a816aa8..6baf0672ec0d 100644 --- a/src/frontend/src/instance/log_handler.rs +++ b/src/frontend/src/instance/log_handler.rs @@ -28,7 +28,7 @@ use servers::query_handler::LogHandler; use session::context::QueryContextRef; use snafu::ResultExt; -use super::Instance; +use crate::instance::Instance; #[async_trait] impl LogHandler for Instance { From 7820cd79e2b0fb29fe59896566257d699b270e36 Mon Sep 17 00:00:00 2001 From: paomian Date: Fri, 14 Jun 2024 14:48:17 +0800 Subject: [PATCH 47/51] chore: fix toml format issue --- src/pipeline/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/pipeline/Cargo.toml b/src/pipeline/Cargo.toml index b07e31fdb7f1..03096b47a7a1 100644 --- a/src/pipeline/Cargo.toml +++ b/src/pipeline/Cargo.toml @@ -38,6 +38,7 @@ futures.workspace = true greptime-proto.workspace = true itertools.workspace = true lazy_static.workspace = true +moka = { workspace = true, features = ["sync"] } once_cell.workspace = true operator.workspace = true paste.workspace = true @@ -52,7 +53,6 @@ table.workspace = true tokio.workspace = true urlencoding = "2.1" yaml-rust = "0.4" -moka = { workspace = true, features = ["sync"] } [dev-dependencies] catalog = { workspace = true, features = ["testing"] } From 0d35e1a5f9ebdc07d8f72e3d62c3a8caf2014345 Mon Sep 17 00:00:00 2001 From: paomian Date: Fri, 14 Jun 2024 16:00:22 +0800 Subject: [PATCH 48/51] chore: update Cargo.lock --- Cargo.lock | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Cargo.lock b/Cargo.lock index 8f21a51372af..3bd96784858f 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7284,7 +7284,7 @@ checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" [[package]] name = "pipeline" -version = "0.8.1" +version = "0.8.2" dependencies = [ "api", "arrow", From b57eed628c6f063ebb7bb62c8cc17f38b5986baa Mon Sep 17 00:00:00 2001 From: paomian Date: Fri, 14 Jun 2024 17:37:43 +0800 Subject: [PATCH 49/51] chore: fix by pr commit --- src/frontend/src/instance/log_handler.rs | 3 +- src/operator/src/insert.rs | 12 ++--- src/operator/src/metrics.rs | 4 +- src/pipeline/src/manager/error.rs | 14 ++--- src/pipeline/src/manager/pipeline_operator.rs | 9 ++-- src/pipeline/src/manager/table.rs | 54 ++++++++----------- src/servers/src/http/event.rs | 40 +++++++++----- src/servers/src/query_handler.rs | 3 +- 8 files changed, 68 insertions(+), 71 deletions(-) diff --git a/src/frontend/src/instance/log_handler.rs b/src/frontend/src/instance/log_handler.rs index 6baf0672ec0d..6ef48205cc56 100644 --- a/src/frontend/src/instance/log_handler.rs +++ b/src/frontend/src/instance/log_handler.rs @@ -19,6 +19,7 @@ use async_trait::async_trait; use auth::{PermissionChecker, PermissionCheckerRef, PermissionReq}; use client::Output; use common_error::ext::BoxedError; +use pipeline::table::PipelineVersion; use pipeline::{GreptimeTransformer, Pipeline}; use servers::error::{ AuthSnafu, ExecuteGrpcRequestSnafu, PipelineSnafu, Result as ServerResult, @@ -49,7 +50,7 @@ impl LogHandler for Instance { async fn get_pipeline( &self, name: &str, - version: Option, + version: PipelineVersion, query_ctx: QueryContextRef, ) -> ServerResult>> { self.pipeline_operator diff --git a/src/operator/src/insert.rs b/src/operator/src/insert.rs index d0e31c98e624..4caf06c0ca90 100644 --- a/src/operator/src/insert.rs +++ b/src/operator/src/insert.rs @@ -666,7 +666,7 @@ impl Inserter { info!("Table `{table_ref}` does not exist, try creating the log table"); // Set append_mode to true for log table. - // because log table does not need to esure the ts and tags unique. + // because log tables should keep rows with the same ts and tags. create_table_expr .table_options .insert("append_mode".to_string(), "true".to_string()); @@ -676,17 +676,11 @@ impl Inserter { match res { Ok(table) => { - info!( - "Successfully created a log table {}.{}.{}", - table_ref.catalog, table_ref.schema, table_ref.table, - ); + info!("Successfully created a log table {}", table_ref); Ok(table) } Err(err) => { - error!( - "Failed to create a log table {}.{}.{}: {}", - table_ref.catalog, table_ref.schema, table_ref.table, err - ); + error!(err; "Failed to create a log table {}", table_ref); Err(err) } } diff --git a/src/operator/src/metrics.rs b/src/operator/src/metrics.rs index f03ec99008a5..f902465677b3 100644 --- a/src/operator/src/metrics.rs +++ b/src/operator/src/metrics.rs @@ -52,8 +52,8 @@ lazy_static! { ) .unwrap(); pub static ref CREATE_OR_ALTER_TABLES: HistogramVec = register_histogram_vec!( - "greptime_create_or_alter_tables", - "grpc region request", + "greptime_table_operator_create_or_alter_tables", + "table operator create or alter tables", &["table_type"] ) .unwrap(); diff --git a/src/pipeline/src/manager/error.rs b/src/pipeline/src/manager/error.rs index 60f891fd4774..ad5d8a96bebd 100644 --- a/src/pipeline/src/manager/error.rs +++ b/src/pipeline/src/manager/error.rs @@ -17,19 +17,13 @@ use std::any::Any; use common_error::ext::ErrorExt; use common_error::status_code::StatusCode; use common_macro::stack_trace_debug; +use datatypes::timestamp::TimestampNanosecond; use snafu::{Location, Snafu}; #[derive(Snafu)] #[snafu(visibility(pub))] #[stack_trace_debug] pub enum Error { - #[snafu(display("Failed to find column in pipeline table, name: {}", name))] - FindColumnInPipelineTable { - name: String, - #[snafu(implicit)] - location: Location, - }, - #[snafu(display("Pipeline table not found"))] PipelineTableNotFound { #[snafu(implicit)] @@ -50,10 +44,10 @@ pub enum Error { location: Location, }, - #[snafu(display("Pipeline not found, name: {}, version: {}", name, version.clone().unwrap_or("latest".to_string())))] + #[snafu(display("Pipeline not found, name: {}, version: {}", name, version.map(|ts| ts.0.to_iso8601_string()).unwrap_or("latest".to_string())))] PipelineNotFound { name: String, - version: Option, + version: Option, #[snafu(implicit)] location: Location, }, @@ -115,7 +109,7 @@ impl ErrorExt for Error { fn status_code(&self) -> StatusCode { use Error::*; match self { - FindColumnInPipelineTable { .. } | CastType { .. } => StatusCode::Unexpected, + CastType { .. } => StatusCode::Unexpected, PipelineTableNotFound { .. } => StatusCode::TableNotFound, InsertPipeline { source, .. } => source.status_code(), CollectRecords { source, .. } => source.status_code(), diff --git a/src/pipeline/src/manager/pipeline_operator.rs b/src/pipeline/src/manager/pipeline_operator.rs index 9229e9f32e94..390a48d834a4 100644 --- a/src/pipeline/src/manager/pipeline_operator.rs +++ b/src/pipeline/src/manager/pipeline_operator.rs @@ -27,7 +27,7 @@ use snafu::{OptionExt, ResultExt}; use table::TableRef; use crate::error::{CatalogSnafu, CreateTableSnafu, PipelineTableNotFoundSnafu, Result}; -use crate::table::{PipelineTable, PipelineTableRef}; +use crate::table::{PipelineTable, PipelineTableRef, PipelineVersion}; use crate::{GreptimeTransformer, Pipeline}; pub const PIPELINE_TABLE_NAME: &str = "pipelines"; @@ -131,8 +131,9 @@ impl PipelineOperator { .context(PipelineTableNotFoundSnafu)?; info!( - "Created pipelines table {}.", - table.table_info().full_table_name() + "Created pipelines table {} with table id {}.", + table.table_info().full_table_name(), + table.table_info().table_id() ); // put to cache @@ -182,7 +183,7 @@ impl PipelineOperator { &self, query_ctx: QueryContextRef, name: &str, - version: Option, + version: PipelineVersion, ) -> Result>> { self.create_pipeline_table_if_not_exists(query_ctx.clone()) .await?; diff --git a/src/pipeline/src/manager/table.rs b/src/pipeline/src/manager/table.rs index 0bca6ef69f85..662e740a04c3 100644 --- a/src/pipeline/src/manager/table.rs +++ b/src/pipeline/src/manager/table.rs @@ -49,6 +49,8 @@ use crate::error::{ use crate::etl::transform::GreptimeTransformer; use crate::etl::{parse, Content, Pipeline}; +pub type PipelineVersion = Option; + pub type PipelineTableRef = Arc; pub const PIPELINE_TABLE_NAME: &str = "pipelines"; @@ -61,8 +63,8 @@ pub const PIPELINE_TABLE_CREATED_AT_COLUMN_NAME: &str = "created_at"; /// Pipeline table cache size. pub const PIPELINES_CACHE_SIZE: u64 = 10000; -/// Pipeline table cache time to live. unit: second -pub const PIPELINES_CACHE_TTL: u64 = 10; +/// Pipeline table cache time to live. +pub const PIPELINES_CACHE_TTL: Duration = Duration::from_secs(10); /// PipelineTable is a table that stores the pipeline schema and content. /// Every catalog has its own pipeline table. @@ -89,12 +91,13 @@ impl PipelineTable { query_engine, pipelines: Cache::builder() .max_capacity(PIPELINES_CACHE_SIZE) - .time_to_live(Duration::from_secs(10)) + .time_to_live(PIPELINES_CACHE_TTL) .build(), } } /// Build the schema for the pipeline table. + /// Returns the (time index, primary keys, column) definitions. pub fn build_pipeline_schema() -> (String, Vec, Vec) { ( PIPELINE_TABLE_CREATED_AT_COLUMN_NAME.to_string(), @@ -204,15 +207,18 @@ impl PipelineTable { .map_err(|e| CompilePipelineSnafu { reason: e }.build()) } - fn generate_pipeline_cache_key(schema: &str, name: &str, version: Option<&str>) -> String { - format!("{}-{}-{}", schema, name, version.unwrap_or("latest")) + fn generate_pipeline_cache_key(schema: &str, name: &str, version: PipelineVersion) -> String { + match version { + Some(version) => format!("{}/{}/{}", schema, name, Into::::into(version)), + None => format!("{}/{}/latest", schema, name), + } } fn get_compiled_pipeline_from_cache( &self, schema: &str, name: &str, - version: Option<&str>, + version: PipelineVersion, ) -> Option>> { self.pipelines .get(&Self::generate_pipeline_cache_key(schema, name, version)) @@ -277,21 +283,17 @@ impl PipelineTable { &self, schema: &str, name: &str, - version: Option, + version: PipelineVersion, ) -> Result>> { - if let Some(pipeline) = - self.get_compiled_pipeline_from_cache(schema, name, version.as_deref()) - { + if let Some(pipeline) = self.get_compiled_pipeline_from_cache(schema, name, version) { return Ok(pipeline); } - let pipeline = self - .find_pipeline_by_name(schema, name, version.as_ref()) - .await?; + let pipeline = self.find_pipeline_by_name(schema, name, version).await?; let compiled_pipeline = Arc::new(Self::compile_pipeline(&pipeline.0)?); self.pipelines.insert( - Self::generate_pipeline_cache_key(schema, name, version.as_deref()), + Self::generate_pipeline_cache_key(schema, name, version), compiled_pipeline.clone(), ); Ok(compiled_pipeline) @@ -318,7 +320,7 @@ impl PipelineTable { compiled_pipeline.clone(), ); self.pipelines.insert( - Self::generate_pipeline_cache_key(schema, name, Some(&version.to_iso8601_string())), + Self::generate_pipeline_cache_key(schema, name, Some(TimestampNanosecond(version))), compiled_pipeline.clone(), ); } @@ -330,7 +332,7 @@ impl PipelineTable { &self, schema: &str, name: &str, - version: Option<&String>, + version: PipelineVersion, ) -> Result<(String, TimestampNanosecond)> { let table_info = self.table.table_info(); @@ -349,7 +351,7 @@ impl PipelineTable { let filter = if let Some(v) = version { and( schema_and_name_filter, - col(PIPELINE_TABLE_CREATED_AT_COLUMN_NAME).eq(lit(v)), + col(PIPELINE_TABLE_CREATED_AT_COLUMN_NAME).eq(lit(v.0.to_iso8601_string())), ) } else { schema_and_name_filter @@ -390,20 +392,11 @@ impl PipelineTable { .await .context(CollectRecordsSnafu)?; - ensure!( - !records.is_empty(), - PipelineNotFoundSnafu { - name, - version: version.cloned() - } - ); + ensure!(!records.is_empty(), PipelineNotFoundSnafu { name, version }); ensure!( records.len() == 1 && records[0].num_columns() == 2, - PipelineNotFoundSnafu { - name, - version: version.cloned() - } + PipelineNotFoundSnafu { name, version } ); let pipeline_content_column = records[0].column(0); @@ -435,10 +428,7 @@ impl PipelineTable { ensure!( pipeline_content.len() == 1, - PipelineNotFoundSnafu { - name, - version: version.cloned() - } + PipelineNotFoundSnafu { name, version } ); // Safety: asserted above diff --git a/src/servers/src/http/event.rs b/src/servers/src/http/event.rs index f49cf63f8c42..f9939b80572e 100644 --- a/src/servers/src/http/event.rs +++ b/src/servers/src/http/event.rs @@ -23,8 +23,11 @@ use axum::http::{Request, StatusCode}; use axum::response::{IntoResponse, Response}; use axum::{async_trait, BoxError, Extension, TypedHeader}; use common_telemetry::{error, warn}; +use common_time::Timestamp; +use datatypes::timestamp::TimestampNanosecond; use mime_guess::mime; use pipeline::error::{CastTypeSnafu, PipelineTransformSnafu}; +use pipeline::table::PipelineVersion; use pipeline::Value as PipelineValue; use schemars::JsonSchema; use serde::{Deserialize, Serialize}; @@ -128,10 +131,12 @@ pub async fn add_pipeline( /// Transform NDJSON array into a single array fn transform_ndjson_array_factory( + values: impl IntoIterator>, ignore_error: bool, -) -> impl FnMut(Result, std::result::Result) -> Result { - move |acc, item| { - acc.and_then(|acc| match acc { +) -> Result { + values.into_iter().try_fold( + Value::Array(Vec::with_capacity(100)), + |acc, item| match acc { Value::Array(mut acc_array) => { if let Ok(item_value) = item { match item_value { @@ -144,10 +149,10 @@ fn transform_ndjson_array_factory( _ => { if !ignore_error { warn!("invalid item in array: {:?}", item_value); - return Err(InvalidParameterSnafu { + return InvalidParameterSnafu { reason: format!("invalid item:{} in array", item_value), } - .build()); + .fail(); } } } @@ -160,8 +165,8 @@ fn transform_ndjson_array_factory( } } _ => unreachable!("invalid acc: {:?}", acc), - }) - } + }, + ) } #[axum_macros::debug_handler] @@ -179,15 +184,26 @@ pub async fn log_ingester( reason: "table is required", })?; - let version = query_params.version; + let version = match query_params.version { + Some(version) => { + let ts = Timestamp::from_str_utc(&version).map_err(|e| { + InvalidParameterSnafu { + reason: format!("invalid pipeline version: {} with error: {}", &version, e), + } + .build() + })?; + Some(TimestampNanosecond(ts)) + } + None => None, + }; let ignore_errors = query_params.ignore_errors.unwrap_or(false); let m: mime::Mime = content_type.clone().into(); let value = match m.subtype() { - mime::JSON => Deserializer::from_str(&payload).into_iter::().fold( - Ok(Value::Array(Vec::with_capacity(100))), - transform_ndjson_array_factory(ignore_errors), + mime::JSON => transform_ndjson_array_factory( + Deserializer::from_str(&payload).into_iter(), + ignore_errors, )?, // add more content type support _ => UnsupportedContentTypeSnafu { content_type }.fail()?, @@ -207,7 +223,7 @@ pub async fn log_ingester( async fn ingest_logs_inner( state: LogHandlerRef, pipeline_name: String, - version: Option, + version: PipelineVersion, table_name: String, payload: Value, query_ctx: QueryContextRef, diff --git a/src/servers/src/query_handler.rs b/src/servers/src/query_handler.rs index 2e860b7f9299..f0c1170e0746 100644 --- a/src/servers/src/query_handler.rs +++ b/src/servers/src/query_handler.rs @@ -35,6 +35,7 @@ use common_query::Output; use headers::HeaderValue; use opentelemetry_proto::tonic::collector::metrics::v1::ExportMetricsServiceRequest; use opentelemetry_proto::tonic::collector::trace::v1::ExportTraceServiceRequest; +use pipeline::table::PipelineVersion; use pipeline::{GreptimeTransformer, Pipeline}; use serde_json::Value; use session::context::QueryContextRef; @@ -132,7 +133,7 @@ pub trait LogHandler { async fn get_pipeline( &self, name: &str, - version: Option, + version: PipelineVersion, query_ctx: QueryContextRef, ) -> Result>>; From 4433e6c0966aadf9c4929bfa72aaecf315933373 Mon Sep 17 00:00:00 2001 From: paomian Date: Fri, 14 Jun 2024 23:29:56 +0800 Subject: [PATCH 50/51] chore: fix some issue by pr commit --- src/operator/src/insert.rs | 12 +++--------- src/operator/src/metrics.rs | 6 +++--- src/pipeline/src/manager/table.rs | 4 +++- 3 files changed, 9 insertions(+), 13 deletions(-) diff --git a/src/operator/src/insert.rs b/src/operator/src/insert.rs index 4caf06c0ca90..4bae21f987b9 100644 --- a/src/operator/src/insert.rs +++ b/src/operator/src/insert.rs @@ -437,7 +437,7 @@ impl Inserter { let mut table_name_to_ids = HashMap::with_capacity(requests.inserts.len()); let mut create_tables = vec![]; let mut alter_tables = vec![]; - let _timer = crate::metrics::CREATE_OR_ALTER_TABLES + let _timer = crate::metrics::CREATE_ALTER_ON_DEMAND .with_label_values(&[auto_create_table_type.as_str()]) .start_timer(); for req in &requests.inserts { @@ -636,17 +636,11 @@ impl Inserter { match res { Ok(table) => { - info!( - "Successfully created table {}.{}.{}", - table_ref.catalog, table_ref.schema, table_ref.table, - ); + info!("Successfully created table {}", table_ref,); Ok(table) } Err(err) => { - error!( - "Failed to create table {}.{}.{}: {}", - table_ref.catalog, table_ref.schema, table_ref.table, err - ); + error!(err; "Failed to create table {}", table_ref); Err(err) } } diff --git a/src/operator/src/metrics.rs b/src/operator/src/metrics.rs index f902465677b3..9a77f9844d38 100644 --- a/src/operator/src/metrics.rs +++ b/src/operator/src/metrics.rs @@ -51,9 +51,9 @@ lazy_static! { "DDL operator create view" ) .unwrap(); - pub static ref CREATE_OR_ALTER_TABLES: HistogramVec = register_histogram_vec!( - "greptime_table_operator_create_or_alter_tables", - "table operator create or alter tables", + pub static ref CREATE_ALTER_ON_DEMAND: HistogramVec = register_histogram_vec!( + "greptime_table_operator_create_alter_on_demand", + "table operator duration to create or alter tables on demand", &["table_type"] ) .unwrap(); diff --git a/src/pipeline/src/manager/table.rs b/src/pipeline/src/manager/table.rs index 662e740a04c3..9338b8e03ca0 100644 --- a/src/pipeline/src/manager/table.rs +++ b/src/pipeline/src/manager/table.rs @@ -49,6 +49,8 @@ use crate::error::{ use crate::etl::transform::GreptimeTransformer; use crate::etl::{parse, Content, Pipeline}; +/// Pipeline version. string formatted as iso8601 timestamp with nanosecond precision +/// if the version is None, it means the latest version of the pipeline pub type PipelineVersion = Option; pub type PipelineTableRef = Arc; @@ -209,7 +211,7 @@ impl PipelineTable { fn generate_pipeline_cache_key(schema: &str, name: &str, version: PipelineVersion) -> String { match version { - Some(version) => format!("{}/{}/{}", schema, name, Into::::into(version)), + Some(version) => format!("{}/{}/{}", schema, name, i64::from(version)), None => format!("{}/{}/latest", schema, name), } } From a6734d393bd9ec66336a06d2d7142211f1d087a1 Mon Sep 17 00:00:00 2001 From: paomian Date: Sat, 15 Jun 2024 00:21:25 +0800 Subject: [PATCH 51/51] chore: add more doc for pipeline version --- src/pipeline/src/manager/table.rs | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/pipeline/src/manager/table.rs b/src/pipeline/src/manager/table.rs index 9338b8e03ca0..d037ae3d4832 100644 --- a/src/pipeline/src/manager/table.rs +++ b/src/pipeline/src/manager/table.rs @@ -49,8 +49,10 @@ use crate::error::{ use crate::etl::transform::GreptimeTransformer; use crate::etl::{parse, Content, Pipeline}; -/// Pipeline version. string formatted as iso8601 timestamp with nanosecond precision -/// if the version is None, it means the latest version of the pipeline +/// Pipeline version. An optional timestamp with nanosecond precision. +/// If the version is None, it means the latest version of the pipeline. +/// User can specify the version by providing a timestamp string formatted as iso8601. +/// When it used in cache key, it will be converted to i64 meaning the number of nanoseconds since the epoch. pub type PipelineVersion = Option; pub type PipelineTableRef = Arc;