From 6658bb925fd88e5e5319b427a70340091f578e96 Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw Date: Sat, 19 Oct 2024 22:03:06 +0530 Subject: [PATCH 01/36] adding metadata fn --- crates/arroyo-planner/src/lib.rs | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/crates/arroyo-planner/src/lib.rs b/crates/arroyo-planner/src/lib.rs index f737498db..585ff54ee 100644 --- a/crates/arroyo-planner/src/lib.rs +++ b/crates/arroyo-planner/src/lib.rs @@ -194,7 +194,18 @@ impl ArroyoSchemaProvider { ) }), ); - + // Registering kafka connector metadata function + functions.insert( + "metadata".to_string(), + Arc::new(create_udf( + "metadata", + vec![DataType::Utf8], + Arc::new(DataType::Utf8), + Volatility::Volatile, + #[allow(deprecated)] + make_scalar_function(fn_impl), + )), + ); let mut registry = Self { functions, ..Default::default() From 3bc6929639fefa0c2da9577a741745452b88b59c Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw Date: Mon, 21 Oct 2024 10:25:41 +0530 Subject: [PATCH 02/36] Adding check if metadata column fn --- crates/arroyo-planner/src/tables.rs | 18 +++++++++++++++--- 1 file changed, 15 insertions(+), 3 deletions(-) diff --git a/crates/arroyo-planner/src/tables.rs b/crates/arroyo-planner/src/tables.rs index aff07e9e0..b8213020d 100644 --- a/crates/arroyo-planner/src/tables.rs +++ b/crates/arroyo-planner/src/tables.rs @@ -4,6 +4,7 @@ use std::{collections::HashMap, time::Duration}; use arrow_schema::{DataType, Field, FieldRef, Schema}; use arroyo_connectors::connector_for_type; +use datafusion::logical_expr::expr::ScalarFunction; use crate::extension::remote_table::RemoteTableExtension; use crate::types::convert_data_type; @@ -25,8 +26,7 @@ use datafusion::common::{plan_err, Column, DataFusionError}; use datafusion::execution::context::SessionState; use datafusion::execution::FunctionRegistry; use datafusion::logical_expr::{ - CreateMemoryTable, CreateView, DdlStatement, DmlStatement, Expr, Extension, LogicalPlan, - WriteOp, + CreateMemoryTable, CreateView, DdlStatement, DmlStatement, Expr, Extension, LogicalPlan, WriteOp }; use datafusion::optimizer::common_subexpr_eliminate::CommonSubexprEliminate; use datafusion::optimizer::decorrelate_predicate_subquery::DecorrelatePredicateSubquery; @@ -97,6 +97,17 @@ impl FieldSpec { FieldSpec::VirtualField { field, .. } => field, } } + fn is_metadata_virtual(&self) -> bool { + match self { + FieldSpec::VirtualField { expression, .. } => { + if let Expr::ScalarFunction(ScalarFunction { func, .. }) = expression { + return func.name() == "metadata"; + } + false + } + _ => false, + } + } } impl From for FieldSpec { @@ -255,7 +266,7 @@ impl ConnectorTable { let schema_fields: Vec = input_to_schema_fields .iter() - .filter(|f| !f.is_virtual()) + .filter(|f| f.is_metadata_virtual() || !f.is_virtual()) .map(|f| { let struct_field = f.field(); struct_field.clone().try_into().map_err(|_| { @@ -267,6 +278,7 @@ impl ConnectorTable { }) }) .collect::>()?; + println!("schema_fields: {:?}", schema_fields); let bad_data = BadData::from_opts(options) .map_err(|e| DataFusionError::Plan(format!("Invalid bad_data: '{e}'")))?; From abff63a55cd0dd6a2024f6a0dbe1342771ea5369 Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw Date: Mon, 21 Oct 2024 10:27:09 +0530 Subject: [PATCH 03/36] remove log for debugs --- crates/arroyo-planner/src/tables.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/crates/arroyo-planner/src/tables.rs b/crates/arroyo-planner/src/tables.rs index b8213020d..df5cae6ba 100644 --- a/crates/arroyo-planner/src/tables.rs +++ b/crates/arroyo-planner/src/tables.rs @@ -278,7 +278,6 @@ impl ConnectorTable { }) }) .collect::>()?; - println!("schema_fields: {:?}", schema_fields); let bad_data = BadData::from_opts(options) .map_err(|e| DataFusionError::Plan(format!("Invalid bad_data: '{e}'")))?; From 440f7008f2a335aaadeb815f0d612ae8923ec86d Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw Date: Wed, 23 Oct 2024 16:13:20 +0530 Subject: [PATCH 04/36] updating schema to columns --- crates/arroyo-planner/src/tables.rs | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/crates/arroyo-planner/src/tables.rs b/crates/arroyo-planner/src/tables.rs index df5cae6ba..0c4cc8c5f 100644 --- a/crates/arroyo-planner/src/tables.rs +++ b/crates/arroyo-planner/src/tables.rs @@ -502,10 +502,12 @@ impl Table { columns: &[ColumnDef], schema_provider: &ArroyoSchemaProvider, ) -> Result> { + println!("columns {:?}", columns); let struct_field_pairs = columns .iter() .map(|column| { let name = column.name.value.to_string(); + println!("column name {:?}", name); let (data_type, extension) = convert_data_type(&column.data_type)?; let nullable = !column .options @@ -522,6 +524,14 @@ impl Table { generation_expr, .. } = &option.option { + // check if generation_expr is a scalar function and if it is, check if it is a metadata function + if let Some(sqlparser::ast::Expr::Function(sqlparser::ast::Function { name, .. })) = generation_expr { + if name.0.iter().any(|ident| ident.value.to_lowercase() == "metadata") { + println!("Detected metadata function!"); + // TODO: add metadata to the struct field like argument of metadata function. + return None; + } + } generation_expr.clone() } else { None From 3abecab242121a12f4c7cd3634af00f1d242accb Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw Date: Thu, 24 Oct 2024 11:17:57 +0530 Subject: [PATCH 05/36] Adding validations and data to kafka messages --- .../arroyo-connectors/src/kafka/source/mod.rs | 44 ++++++++++++++++++- crates/arroyo-planner/src/tables.rs | 28 ++++++++---- 2 files changed, 63 insertions(+), 9 deletions(-) diff --git a/crates/arroyo-connectors/src/kafka/source/mod.rs b/crates/arroyo-connectors/src/kafka/source/mod.rs index d0d03dc63..558dcabfb 100644 --- a/crates/arroyo-connectors/src/kafka/source/mod.rs +++ b/crates/arroyo-connectors/src/kafka/source/mod.rs @@ -37,6 +37,24 @@ pub struct KafkaSourceFunc { pub messages_per_second: NonZeroU32, } +#[derive(Debug, Clone, Encode, Decode)] +pub struct KafkaMetadata { + pub offset_id: i64, + pub partition: i32, + pub topic_name: String, +} + +impl KafkaMetadata { + pub fn get(&self, key: &str) -> Option { + match key { + "offset_id" => Some(serde_json::Value::Number(serde_json::Number::from(self.offset_id))), + "partition" => Some(serde_json::Value::Number(serde_json::Number::from(self.partition))), + "topic" => Some(serde_json::Value::String(self.topic_name.clone())), + _ => None, + } + } +} + #[derive(Copy, Clone, Debug, Encode, Decode, PartialEq, PartialOrd)] pub struct KafkaState { partition: i32, @@ -177,8 +195,32 @@ impl KafkaSourceFunc { let timestamp = msg.timestamp().to_millis() .ok_or_else(|| UserError::new("Failed to read timestamp from Kafka record", "The message read from Kafka did not contain a message timestamp"))?; + let metadata = KafkaMetadata { + offset_id: msg.offset(), + partition: msg.partition(), + topic_name: self.topic.clone(), + }; + + let mut deser = serde_json::from_slice::(&v) + .map_err(|e| UserError::new("Failed to deserialize JSON", e.to_string()))?; + + if let Some(schema) = &ctx.out_schema { + for field in schema.schema.fields() { + if let Some(metadata_key) = field.metadata().get("metadata_argument_") { + if let Some(value) = metadata.get(metadata_key) { + if let serde_json::Value::Object(ref mut map) = deser { + map.insert(field.name().to_string(), value.clone()); + } + } else { + error!("Unsupported metadata field: {}", metadata_key); + } + } + } + } + let updated_json = serde_json::to_vec(&deser) + .map_err(|e| UserError::new("Failed to serialize updated JSON", e.to_string()))?; - ctx.deserialize_slice(v, from_millis(timestamp as u64)).await?; + ctx.deserialize_slice(&updated_json, from_millis(timestamp as u64)).await?; if ctx.should_flush() { ctx.flush_buffer().await?; diff --git a/crates/arroyo-planner/src/tables.rs b/crates/arroyo-planner/src/tables.rs index 0c4cc8c5f..d4947d83a 100644 --- a/crates/arroyo-planner/src/tables.rs +++ b/crates/arroyo-planner/src/tables.rs @@ -52,7 +52,7 @@ use datafusion::optimizer::unwrap_cast_in_comparison::UnwrapCastInComparison; use datafusion::optimizer::OptimizerRule; use datafusion::sql::planner::PlannerContext; use datafusion::sql::sqlparser; -use datafusion::sql::sqlparser::ast::Query; +use datafusion::sql::sqlparser::ast::{FunctionArg, FunctionArguments, Query}; use datafusion::{ optimizer::{optimizer::Optimizer, OptimizerContext}, sql::{ @@ -502,19 +502,17 @@ impl Table { columns: &[ColumnDef], schema_provider: &ArroyoSchemaProvider, ) -> Result> { - println!("columns {:?}", columns); let struct_field_pairs = columns .iter() .map(|column| { let name = column.name.value.to_string(); - println!("column name {:?}", name); let (data_type, extension) = convert_data_type(&column.data_type)?; let nullable = !column .options .iter() .any(|option| matches!(option.option, ColumnOption::NotNull)); - let struct_field = ArroyoExtensionType::add_metadata( + let mut struct_field = ArroyoExtensionType::add_metadata( extension, Field::new(name, data_type, nullable), ); @@ -524,11 +522,25 @@ impl Table { generation_expr, .. } = &option.option { - // check if generation_expr is a scalar function and if it is, check if it is a metadata function - if let Some(sqlparser::ast::Expr::Function(sqlparser::ast::Function { name, .. })) = generation_expr { + if let Some(sqlparser::ast::Expr::Function(sqlparser::ast::Function { name, args, .. })) = generation_expr { if name.0.iter().any(|ident| ident.value.to_lowercase() == "metadata") { - println!("Detected metadata function!"); - // TODO: add metadata to the struct field like argument of metadata function. + match args { + FunctionArguments::List(arg_list) => { + match arg_list.args.first() { + Some(FunctionArg::Unnamed(sqlparser::ast::FunctionArgExpr::Expr(sqlparser::ast::Expr::Value(sqlparser::ast::Value::SingleQuotedString(value))))) => { + let mut metadata: HashMap = Default::default(); + metadata.insert("metadata_argument_".to_string(), value.to_string()); + struct_field.set_metadata(metadata); + } + _ => { + DataFusionError::Plan(format!("Unsupported argument format.")); + } + } + } + _ => { + DataFusionError::Plan(format!("Unsupported argument format.")); + } + } return None; } } From cc8faaff677455cc8e2dd464555ab68722c9c91d Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw Date: Thu, 24 Oct 2024 11:25:38 +0530 Subject: [PATCH 06/36] fmt fixes --- .../arroyo-connectors/src/kafka/source/mod.rs | 8 +++- crates/arroyo-planner/src/tables.rs | 42 +++++++++++++++---- 2 files changed, 40 insertions(+), 10 deletions(-) diff --git a/crates/arroyo-connectors/src/kafka/source/mod.rs b/crates/arroyo-connectors/src/kafka/source/mod.rs index 558dcabfb..8ca6a5400 100644 --- a/crates/arroyo-connectors/src/kafka/source/mod.rs +++ b/crates/arroyo-connectors/src/kafka/source/mod.rs @@ -47,8 +47,12 @@ pub struct KafkaMetadata { impl KafkaMetadata { pub fn get(&self, key: &str) -> Option { match key { - "offset_id" => Some(serde_json::Value::Number(serde_json::Number::from(self.offset_id))), - "partition" => Some(serde_json::Value::Number(serde_json::Number::from(self.partition))), + "offset_id" => Some(serde_json::Value::Number(serde_json::Number::from( + self.offset_id, + ))), + "partition" => Some(serde_json::Value::Number(serde_json::Number::from( + self.partition, + ))), "topic" => Some(serde_json::Value::String(self.topic_name.clone())), _ => None, } diff --git a/crates/arroyo-planner/src/tables.rs b/crates/arroyo-planner/src/tables.rs index a8d0ec4f0..dbaeda110 100644 --- a/crates/arroyo-planner/src/tables.rs +++ b/crates/arroyo-planner/src/tables.rs @@ -26,7 +26,8 @@ use datafusion::common::{plan_err, Column, DataFusionError}; use datafusion::execution::context::SessionState; use datafusion::execution::FunctionRegistry; use datafusion::logical_expr::{ - CreateMemoryTable, CreateView, DdlStatement, DmlStatement, Expr, Extension, LogicalPlan, WriteOp + CreateMemoryTable, CreateView, DdlStatement, DmlStatement, Expr, Extension, LogicalPlan, + WriteOp, }; use datafusion::optimizer::common_subexpr_eliminate::CommonSubexprEliminate; use datafusion::optimizer::decorrelate_predicate_subquery::DecorrelatePredicateSubquery; @@ -527,23 +528,48 @@ impl Table { generation_expr, .. } = &option.option { - if let Some(sqlparser::ast::Expr::Function(sqlparser::ast::Function { name, args, .. })) = generation_expr { - if name.0.iter().any(|ident| ident.value.to_lowercase() == "metadata") { + if let Some(sqlparser::ast::Expr::Function(sqlparser::ast::Function { + name, + args, + .. + })) = generation_expr + { + if name + .0 + .iter() + .any(|ident| ident.value.to_lowercase() == "metadata") + { match args { FunctionArguments::List(arg_list) => { match arg_list.args.first() { - Some(FunctionArg::Unnamed(sqlparser::ast::FunctionArgExpr::Expr(sqlparser::ast::Expr::Value(sqlparser::ast::Value::SingleQuotedString(value))))) => { - let mut metadata: HashMap = Default::default(); - metadata.insert("metadata_argument_".to_string(), value.to_string()); + Some(FunctionArg::Unnamed( + sqlparser::ast::FunctionArgExpr::Expr( + sqlparser::ast::Expr::Value( + sqlparser::ast::Value::SingleQuotedString( + value, + ), + ), + ), + )) => { + let mut metadata: HashMap = + Default::default(); + metadata.insert( + "metadata_argument_".to_string(), + value.to_string(), + ); struct_field.set_metadata(metadata); } _ => { - DataFusionError::Plan(format!("Unsupported argument format.")); + DataFusionError::Plan(format!( + "Unsupported argument format." + )); } } } _ => { - DataFusionError::Plan(format!("Unsupported argument format.")); + DataFusionError::Plan(format!( + "Unsupported argument format." + )); } } return None; From 52041c4aee986f17aab271e573ae13976dd12a59 Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw Date: Thu, 24 Oct 2024 12:12:02 +0530 Subject: [PATCH 07/36] fmt errors --- crates/arroyo-planner/src/tables.rs | 55 ++++++++++++++--------------- 1 file changed, 27 insertions(+), 28 deletions(-) diff --git a/crates/arroyo-planner/src/tables.rs b/crates/arroyo-planner/src/tables.rs index dbaeda110..72da24c63 100644 --- a/crates/arroyo-planner/src/tables.rs +++ b/crates/arroyo-planner/src/tables.rs @@ -523,25 +523,24 @@ impl Table { Field::new(name, data_type, nullable), ); - let generating_expression = column.options.iter().find_map(|option| { - if let ColumnOption::Generated { - generation_expr, .. - } = &option.option - { - if let Some(sqlparser::ast::Expr::Function(sqlparser::ast::Function { - name, - args, - .. - })) = generation_expr + let generating_expression = + column.options.iter().find_map(|option| { + if let ColumnOption::Generated { + generation_expr, .. + } = &option.option { - if name - .0 - .iter() - .any(|ident| ident.value.to_lowercase() == "metadata") + if let Some(sqlparser::ast::Expr::Function( + sqlparser::ast::Function { name, args, .. }, + )) = generation_expr { - match args { - FunctionArguments::List(arg_list) => { - match arg_list.args.first() { + if name + .0 + .iter() + .any(|ident| ident.value.to_lowercase() == "metadata") + { + match args { + FunctionArguments::List(arg_list) => { + match arg_list.args.first() { Some(FunctionArg::Unnamed( sqlparser::ast::FunctionArgExpr::Expr( sqlparser::ast::Expr::Value( @@ -565,21 +564,21 @@ impl Table { )); } } + } + _ => { + DataFusionError::Plan(format!( + "Unsupported argument format." + )); + } } - _ => { - DataFusionError::Plan(format!( - "Unsupported argument format." - )); - } + return None; } - return None; } + generation_expr.clone() + } else { + None } - generation_expr.clone() - } else { - None - } - }); + }); Ok((struct_field, generating_expression)) }) .collect::>>()?; From 616b79375ed2019764e52f8329dd474a850fd197 Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw Date: Thu, 24 Oct 2024 12:23:00 +0530 Subject: [PATCH 08/36] fix clippy errors --- crates/arroyo-connectors/src/kafka/source/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/arroyo-connectors/src/kafka/source/mod.rs b/crates/arroyo-connectors/src/kafka/source/mod.rs index 8ca6a5400..6662a8c98 100644 --- a/crates/arroyo-connectors/src/kafka/source/mod.rs +++ b/crates/arroyo-connectors/src/kafka/source/mod.rs @@ -205,7 +205,7 @@ impl KafkaSourceFunc { topic_name: self.topic.clone(), }; - let mut deser = serde_json::from_slice::(&v) + let mut deser = serde_json::from_slice::(v) .map_err(|e| UserError::new("Failed to deserialize JSON", e.to_string()))?; if let Some(schema) = &ctx.out_schema { From bdcad6060c57e3629f85cc4154005fe735963ddd Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw Date: Thu, 24 Oct 2024 12:43:15 +0530 Subject: [PATCH 09/36] fix clippy errors --- crates/arroyo-planner/src/tables.rs | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/crates/arroyo-planner/src/tables.rs b/crates/arroyo-planner/src/tables.rs index 72da24c63..0bae4048e 100644 --- a/crates/arroyo-planner/src/tables.rs +++ b/crates/arroyo-planner/src/tables.rs @@ -559,16 +559,16 @@ impl Table { struct_field.set_metadata(metadata); } _ => { - DataFusionError::Plan(format!( - "Unsupported argument format." - )); + DataFusionError::Plan( + "Unsupported argument format.".to_string() + ); } } } _ => { - DataFusionError::Plan(format!( - "Unsupported argument format." - )); + DataFusionError::Plan( + "Unsupported argument format.".to_string() + ); } } return None; From 74c4043dd64094f9667ea12114ff8c9878edc194 Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw Date: Thu, 24 Oct 2024 12:52:25 +0530 Subject: [PATCH 10/36] fix clippy --- crates/arroyo-planner/src/tables.rs | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/crates/arroyo-planner/src/tables.rs b/crates/arroyo-planner/src/tables.rs index 0bae4048e..218848a33 100644 --- a/crates/arroyo-planner/src/tables.rs +++ b/crates/arroyo-planner/src/tables.rs @@ -559,16 +559,12 @@ impl Table { struct_field.set_metadata(metadata); } _ => { - DataFusionError::Plan( - "Unsupported argument format.".to_string() - ); + "Unsupported argument format.".to_string(); } } } _ => { - DataFusionError::Plan( - "Unsupported argument format.".to_string() - ); + "Unsupported argument format.".to_string(); } } return None; From e342ed7ed5a0a92b7d978be0e7fbc98e655a3f6e Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw Date: Thu, 24 Oct 2024 12:54:22 +0530 Subject: [PATCH 11/36] fmt fixes --- crates/arroyo-planner/src/tables.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/arroyo-planner/src/tables.rs b/crates/arroyo-planner/src/tables.rs index 218848a33..31bd36579 100644 --- a/crates/arroyo-planner/src/tables.rs +++ b/crates/arroyo-planner/src/tables.rs @@ -564,7 +564,7 @@ impl Table { } } _ => { - "Unsupported argument format.".to_string(); + "Unsupported argument format.".to_string(); } } return None; From 167d3ac285931fb0102dcb01d48c399a8cc8dbc5 Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw Date: Thu, 24 Oct 2024 13:16:16 +0530 Subject: [PATCH 12/36] adding a check for json only --- .../arroyo-connectors/src/kafka/source/mod.rs | 48 +++++++++++-------- 1 file changed, 27 insertions(+), 21 deletions(-) diff --git a/crates/arroyo-connectors/src/kafka/source/mod.rs b/crates/arroyo-connectors/src/kafka/source/mod.rs index 6662a8c98..ea7186c72 100644 --- a/crates/arroyo-connectors/src/kafka/source/mod.rs +++ b/crates/arroyo-connectors/src/kafka/source/mod.rs @@ -199,32 +199,38 @@ impl KafkaSourceFunc { let timestamp = msg.timestamp().to_millis() .ok_or_else(|| UserError::new("Failed to read timestamp from Kafka record", "The message read from Kafka did not contain a message timestamp"))?; - let metadata = KafkaMetadata { - offset_id: msg.offset(), - partition: msg.partition(), - topic_name: self.topic.clone(), - }; - - let mut deser = serde_json::from_slice::(v) - .map_err(|e| UserError::new("Failed to deserialize JSON", e.to_string()))?; - - if let Some(schema) = &ctx.out_schema { - for field in schema.schema.fields() { - if let Some(metadata_key) = field.metadata().get("metadata_argument_") { - if let Some(value) = metadata.get(metadata_key) { - if let serde_json::Value::Object(ref mut map) = deser { - map.insert(field.name().to_string(), value.clone()); + + if let Format::Json(_) = self.format { + let metadata = KafkaMetadata { + offset_id: msg.offset(), + partition: msg.partition(), + topic_name: self.topic.clone(), + }; + + let mut deser = serde_json::from_slice::(v) + .map_err(|e| UserError::new("Failed to deserialize JSON", e.to_string()))?; + + if let Some(schema) = &ctx.out_schema { + for field in schema.schema.fields() { + if let Some(metadata_key) = field.metadata().get("metadata_argument_") { + if let Some(value) = metadata.get(metadata_key) { + if let serde_json::Value::Object(ref mut map) = deser { + map.insert(field.name().to_string(), value.clone()); + } + } else { + error!("Unsupported metadata field: {}", metadata_key); } - } else { - error!("Unsupported metadata field: {}", metadata_key); } } } - } - let updated_json = serde_json::to_vec(&deser) - .map_err(|e| UserError::new("Failed to serialize updated JSON", e.to_string()))?; + let updated_json = serde_json::to_vec(&deser) + .map_err(|e| UserError::new("Failed to serialize updated JSON", e.to_string()))?; - ctx.deserialize_slice(&updated_json, from_millis(timestamp as u64)).await?; + + ctx.deserialize_slice(&updated_json, from_millis(timestamp as u64)).await?; + } else { + ctx.deserialize_slice(v, from_millis(timestamp as u64)).await?; + } if ctx.should_flush() { ctx.flush_buffer().await?; From ec5c023af57ede11e824748b8e7f9d747b082f2c Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw Date: Fri, 25 Oct 2024 14:17:41 +0530 Subject: [PATCH 13/36] removing double serialization and adding values using array builders --- .../src/filesystem/source.rs | 2 +- crates/arroyo-connectors/src/fluvio/source.rs | 2 +- crates/arroyo-connectors/src/kafka/mod.rs | 21 ++- .../arroyo-connectors/src/kafka/source/mod.rs | 56 +------ .../arroyo-connectors/src/kinesis/source.rs | 8 +- .../arroyo-connectors/src/mqtt/source/mod.rs | 2 +- .../arroyo-connectors/src/nats/source/mod.rs | 4 +- .../src/polling_http/operator.rs | 2 +- .../src/single_file/source.rs | 10 +- crates/arroyo-connectors/src/sse/operator.rs | 2 +- .../src/websocket/operator.rs | 3 +- crates/arroyo-formats/src/avro/de.rs | 7 +- crates/arroyo-formats/src/de.rs | 157 +++++++++++++++++- crates/arroyo-operator/src/context.rs | 2 + 14 files changed, 202 insertions(+), 76 deletions(-) diff --git a/crates/arroyo-connectors/src/filesystem/source.rs b/crates/arroyo-connectors/src/filesystem/source.rs index c7abd4f35..b4e48bfbc 100644 --- a/crates/arroyo-connectors/src/filesystem/source.rs +++ b/crates/arroyo-connectors/src/filesystem/source.rs @@ -367,7 +367,7 @@ impl FileSystemSourceFunc { line = line_reader.next() => { match line.transpose()? { Some(line) => { - ctx.deserialize_slice(line.as_bytes(), SystemTime::now()).await?; + ctx.deserialize_slice(line.as_bytes(), SystemTime::now(), (false, 0, 0, "".to_string())).await?; records_read += 1; if ctx.should_flush() { ctx.flush_buffer().await?; diff --git a/crates/arroyo-connectors/src/fluvio/source.rs b/crates/arroyo-connectors/src/fluvio/source.rs index e0315c6f1..e071b03c3 100644 --- a/crates/arroyo-connectors/src/fluvio/source.rs +++ b/crates/arroyo-connectors/src/fluvio/source.rs @@ -166,7 +166,7 @@ impl FluvioSourceFunc { match message { Some((_, Ok(msg))) => { let timestamp = from_millis(msg.timestamp().max(0) as u64); - ctx.deserialize_slice(msg.value(), timestamp).await?; + ctx.deserialize_slice(msg.value(), timestamp, (false, 0, 0, "".to_string())).await?; if ctx.should_flush() { ctx.flush_buffer().await?; diff --git a/crates/arroyo-connectors/src/kafka/mod.rs b/crates/arroyo-connectors/src/kafka/mod.rs index 9d3d97fd4..8f98e9ee0 100644 --- a/crates/arroyo-connectors/src/kafka/mod.rs +++ b/crates/arroyo-connectors/src/kafka/mod.rs @@ -622,7 +622,12 @@ impl KafkaTester { let mut builders = aschema.builders(); let mut error = deserializer - .deserialize_slice(&mut builders, &msg, SystemTime::now()) + .deserialize_slice( + &mut builders, + &msg, + SystemTime::now(), + (false, 0, 0, "".to_string()), + ) .await .into_iter() .next(); @@ -644,7 +649,12 @@ impl KafkaTester { let mut builders = aschema.builders(); let mut error = deserializer - .deserialize_slice(&mut builders, &msg, SystemTime::now()) + .deserialize_slice( + &mut builders, + &msg, + SystemTime::now(), + (false, 0, 0, "".to_string()), + ) .await .into_iter() .next(); @@ -678,7 +688,12 @@ impl KafkaTester { let mut builders = aschema.builders(); let mut error = deserializer - .deserialize_slice(&mut builders, &msg, SystemTime::now()) + .deserialize_slice( + &mut builders, + &msg, + SystemTime::now(), + (false, 0, 0, "".to_string()), + ) .await .into_iter() .next(); diff --git a/crates/arroyo-connectors/src/kafka/source/mod.rs b/crates/arroyo-connectors/src/kafka/source/mod.rs index ea7186c72..00f19152d 100644 --- a/crates/arroyo-connectors/src/kafka/source/mod.rs +++ b/crates/arroyo-connectors/src/kafka/source/mod.rs @@ -37,28 +37,6 @@ pub struct KafkaSourceFunc { pub messages_per_second: NonZeroU32, } -#[derive(Debug, Clone, Encode, Decode)] -pub struct KafkaMetadata { - pub offset_id: i64, - pub partition: i32, - pub topic_name: String, -} - -impl KafkaMetadata { - pub fn get(&self, key: &str) -> Option { - match key { - "offset_id" => Some(serde_json::Value::Number(serde_json::Number::from( - self.offset_id, - ))), - "partition" => Some(serde_json::Value::Number(serde_json::Number::from( - self.partition, - ))), - "topic" => Some(serde_json::Value::String(self.topic_name.clone())), - _ => None, - } - } -} - #[derive(Copy, Clone, Debug, Encode, Decode, PartialEq, PartialOrd)] pub struct KafkaState { partition: i32, @@ -200,37 +178,9 @@ impl KafkaSourceFunc { .ok_or_else(|| UserError::new("Failed to read timestamp from Kafka record", "The message read from Kafka did not contain a message timestamp"))?; - if let Format::Json(_) = self.format { - let metadata = KafkaMetadata { - offset_id: msg.offset(), - partition: msg.partition(), - topic_name: self.topic.clone(), - }; - - let mut deser = serde_json::from_slice::(v) - .map_err(|e| UserError::new("Failed to deserialize JSON", e.to_string()))?; - - if let Some(schema) = &ctx.out_schema { - for field in schema.schema.fields() { - if let Some(metadata_key) = field.metadata().get("metadata_argument_") { - if let Some(value) = metadata.get(metadata_key) { - if let serde_json::Value::Object(ref mut map) = deser { - map.insert(field.name().to_string(), value.clone()); - } - } else { - error!("Unsupported metadata field: {}", metadata_key); - } - } - } - } - let updated_json = serde_json::to_vec(&deser) - .map_err(|e| UserError::new("Failed to serialize updated JSON", e.to_string()))?; - - - ctx.deserialize_slice(&updated_json, from_millis(timestamp as u64)).await?; - } else { - ctx.deserialize_slice(v, from_millis(timestamp as u64)).await?; - } + let metadata_enabled = true; + let kafka_metadata = (metadata_enabled, msg.offset(), msg.partition(), self.topic.clone()); + ctx.deserialize_slice(v, from_millis(timestamp as u64), kafka_metadata).await?; if ctx.should_flush() { ctx.flush_buffer().await?; diff --git a/crates/arroyo-connectors/src/kinesis/source.rs b/crates/arroyo-connectors/src/kinesis/source.rs index 54d9ab150..7587a10eb 100644 --- a/crates/arroyo-connectors/src/kinesis/source.rs +++ b/crates/arroyo-connectors/src/kinesis/source.rs @@ -435,8 +435,12 @@ impl KinesisSourceFunc { let data = record.data.into_inner(); let timestamp = record.approximate_arrival_timestamp.unwrap(); - ctx.deserialize_slice(&data, from_nanos(timestamp.as_nanos() as u128)) - .await?; + ctx.deserialize_slice( + &data, + from_nanos(timestamp.as_nanos() as u128), + (false, 0, 0, "".to_string()), + ) + .await?; if ctx.should_flush() { ctx.flush_buffer().await? diff --git a/crates/arroyo-connectors/src/mqtt/source/mod.rs b/crates/arroyo-connectors/src/mqtt/source/mod.rs index d9646d618..39d575d3b 100644 --- a/crates/arroyo-connectors/src/mqtt/source/mod.rs +++ b/crates/arroyo-connectors/src/mqtt/source/mod.rs @@ -143,7 +143,7 @@ impl MqttSourceFunc { event = eventloop.poll() => { match event { Ok(MqttEvent::Incoming(Incoming::Publish(p))) => { - ctx.deserialize_slice(&p.payload, SystemTime::now()).await?; + ctx.deserialize_slice(&p.payload, SystemTime::now(), (false, 0, 0, "".to_string())).await?; rate_limiter.until_ready().await; } Ok(MqttEvent::Outgoing(Outgoing::Subscribe(_))) => { diff --git a/crates/arroyo-connectors/src/nats/source/mod.rs b/crates/arroyo-connectors/src/nats/source/mod.rs index a550efc93..d0c36527b 100644 --- a/crates/arroyo-connectors/src/nats/source/mod.rs +++ b/crates/arroyo-connectors/src/nats/source/mod.rs @@ -367,7 +367,7 @@ impl NatsSourceFunc { let message_info = msg.info().expect("Couldn't get message information"); let timestamp = message_info.published.into() ; - ctx.deserialize_slice(payload, timestamp).await?; + ctx.deserialize_slice(payload, timestamp, (false, 0, 0, "".to_string())).await?; debug!("---------------------------------------------->"); debug!( @@ -493,7 +493,7 @@ impl NatsSourceFunc { Some(msg) => { let payload = msg.payload.as_ref(); let timestamp = SystemTime::now(); - ctx.deserialize_slice(payload, timestamp).await?; + ctx.deserialize_slice(payload, timestamp, (false, 0, 0, "".to_string())).await?; if ctx.should_flush() { ctx.flush_buffer().await?; } diff --git a/crates/arroyo-connectors/src/polling_http/operator.rs b/crates/arroyo-connectors/src/polling_http/operator.rs index c99ebad4f..5a3bb37b6 100644 --- a/crates/arroyo-connectors/src/polling_http/operator.rs +++ b/crates/arroyo-connectors/src/polling_http/operator.rs @@ -215,7 +215,7 @@ impl PollingHttpSourceFunc { continue; } - ctx.deserialize_slice(&buf, SystemTime::now()).await?; + ctx.deserialize_slice(&buf, SystemTime::now(), (false, 0, 0, "".to_string())).await?; if ctx.should_flush() { ctx.flush_buffer().await?; diff --git a/crates/arroyo-connectors/src/single_file/source.rs b/crates/arroyo-connectors/src/single_file/source.rs index 125a796b8..e8c262926 100644 --- a/crates/arroyo-connectors/src/single_file/source.rs +++ b/crates/arroyo-connectors/src/single_file/source.rs @@ -52,9 +52,13 @@ impl SingleFileSourceFunc { continue; } - ctx.deserialize_slice(s.as_bytes(), SystemTime::now()) - .await - .unwrap(); + ctx.deserialize_slice( + s.as_bytes(), + SystemTime::now(), + (false, 0, 0, "".to_string()), + ) + .await + .unwrap(); if ctx.should_flush() { ctx.flush_buffer().await.unwrap(); } diff --git a/crates/arroyo-connectors/src/sse/operator.rs b/crates/arroyo-connectors/src/sse/operator.rs index ed691f75c..e562bd980 100644 --- a/crates/arroyo-connectors/src/sse/operator.rs +++ b/crates/arroyo-connectors/src/sse/operator.rs @@ -172,7 +172,7 @@ impl SSESourceFunc { if events.is_empty() || events.contains(&event.event_type) { ctx.deserialize_slice( - event.data.as_bytes(), SystemTime::now()).await?; + event.data.as_bytes(), SystemTime::now(), (false, 0, 0, "".to_string())).await?; if ctx.should_flush() { ctx.flush_buffer().await?; diff --git a/crates/arroyo-connectors/src/websocket/operator.rs b/crates/arroyo-connectors/src/websocket/operator.rs index 74a746de9..f4c96dd68 100644 --- a/crates/arroyo-connectors/src/websocket/operator.rs +++ b/crates/arroyo-connectors/src/websocket/operator.rs @@ -122,7 +122,8 @@ impl WebsocketSourceFunc { msg: &[u8], ctx: &mut ArrowContext, ) -> Result<(), UserError> { - ctx.deserialize_slice(msg, SystemTime::now()).await?; + ctx.deserialize_slice(msg, SystemTime::now(), (false, 0, 0, "".to_string())) + .await?; if ctx.should_flush() { ctx.flush_buffer().await?; diff --git a/crates/arroyo-formats/src/avro/de.rs b/crates/arroyo-formats/src/avro/de.rs index a771699a9..7967dc8d0 100644 --- a/crates/arroyo-formats/src/avro/de.rs +++ b/crates/arroyo-formats/src/avro/de.rs @@ -281,7 +281,12 @@ mod tests { deserializer_with_schema(format.clone(), writer_schema); let errors = deserializer - .deserialize_slice(&mut builders, message, SystemTime::now()) + .deserialize_slice( + &mut builders, + message, + SystemTime::now(), + (false, 0, 0, "".to_string()), + ) .await; assert_eq!(errors, vec![]); diff --git a/crates/arroyo-formats/src/de.rs b/crates/arroyo-formats/src/de.rs index 3b2edba74..7c70b6336 100644 --- a/crates/arroyo-formats/src/de.rs +++ b/crates/arroyo-formats/src/de.rs @@ -1,6 +1,7 @@ use crate::avro::de; use crate::proto::schema::get_pool; use crate::{proto, should_flush}; +use arrow::array::{Int32Builder, Int64Builder}; use arrow::compute::kernels; use arrow_array::builder::{ ArrayBuilder, GenericByteBuilder, StringBuilder, TimestampNanosecondBuilder, @@ -82,6 +83,7 @@ pub struct ArrowDeserializer { schema_registry: Arc>>, proto_pool: DescriptorPool, schema_resolver: Arc, + kafka_metadata_builder: Option<(Int64Builder, Int32Builder, StringBuilder)>, } impl ArrowDeserializer { @@ -158,6 +160,7 @@ impl ArrowDeserializer { proto_pool, buffered_count: 0, buffered_since: Instant::now(), + kafka_metadata_builder: None, } } @@ -166,11 +169,12 @@ impl ArrowDeserializer { buffer: &mut [Box], msg: &[u8], timestamp: SystemTime, + kafka_metadata: (bool, i64, i32, String), ) -> Vec { match &*self.format { Format::Avro(_) => self.deserialize_slice_avro(buffer, msg, timestamp).await, _ => FramingIterator::new(self.framing.clone(), msg) - .map(|t| self.deserialize_single(buffer, t, timestamp)) + .map(|t| self.deserialize_single(buffer, t, timestamp, kafka_metadata.clone())) .filter_map(|t| t.err()) .collect(), } @@ -195,6 +199,31 @@ impl ArrowDeserializer { .map(|batch| { let mut columns = batch.columns().to_vec(); columns.insert(self.schema.timestamp_index, Arc::new(timestamp.finish())); + + if let Some((offset_builder, partition_builder, topic_builder)) = + &mut self.kafka_metadata_builder + { + if let Some((topic_idx, _)) = + self.schema.schema.column_with_name("topic") + { + columns.remove(topic_idx); + columns.insert(topic_idx, Arc::new(topic_builder.finish())); + } + + if let Some((partition_idx, _)) = + self.schema.schema.column_with_name("partition") + { + columns.remove(partition_idx); + columns.insert(partition_idx, Arc::new(partition_builder.finish())); + } + + if let Some((offset_idx, _)) = + self.schema.schema.column_with_name("offset") + { + columns.remove(offset_idx); + columns.insert(offset_idx, Arc::new(offset_builder.finish())); + } + } RecordBatch::try_new(self.schema.schema.clone(), columns).unwrap() }), ), @@ -225,6 +254,7 @@ impl ArrowDeserializer { buffer: &mut [Box], msg: &[u8], timestamp: SystemTime, + kafka_metadata: (bool, i64, i32, String), ) -> Result<(), SourceError> { match &*self.format { Format::RawString(_) @@ -233,10 +263,28 @@ impl ArrowDeserializer { }) => { self.deserialize_raw_string(buffer, msg); add_timestamp(buffer, self.schema.timestamp_index, timestamp); + if kafka_metadata.0 { + add_kafka_metadata( + buffer, + &self.schema, + &kafka_metadata.3, + kafka_metadata.2, + kafka_metadata.1, + ); + } } Format::RawBytes(_) => { self.deserialize_raw_bytes(buffer, msg); add_timestamp(buffer, self.schema.timestamp_index, timestamp); + if kafka_metadata.0 { + add_kafka_metadata( + buffer, + &self.schema, + &kafka_metadata.3, + kafka_metadata.2, + kafka_metadata.1, + ); + } } Format::Json(json) => { let msg = if json.confluent_schema_registry { @@ -249,10 +297,29 @@ impl ArrowDeserializer { panic!("json decoder not initialized"); }; + if kafka_metadata.0 { + self.kafka_metadata_builder.get_or_insert_with(|| { + ( + Int64Builder::new(), + Int32Builder::new(), + StringBuilder::new(), + ) + }); + } + decoder .decode(msg) .map_err(|e| SourceError::bad_data(format!("invalid JSON: {:?}", e)))?; timestamp_builder.append_value(to_nanos(timestamp) as i64); + if kafka_metadata.0 { + if let Some((offset_builder, partition_builder, topic_builder)) = + &mut self.kafka_metadata_builder + { + offset_builder.append_value(kafka_metadata.1); + partition_builder.append_value(kafka_metadata.2); + topic_builder.append_value(kafka_metadata.3.clone()); + } + } self.buffered_count += 1; } Format::Protobuf(proto) => { @@ -269,6 +336,15 @@ impl ArrowDeserializer { .decode(json.to_string().as_bytes()) .map_err(|e| SourceError::bad_data(format!("invalid JSON: {:?}", e)))?; timestamp_builder.append_value(to_nanos(timestamp) as i64); + if kafka_metadata.0 { + add_kafka_metadata( + buffer, + &self.schema, + &kafka_metadata.3, + kafka_metadata.2, + kafka_metadata.1, + ); + } self.buffered_count += 1; } } @@ -400,6 +476,66 @@ pub(crate) fn add_timestamp( .append_value(to_nanos(timestamp) as i64); } +pub(crate) fn add_kafka_metadata_partition( + builder: &mut [Box], + idx: usize, + partition: i32, +) { + builder[idx] + .as_any_mut() + .downcast_mut::() + .expect("kafka_metadata.partition column has incorrect type") + .append_value(partition); +} + +pub(crate) fn add_kafka_metadata_offset( + builder: &mut [Box], + idx: usize, + offset: i64, +) { + builder[idx] + .as_any_mut() + .downcast_mut::() + .expect("kafka_metadata.offset column has incorrect type") + .append_value(offset); +} + +pub(crate) fn add_kafka_metadata_topic( + builder: &mut [Box], + idx: usize, + topic: &str, +) { + builder[idx] + .as_any_mut() + .downcast_mut::() + .expect("kafka_metadata.topic column has incorrect type") + .append_value(topic) +} + +pub(crate) fn add_kafka_metadata( + builder: &mut [Box], + schema: &ArroyoSchema, + topic: &str, + partition: i32, + offset: i64, +) { + let (topic_idx, _) = schema + .schema + .column_with_name("topic") + .expect("No column named 'topic' in the schema"); + let (partition_idx, _) = schema + .schema + .column_with_name("partition") + .expect("No column named 'partition' in the schema"); + let (offset_idx, _) = schema + .schema + .column_with_name("offset") + .expect("No column named 'offset' in the schema"); + add_kafka_metadata_topic(builder, topic_idx, topic); + add_kafka_metadata_partition(builder, partition_idx, partition); + add_kafka_metadata_offset(builder, offset_idx, offset); +} + #[cfg(test)] mod tests { use crate::de::{ArrowDeserializer, FramingIterator}; @@ -534,7 +670,8 @@ mod tests { .deserialize_slice( &mut arrays[..], json!({ "x": 5 }).to_string().as_bytes(), - now + now, + (false, 0, 0, "".to_string()) ) .await, vec![] @@ -544,7 +681,8 @@ mod tests { .deserialize_slice( &mut arrays[..], json!({ "x": "hello" }).to_string().as_bytes(), - now + now, + (false, 0, 0, "".to_string()) ) .await, vec![] @@ -570,7 +708,8 @@ mod tests { .deserialize_slice( &mut arrays[..], json!({ "x": 5 }).to_string().as_bytes(), - SystemTime::now() + SystemTime::now(), + (false, 0, 0, "".to_string()) ) .await, vec![] @@ -580,7 +719,8 @@ mod tests { .deserialize_slice( &mut arrays[..], json!({ "x": "hello" }).to_string().as_bytes(), - SystemTime::now() + SystemTime::now(), + (false, 0, 0, "".to_string()) ) .await, vec![] @@ -619,7 +759,12 @@ mod tests { let time = SystemTime::now(); let result = deserializer - .deserialize_slice(&mut arrays, &[0, 1, 2, 3, 4, 5], time) + .deserialize_slice( + &mut arrays, + &[0, 1, 2, 3, 4, 5], + time, + (false, 0, 0, "".to_string()), + ) .await; assert!(result.is_empty()); diff --git a/crates/arroyo-operator/src/context.rs b/crates/arroyo-operator/src/context.rs index d0009a230..4e1668d81 100644 --- a/crates/arroyo-operator/src/context.rs +++ b/crates/arroyo-operator/src/context.rs @@ -670,6 +670,7 @@ impl ArrowContext { &mut self, msg: &[u8], time: SystemTime, + kafka_metadata: (bool, i64, i32, String), ) -> Result<(), UserError> { let deserializer = self .deserializer @@ -688,6 +689,7 @@ impl ArrowContext { &mut self.buffer.as_mut().expect("no out schema").buffer, msg, time, + kafka_metadata, ) .await; self.collect_source_errors(errors).await?; From 4f3946931d3752de0e724ea51b63b526a7eca8e5 Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw Date: Fri, 25 Oct 2024 18:13:10 +0530 Subject: [PATCH 14/36] taking kafka metadata from configs --- crates/arroyo-api/src/pipelines.rs | 2 + crates/arroyo-connectors/src/blackhole/mod.rs | 16 ++- crates/arroyo-connectors/src/confluent/mod.rs | 8 +- .../arroyo-connectors/src/filesystem/delta.rs | 8 +- .../arroyo-connectors/src/filesystem/mod.rs | 10 +- .../src/filesystem/source.rs | 2 +- crates/arroyo-connectors/src/fluvio/mod.rs | 8 +- crates/arroyo-connectors/src/fluvio/source.rs | 2 +- crates/arroyo-connectors/src/impulse/mod.rs | 8 ++ crates/arroyo-connectors/src/kafka/mod.rs | 22 +++- .../arroyo-connectors/src/kafka/source/mod.rs | 8 +- .../src/kafka/source/test.rs | 2 + crates/arroyo-connectors/src/kinesis/mod.rs | 8 +- .../arroyo-connectors/src/kinesis/source.rs | 1 + crates/arroyo-connectors/src/mqtt/mod.rs | 8 +- .../arroyo-connectors/src/mqtt/source/mod.rs | 2 +- crates/arroyo-connectors/src/nats/mod.rs | 8 +- .../arroyo-connectors/src/nats/source/mod.rs | 4 +- crates/arroyo-connectors/src/nexmark/mod.rs | 8 ++ .../arroyo-connectors/src/polling_http/mod.rs | 8 ++ .../src/polling_http/operator.rs | 2 +- crates/arroyo-connectors/src/preview/mod.rs | 6 + crates/arroyo-connectors/src/redis/mod.rs | 8 ++ .../arroyo-connectors/src/single_file/mod.rs | 8 ++ .../src/single_file/source.rs | 1 + crates/arroyo-connectors/src/sse/mod.rs | 8 ++ crates/arroyo-connectors/src/sse/operator.rs | 2 +- crates/arroyo-connectors/src/stdout/mod.rs | 16 ++- crates/arroyo-connectors/src/webhook/mod.rs | 8 +- crates/arroyo-connectors/src/websocket/mod.rs | 8 ++ .../src/websocket/operator.rs | 2 +- crates/arroyo-formats/src/avro/de.rs | 1 + crates/arroyo-formats/src/de.rs | 113 +++++++++++------- crates/arroyo-operator/src/connector.rs | 16 ++- crates/arroyo-operator/src/context.rs | 2 + crates/arroyo-planner/src/tables.rs | 20 ++-- crates/arroyo-planner/src/test/mod.rs | 2 + crates/arroyo-rpc/src/lib.rs | 4 + 38 files changed, 294 insertions(+), 76 deletions(-) diff --git a/crates/arroyo-api/src/pipelines.rs b/crates/arroyo-api/src/pipelines.rs index 52ce19b55..a42d30ba7 100644 --- a/crates/arroyo-api/src/pipelines.rs +++ b/crates/arroyo-api/src/pipelines.rs @@ -159,6 +159,8 @@ async fn compile_sql<'a>( .unwrap_or(json!({})), &table.config, Some(&table.schema), + Some(true), + None, ) .map_err(log_and_map)?; diff --git a/crates/arroyo-connectors/src/blackhole/mod.rs b/crates/arroyo-connectors/src/blackhole/mod.rs index bed4f4726..f9b8f16e3 100644 --- a/crates/arroyo-connectors/src/blackhole/mod.rs +++ b/crates/arroyo-connectors/src/blackhole/mod.rs @@ -78,8 +78,18 @@ impl Connector for BlackholeConnector { _options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, + _enable_metadata: Option, + _metadata_fields: Option>, ) -> anyhow::Result { - self.from_config(None, name, EmptyConfig {}, EmptyConfig {}, schema) + self.from_config( + None, + name, + EmptyConfig {}, + EmptyConfig {}, + schema, + None, + None, + ) } fn from_config( @@ -89,6 +99,8 @@ impl Connector for BlackholeConnector { config: Self::ProfileT, table: Self::TableT, s: Option<&ConnectionSchema>, + _enable_metadata: Option, + _metadata_fields: Option>, ) -> anyhow::Result { let description = "Blackhole".to_string(); @@ -99,6 +111,8 @@ impl Connector for BlackholeConnector { format: None, bad_data: None, framing: None, + enable_metadata: None, + metadata_fields: None, }; Ok(Connection { diff --git a/crates/arroyo-connectors/src/confluent/mod.rs b/crates/arroyo-connectors/src/confluent/mod.rs index c694b9ef7..fdd215804 100644 --- a/crates/arroyo-connectors/src/confluent/mod.rs +++ b/crates/arroyo-connectors/src/confluent/mod.rs @@ -161,6 +161,8 @@ impl Connector for ConfluentConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, profile: Option<&ConnectionProfile>, + _enable_metadata: Option, + _metadata_fields: Option>, ) -> anyhow::Result { let connection = profile .map(|p| { @@ -172,7 +174,7 @@ impl Connector for ConfluentConnector { let table = KafkaConnector::table_from_options(options)?; - self.from_config(None, name, connection, table, schema) + self.from_config(None, name, connection, table, schema, None, None) } fn from_config( @@ -182,11 +184,13 @@ impl Connector for ConfluentConnector { config: Self::ProfileT, mut table: Self::TableT, schema: Option<&ConnectionSchema>, + _enable_metadata: Option, + _metadata_fields: Option>, ) -> anyhow::Result { table .client_configs .insert("client.id".to_string(), CLIENT_ID.to_string()); - KafkaConnector {}.from_config(id, name, config.into(), table, schema) + KafkaConnector {}.from_config(id, name, config.into(), table, schema, None, None) } fn make_operator( diff --git a/crates/arroyo-connectors/src/filesystem/delta.rs b/crates/arroyo-connectors/src/filesystem/delta.rs index fc384668f..c2c98a881 100644 --- a/crates/arroyo-connectors/src/filesystem/delta.rs +++ b/crates/arroyo-connectors/src/filesystem/delta.rs @@ -77,6 +77,8 @@ impl Connector for DeltaLakeConnector { config: Self::ProfileT, table: Self::TableT, schema: Option<&ConnectionSchema>, + _enable_metadata: Option, + _metadata_fields: Option>, ) -> anyhow::Result { let TableType::Sink { write_path, @@ -123,6 +125,8 @@ impl Connector for DeltaLakeConnector { format: Some(format), bad_data: schema.bad_data.clone(), framing: schema.framing.clone(), + enable_metadata: None, + metadata_fields: None, }; Ok(Connection { @@ -142,10 +146,12 @@ impl Connector for DeltaLakeConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, + _enable_metadata: Option, + _metadata_fields: Option>, ) -> anyhow::Result { let table = file_system_sink_from_options(options, schema, CommitStyle::DeltaLake)?; - self.from_config(None, name, EmptyConfig {}, table, schema) + self.from_config(None, name, EmptyConfig {}, table, schema, None, None) } fn make_operator( diff --git a/crates/arroyo-connectors/src/filesystem/mod.rs b/crates/arroyo-connectors/src/filesystem/mod.rs index c81e24c35..626ea3032 100644 --- a/crates/arroyo-connectors/src/filesystem/mod.rs +++ b/crates/arroyo-connectors/src/filesystem/mod.rs @@ -114,6 +114,8 @@ impl Connector for FileSystemConnector { config: Self::ProfileT, table: Self::TableT, schema: Option<&ConnectionSchema>, + _enable_metadata: Option, + _metadata_fields: Option>, ) -> anyhow::Result { let (description, connection_type) = match table.table_type { TableType::Source { .. } => ("FileSystem".to_string(), ConnectionType::Source), @@ -168,6 +170,8 @@ impl Connector for FileSystemConnector { format: Some(format), bad_data: schema.bad_data.clone(), framing: schema.framing.clone(), + enable_metadata: None, + metadata_fields: None, }; Ok(Connection { @@ -187,6 +191,8 @@ impl Connector for FileSystemConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, + _enable_metadata: Option, + _metadata_fields: Option>, ) -> anyhow::Result { match options.remove("type") { Some(t) if t == "source" => { @@ -210,12 +216,14 @@ impl Connector for FileSystemConnector { }, }, schema, + None, + None, ) } Some(t) if t == "sink" => { let table = file_system_sink_from_options(options, schema, CommitStyle::Direct)?; - self.from_config(None, name, EmptyConfig {}, table, schema) + self.from_config(None, name, EmptyConfig {}, table, schema, None, None) } Some(t) => bail!("unknown type: {}", t), None => bail!("must have type set"), diff --git a/crates/arroyo-connectors/src/filesystem/source.rs b/crates/arroyo-connectors/src/filesystem/source.rs index b4e48bfbc..48f547b23 100644 --- a/crates/arroyo-connectors/src/filesystem/source.rs +++ b/crates/arroyo-connectors/src/filesystem/source.rs @@ -367,7 +367,7 @@ impl FileSystemSourceFunc { line = line_reader.next() => { match line.transpose()? { Some(line) => { - ctx.deserialize_slice(line.as_bytes(), SystemTime::now(), (false, 0, 0, "".to_string())).await?; + ctx.deserialize_slice(line.as_bytes(), SystemTime::now(), (false, 0, 0, "".to_string()), None).await?; records_read += 1; if ctx.should_flush() { ctx.flush_buffer().await?; diff --git a/crates/arroyo-connectors/src/fluvio/mod.rs b/crates/arroyo-connectors/src/fluvio/mod.rs index c5b2b5d09..006f1ebf2 100644 --- a/crates/arroyo-connectors/src/fluvio/mod.rs +++ b/crates/arroyo-connectors/src/fluvio/mod.rs @@ -88,6 +88,8 @@ impl Connector for FluvioConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, + _enable_metadata: Option, + _metadata_fields: Option>, ) -> anyhow::Result { let endpoint = options.remove("endpoint"); let topic = pull_opt("topic", options)?; @@ -116,7 +118,7 @@ impl Connector for FluvioConnector { type_: table_type, }; - Self::from_config(self, None, name, EmptyConfig {}, table, schema) + Self::from_config(self, None, name, EmptyConfig {}, table, schema, None, None) } fn from_config( @@ -126,6 +128,8 @@ impl Connector for FluvioConnector { config: EmptyConfig, table: FluvioTable, schema: Option<&ConnectionSchema>, + _enable_metadata: Option, + _metadata_fields: Option>, ) -> anyhow::Result { let (typ, desc) = match table.type_ { TableType::Source { .. } => ( @@ -154,6 +158,8 @@ impl Connector for FluvioConnector { format: Some(format), bad_data: schema.bad_data.clone(), framing: schema.framing.clone(), + enable_metadata: None, + metadata_fields: None, }; Ok(Connection { diff --git a/crates/arroyo-connectors/src/fluvio/source.rs b/crates/arroyo-connectors/src/fluvio/source.rs index e071b03c3..c11a78feb 100644 --- a/crates/arroyo-connectors/src/fluvio/source.rs +++ b/crates/arroyo-connectors/src/fluvio/source.rs @@ -166,7 +166,7 @@ impl FluvioSourceFunc { match message { Some((_, Ok(msg))) => { let timestamp = from_millis(msg.timestamp().max(0) as u64); - ctx.deserialize_slice(msg.value(), timestamp, (false, 0, 0, "".to_string())).await?; + ctx.deserialize_slice(msg.value(), timestamp, (false, 0, 0, "".to_string()), None).await?; if ctx.should_flush() { ctx.flush_buffer().await?; diff --git a/crates/arroyo-connectors/src/impulse/mod.rs b/crates/arroyo-connectors/src/impulse/mod.rs index 2cee9b1c7..94739ae68 100644 --- a/crates/arroyo-connectors/src/impulse/mod.rs +++ b/crates/arroyo-connectors/src/impulse/mod.rs @@ -101,6 +101,8 @@ impl Connector for ImpulseConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, + _enable_metadata: Option, + _metadata_fields: Option>, ) -> anyhow::Result { let event_rate = f64::from_str(&pull_opt("event_rate", options)?) .map_err(|_| anyhow!("invalid value for event_rate; expected float"))?; @@ -134,6 +136,8 @@ impl Connector for ImpulseConnector { message_count, }, None, + None, + None, ) } @@ -144,6 +148,8 @@ impl Connector for ImpulseConnector { config: Self::ProfileT, table: Self::TableT, _: Option<&ConnectionSchema>, + _enable_metadata: Option, + _metadata_fields: Option>, ) -> anyhow::Result { let description = format!( "{}Impulse<{} eps{}>", @@ -166,6 +172,8 @@ impl Connector for ImpulseConnector { format: None, bad_data: None, framing: None, + enable_metadata: None, + metadata_fields: None, }; Ok(Connection { diff --git a/crates/arroyo-connectors/src/kafka/mod.rs b/crates/arroyo-connectors/src/kafka/mod.rs index 8f98e9ee0..d19439e90 100644 --- a/crates/arroyo-connectors/src/kafka/mod.rs +++ b/crates/arroyo-connectors/src/kafka/mod.rs @@ -188,6 +188,8 @@ impl Connector for KafkaConnector { config: KafkaConfig, table: KafkaTable, schema: Option<&ConnectionSchema>, + enable_metadata: Option, + metadata_fields: Option>, ) -> anyhow::Result { let (typ, desc) = match table.type_ { TableType::Source { .. } => ( @@ -214,6 +216,8 @@ impl Connector for KafkaConnector { format: Some(format), bad_data: schema.bad_data.clone(), framing: schema.framing.clone(), + enable_metadata, + metadata_fields, }; Ok(Connection { @@ -312,6 +316,8 @@ impl Connector for KafkaConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, profile: Option<&ConnectionProfile>, + enable_metadata: Option, + metadata_fields: Option>, ) -> anyhow::Result { let connection = profile .map(|p| { @@ -323,7 +329,16 @@ impl Connector for KafkaConnector { let table = Self::table_from_options(options)?; - Self::from_config(self, None, name, connection, table, schema) + Self::from_config( + self, + None, + name, + connection, + table, + schema, + enable_metadata, + metadata_fields, + ) } fn make_operator( @@ -383,6 +398,8 @@ impl Connector for KafkaConnector { .unwrap_or(u32::MAX), ) .unwrap(), + enable_metadata: config.enable_metadata, + metadata_fields: config.metadata_fields, }))) } TableType::Sink { @@ -627,6 +644,7 @@ impl KafkaTester { &msg, SystemTime::now(), (false, 0, 0, "".to_string()), + None, ) .await .into_iter() @@ -654,6 +672,7 @@ impl KafkaTester { &msg, SystemTime::now(), (false, 0, 0, "".to_string()), + None, ) .await .into_iter() @@ -693,6 +712,7 @@ impl KafkaTester { &msg, SystemTime::now(), (false, 0, 0, "".to_string()), + None, ) .await .into_iter() diff --git a/crates/arroyo-connectors/src/kafka/source/mod.rs b/crates/arroyo-connectors/src/kafka/source/mod.rs index 00f19152d..7c7ca8ff1 100644 --- a/crates/arroyo-connectors/src/kafka/source/mod.rs +++ b/crates/arroyo-connectors/src/kafka/source/mod.rs @@ -35,6 +35,8 @@ pub struct KafkaSourceFunc { pub schema_resolver: Option>, pub client_configs: HashMap, pub messages_per_second: NonZeroU32, + pub enable_metadata: Option, + pub metadata_fields: Option>, } #[derive(Copy, Clone, Debug, Encode, Decode, PartialEq, PartialOrd)] @@ -178,9 +180,9 @@ impl KafkaSourceFunc { .ok_or_else(|| UserError::new("Failed to read timestamp from Kafka record", "The message read from Kafka did not contain a message timestamp"))?; - let metadata_enabled = true; - let kafka_metadata = (metadata_enabled, msg.offset(), msg.partition(), self.topic.clone()); - ctx.deserialize_slice(v, from_millis(timestamp as u64), kafka_metadata).await?; + let kafka_metadata = (self.enable_metadata.unwrap_or(false), msg.offset(), msg.partition(), self.topic.clone()); + + ctx.deserialize_slice(v, from_millis(timestamp as u64), kafka_metadata, self.metadata_fields.clone()).await?; if ctx.should_flush() { ctx.flush_buffer().await?; diff --git a/crates/arroyo-connectors/src/kafka/source/test.rs b/crates/arroyo-connectors/src/kafka/source/test.rs index 0057b7ae8..31b8187e7 100644 --- a/crates/arroyo-connectors/src/kafka/source/test.rs +++ b/crates/arroyo-connectors/src/kafka/source/test.rs @@ -87,6 +87,8 @@ impl KafkaTopicTester { schema_resolver: None, client_configs: HashMap::new(), messages_per_second: NonZeroU32::new(100).unwrap(), + enable_metadata: Some(false), + metadata_fields: None, }); let (to_control_tx, control_rx) = channel(128); diff --git a/crates/arroyo-connectors/src/kinesis/mod.rs b/crates/arroyo-connectors/src/kinesis/mod.rs index 085b00ec3..ae01f23ec 100644 --- a/crates/arroyo-connectors/src/kinesis/mod.rs +++ b/crates/arroyo-connectors/src/kinesis/mod.rs @@ -83,6 +83,8 @@ impl Connector for KinesisConnector { config: Self::ProfileT, table: Self::TableT, schema: Option<&ConnectionSchema>, + _enable_metadata: Option, + _metadata_fields: Option>, ) -> anyhow::Result { let (connection_type, description) = match table.type_ { TableType::Source { .. } => ( @@ -111,6 +113,8 @@ impl Connector for KinesisConnector { format: Some(format), bad_data: schema.bad_data.clone(), framing: schema.framing.clone(), + enable_metadata: None, + metadata_fields: None, }; Ok(Connection { @@ -130,6 +134,8 @@ impl Connector for KinesisConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, + _enable_metadata: Option, + _metadata_fields: Option>, ) -> anyhow::Result { let typ = pull_opt("type", options)?; let table_type = match typ.as_str() { @@ -166,7 +172,7 @@ impl Connector for KinesisConnector { aws_region: options.remove("aws_region").map(|s| s.to_string()), }; - Self::from_config(self, None, name, EmptyConfig {}, table, schema) + Self::from_config(self, None, name, EmptyConfig {}, table, schema, None, None) } fn make_operator( diff --git a/crates/arroyo-connectors/src/kinesis/source.rs b/crates/arroyo-connectors/src/kinesis/source.rs index 7587a10eb..240410f55 100644 --- a/crates/arroyo-connectors/src/kinesis/source.rs +++ b/crates/arroyo-connectors/src/kinesis/source.rs @@ -439,6 +439,7 @@ impl KinesisSourceFunc { &data, from_nanos(timestamp.as_nanos() as u128), (false, 0, 0, "".to_string()), + None, ) .await?; diff --git a/crates/arroyo-connectors/src/mqtt/mod.rs b/crates/arroyo-connectors/src/mqtt/mod.rs index fc6e6d45f..592c87854 100644 --- a/crates/arroyo-connectors/src/mqtt/mod.rs +++ b/crates/arroyo-connectors/src/mqtt/mod.rs @@ -156,6 +156,8 @@ impl Connector for MqttConnector { config: MqttConfig, table: MqttTable, schema: Option<&ConnectionSchema>, + _enable_metadata: Option, + _metadata_fields: Option>, ) -> anyhow::Result { let (typ, desc) = match table.type_ { TableType::Source { .. } => ( @@ -182,6 +184,8 @@ impl Connector for MqttConnector { format: Some(format), bad_data: schema.bad_data.clone(), framing: schema.framing.clone(), + enable_metadata: None, + metadata_fields: None, }; Ok(Connection { @@ -242,6 +246,8 @@ impl Connector for MqttConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, profile: Option<&ConnectionProfile>, + _enable_metadata: Option, + _metadata_fields: Option>, ) -> anyhow::Result { let connection = profile .map(|p| { @@ -253,7 +259,7 @@ impl Connector for MqttConnector { let table = Self::table_from_options(options)?; - Self::from_config(self, None, name, connection, table, schema) + Self::from_config(self, None, name, connection, table, schema, None, None) } fn make_operator( diff --git a/crates/arroyo-connectors/src/mqtt/source/mod.rs b/crates/arroyo-connectors/src/mqtt/source/mod.rs index 39d575d3b..6da21e7db 100644 --- a/crates/arroyo-connectors/src/mqtt/source/mod.rs +++ b/crates/arroyo-connectors/src/mqtt/source/mod.rs @@ -143,7 +143,7 @@ impl MqttSourceFunc { event = eventloop.poll() => { match event { Ok(MqttEvent::Incoming(Incoming::Publish(p))) => { - ctx.deserialize_slice(&p.payload, SystemTime::now(), (false, 0, 0, "".to_string())).await?; + ctx.deserialize_slice(&p.payload, SystemTime::now(), (false, 0, 0, "".to_string()), None).await?; rate_limiter.until_ready().await; } Ok(MqttEvent::Outgoing(Outgoing::Subscribe(_))) => { diff --git a/crates/arroyo-connectors/src/nats/mod.rs b/crates/arroyo-connectors/src/nats/mod.rs index da6a89199..71b7c12b7 100644 --- a/crates/arroyo-connectors/src/nats/mod.rs +++ b/crates/arroyo-connectors/src/nats/mod.rs @@ -246,6 +246,8 @@ impl Connector for NatsConnector { config: NatsConfig, table: NatsTable, schema: Option<&ConnectionSchema>, + _enable_metadata: Option, + _metadata_fields: Option>, ) -> anyhow::Result { let stream_or_subject = match &table.connector_type { ConnectorType::Source { source_type, .. } => { @@ -295,6 +297,8 @@ impl Connector for NatsConnector { format: Some(format), bad_data: schema.bad_data.clone(), framing: schema.framing.clone(), + enable_metadata: None, + metadata_fields: None, }; Ok(Connection { @@ -314,6 +318,8 @@ impl Connector for NatsConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, profile: Option<&ConnectionProfile>, + _enable_metadata: Option, + _metadata_fields: Option>, ) -> anyhow::Result { let connection = profile .map(|p| { @@ -325,7 +331,7 @@ impl Connector for NatsConnector { let table = Self::table_from_options(options)?; - Self::from_config(self, None, name, connection, table, schema) + Self::from_config(self, None, name, connection, table, schema, None, None) } fn make_operator( diff --git a/crates/arroyo-connectors/src/nats/source/mod.rs b/crates/arroyo-connectors/src/nats/source/mod.rs index d0c36527b..5a246c0c4 100644 --- a/crates/arroyo-connectors/src/nats/source/mod.rs +++ b/crates/arroyo-connectors/src/nats/source/mod.rs @@ -367,7 +367,7 @@ impl NatsSourceFunc { let message_info = msg.info().expect("Couldn't get message information"); let timestamp = message_info.published.into() ; - ctx.deserialize_slice(payload, timestamp, (false, 0, 0, "".to_string())).await?; + ctx.deserialize_slice(payload, timestamp, (false, 0, 0, "".to_string()), None).await?; debug!("---------------------------------------------->"); debug!( @@ -493,7 +493,7 @@ impl NatsSourceFunc { Some(msg) => { let payload = msg.payload.as_ref(); let timestamp = SystemTime::now(); - ctx.deserialize_slice(payload, timestamp, (false, 0, 0, "".to_string())).await?; + ctx.deserialize_slice(payload, timestamp, (false, 0, 0, "".to_string()), None).await?; if ctx.should_flush() { ctx.flush_buffer().await?; } diff --git a/crates/arroyo-connectors/src/nexmark/mod.rs b/crates/arroyo-connectors/src/nexmark/mod.rs index 5e3636921..c3d9f4e21 100644 --- a/crates/arroyo-connectors/src/nexmark/mod.rs +++ b/crates/arroyo-connectors/src/nexmark/mod.rs @@ -158,6 +158,8 @@ impl Connector for NexmarkConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, + _enable_metadata: Option, + _metadata_fields: Option>, ) -> anyhow::Result { let event_rate = f64::from_str(&pull_opt("event_rate", options)?) .map_err(|_| anyhow!("invalid value for event_rate; expected float"))?; @@ -183,6 +185,8 @@ impl Connector for NexmarkConnector { runtime, }, None, + None, + None, ) } @@ -193,6 +197,8 @@ impl Connector for NexmarkConnector { config: Self::ProfileT, table: Self::TableT, _: Option<&ConnectionSchema>, + _enable_metadata: Option, + _metadata_fields: Option>, ) -> anyhow::Result { let description = format!( "{}Nexmark<{} eps>", @@ -211,6 +217,8 @@ impl Connector for NexmarkConnector { format: None, bad_data: None, framing: None, + enable_metadata: None, + metadata_fields: None, }; Ok(Connection { diff --git a/crates/arroyo-connectors/src/polling_http/mod.rs b/crates/arroyo-connectors/src/polling_http/mod.rs index d620ce4d5..3fc53f190 100644 --- a/crates/arroyo-connectors/src/polling_http/mod.rs +++ b/crates/arroyo-connectors/src/polling_http/mod.rs @@ -152,6 +152,8 @@ impl Connector for PollingHTTPConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, + _enable_metadata: Option, + _metadata_fields: Option>, ) -> anyhow::Result { let endpoint = pull_opt("endpoint", options)?; let headers = options.remove("headers"); @@ -183,6 +185,8 @@ impl Connector for PollingHTTPConnector { emit_behavior, }, schema, + None, + None, ) } @@ -193,6 +197,8 @@ impl Connector for PollingHTTPConnector { config: Self::ProfileT, table: Self::TableT, schema: Option<&ConnectionSchema>, + _enable_metadata: Option, + _metadata_fields: Option>, ) -> anyhow::Result { let description = format!("PollingHTTPSource<{}>", table.endpoint); @@ -222,6 +228,8 @@ impl Connector for PollingHTTPConnector { format: Some(format), bad_data: schema.bad_data.clone(), framing: schema.framing.clone(), + enable_metadata: None, + metadata_fields: None, }; Ok(Connection { diff --git a/crates/arroyo-connectors/src/polling_http/operator.rs b/crates/arroyo-connectors/src/polling_http/operator.rs index 5a3bb37b6..733c18cc4 100644 --- a/crates/arroyo-connectors/src/polling_http/operator.rs +++ b/crates/arroyo-connectors/src/polling_http/operator.rs @@ -215,7 +215,7 @@ impl PollingHttpSourceFunc { continue; } - ctx.deserialize_slice(&buf, SystemTime::now(), (false, 0, 0, "".to_string())).await?; + ctx.deserialize_slice(&buf, SystemTime::now(), (false, 0, 0, "".to_string()), None).await?; if ctx.should_flush() { ctx.flush_buffer().await?; diff --git a/crates/arroyo-connectors/src/preview/mod.rs b/crates/arroyo-connectors/src/preview/mod.rs index d69939d6e..4675d09f0 100644 --- a/crates/arroyo-connectors/src/preview/mod.rs +++ b/crates/arroyo-connectors/src/preview/mod.rs @@ -71,6 +71,8 @@ impl Connector for PreviewConnector { _: &mut HashMap, _: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, + _enable_metadata: Option, + _metadata_fields: Option>, ) -> anyhow::Result { bail!("Preview connector cannot be created in SQL"); } @@ -82,6 +84,8 @@ impl Connector for PreviewConnector { config: Self::ProfileT, table: Self::TableT, schema: Option<&ConnectionSchema>, + _enable_metadata: Option, + _metadata_fields: Option>, ) -> anyhow::Result { let description = "PreviewSink".to_string(); @@ -96,6 +100,8 @@ impl Connector for PreviewConnector { format: None, bad_data: schema.bad_data.clone(), framing: schema.framing.clone(), + enable_metadata: None, + metadata_fields: None, }; Ok(Connection { diff --git a/crates/arroyo-connectors/src/redis/mod.rs b/crates/arroyo-connectors/src/redis/mod.rs index 16e8d7145..ceb4f00f6 100644 --- a/crates/arroyo-connectors/src/redis/mod.rs +++ b/crates/arroyo-connectors/src/redis/mod.rs @@ -227,6 +227,8 @@ impl Connector for RedisConnector { options: &mut HashMap, s: Option<&ConnectionSchema>, profile: Option<&ConnectionProfile>, + _enable_metadata: Option, + _metadata_fields: Option>, ) -> anyhow::Result { let connection_config = match profile { Some(connection_profile) => { @@ -348,6 +350,8 @@ impl Connector for RedisConnector { connector_type: sink, }, s, + None, + None, ) } @@ -358,6 +362,8 @@ impl Connector for RedisConnector { config: Self::ProfileT, table: Self::TableT, schema: Option<&ConnectionSchema>, + _enable_metadata: Option, + _metadata_fields: Option>, ) -> anyhow::Result { let schema = schema .map(|s| s.to_owned()) @@ -378,6 +384,8 @@ impl Connector for RedisConnector { format: Some(format), bad_data: schema.bad_data.clone(), framing: schema.framing.clone(), + enable_metadata: None, + metadata_fields: None, }; Ok(Connection { diff --git a/crates/arroyo-connectors/src/single_file/mod.rs b/crates/arroyo-connectors/src/single_file/mod.rs index 77a4f9bff..01793ca24 100644 --- a/crates/arroyo-connectors/src/single_file/mod.rs +++ b/crates/arroyo-connectors/src/single_file/mod.rs @@ -84,6 +84,8 @@ impl Connector for SingleFileConnector { config: Self::ProfileT, table: Self::TableT, schema: Option<&ConnectionSchema>, + _enable_metadata: Option, + _metadata_fields: Option>, ) -> anyhow::Result { let schema = schema .map(|s| s.to_owned()) @@ -103,6 +105,8 @@ impl Connector for SingleFileConnector { format: Some(format), bad_data: schema.bad_data.clone(), framing: schema.framing.clone(), + enable_metadata: None, + metadata_fields: None, }; Ok(Connection { @@ -122,6 +126,8 @@ impl Connector for SingleFileConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, + _enable_metadata: Option, + _metadata_fields: Option>, ) -> anyhow::Result { let path = pull_opt("path", options)?; let Ok(table_type) = pull_opt("type", options)?.try_into() else { @@ -146,6 +152,8 @@ impl Connector for SingleFileConnector { wait_for_control, }, schema, + None, + None, ) } diff --git a/crates/arroyo-connectors/src/single_file/source.rs b/crates/arroyo-connectors/src/single_file/source.rs index e8c262926..690d19e11 100644 --- a/crates/arroyo-connectors/src/single_file/source.rs +++ b/crates/arroyo-connectors/src/single_file/source.rs @@ -56,6 +56,7 @@ impl SingleFileSourceFunc { s.as_bytes(), SystemTime::now(), (false, 0, 0, "".to_string()), + None, ) .await .unwrap(); diff --git a/crates/arroyo-connectors/src/sse/mod.rs b/crates/arroyo-connectors/src/sse/mod.rs index 5da89ebbb..fa1a3502a 100644 --- a/crates/arroyo-connectors/src/sse/mod.rs +++ b/crates/arroyo-connectors/src/sse/mod.rs @@ -78,6 +78,8 @@ impl Connector for SSEConnector { config: Self::ProfileT, table: Self::TableT, schema: Option<&ConnectionSchema>, + _enable_metadata: Option, + _metadata_fields: Option>, ) -> anyhow::Result { let description = format!("SSESource<{}>", table.endpoint); @@ -107,6 +109,8 @@ impl Connector for SSEConnector { format: Some(format), bad_data: schema.bad_data.clone(), framing: schema.framing.clone(), + enable_metadata: None, + metadata_fields: None, }; Ok(Connection { @@ -126,6 +130,8 @@ impl Connector for SSEConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, + _enable_metadata: Option, + _metadata_fields: Option>, ) -> anyhow::Result { let endpoint = pull_opt("endpoint", options)?; let headers = options.remove("headers"); @@ -141,6 +147,8 @@ impl Connector for SSEConnector { headers: headers.map(VarStr::new), }, schema, + None, + None, ) } diff --git a/crates/arroyo-connectors/src/sse/operator.rs b/crates/arroyo-connectors/src/sse/operator.rs index e562bd980..2c9e233f2 100644 --- a/crates/arroyo-connectors/src/sse/operator.rs +++ b/crates/arroyo-connectors/src/sse/operator.rs @@ -172,7 +172,7 @@ impl SSESourceFunc { if events.is_empty() || events.contains(&event.event_type) { ctx.deserialize_slice( - event.data.as_bytes(), SystemTime::now(), (false, 0, 0, "".to_string())).await?; + event.data.as_bytes(), SystemTime::now(), (false, 0, 0, "".to_string()), None).await?; if ctx.should_flush() { ctx.flush_buffer().await?; diff --git a/crates/arroyo-connectors/src/stdout/mod.rs b/crates/arroyo-connectors/src/stdout/mod.rs index dfb1be021..3e0a34d3c 100644 --- a/crates/arroyo-connectors/src/stdout/mod.rs +++ b/crates/arroyo-connectors/src/stdout/mod.rs @@ -76,8 +76,18 @@ impl Connector for StdoutConnector { _options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, + _enable_metadata: Option, + _metadata_fields: Option>, ) -> anyhow::Result { - self.from_config(None, name, EmptyConfig {}, EmptyConfig {}, schema) + self.from_config( + None, + name, + EmptyConfig {}, + EmptyConfig {}, + schema, + None, + None, + ) } fn from_config( @@ -87,6 +97,8 @@ impl Connector for StdoutConnector { config: Self::ProfileT, table: Self::TableT, schema: Option<&ConnectionSchema>, + _enable_metadata: Option, + _metadata_fields: Option>, ) -> anyhow::Result { let description = "StdoutSink".to_string(); @@ -107,6 +119,8 @@ impl Connector for StdoutConnector { format: Some(format), bad_data: schema.bad_data.clone(), framing: schema.framing.clone(), + enable_metadata: None, + metadata_fields: None, }; Ok(Connection { diff --git a/crates/arroyo-connectors/src/webhook/mod.rs b/crates/arroyo-connectors/src/webhook/mod.rs index 84e8d681c..c8718b9f9 100644 --- a/crates/arroyo-connectors/src/webhook/mod.rs +++ b/crates/arroyo-connectors/src/webhook/mod.rs @@ -142,6 +142,8 @@ impl Connector for WebhookConnector { config: Self::ProfileT, table: Self::TableT, schema: Option<&ConnectionSchema>, + _enable_metadata: Option, + _metadata_fields: Option>, ) -> anyhow::Result { let description = format!("WebhookSink<{}>", table.endpoint.sub_env_vars()?); @@ -162,6 +164,8 @@ impl Connector for WebhookConnector { format: Some(format), bad_data: schema.bad_data.clone(), framing: schema.framing.clone(), + enable_metadata: None, + metadata_fields: None, }; Ok(Connection { @@ -181,6 +185,8 @@ impl Connector for WebhookConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, + _enable_metadata: Option, + _metadata_fields: Option>, ) -> anyhow::Result { let endpoint = pull_opt("endpoint", options)?; @@ -201,7 +207,7 @@ impl Connector for WebhookConnector { )?; let _ = Self::construct_test_request(&client, &table)?; - self.from_config(None, name, EmptyConfig {}, table, schema) + self.from_config(None, name, EmptyConfig {}, table, schema, None, None) } fn make_operator( diff --git a/crates/arroyo-connectors/src/websocket/mod.rs b/crates/arroyo-connectors/src/websocket/mod.rs index bf6d50660..425445755 100644 --- a/crates/arroyo-connectors/src/websocket/mod.rs +++ b/crates/arroyo-connectors/src/websocket/mod.rs @@ -220,6 +220,8 @@ impl Connector for WebsocketConnector { config: Self::ProfileT, table: Self::TableT, schema: Option<&ConnectionSchema>, + _enable_metadata: Option, + _metadata_fields: Option>, ) -> anyhow::Result { let description = format!("WebsocketSource<{}>", table.endpoint); @@ -249,6 +251,8 @@ impl Connector for WebsocketConnector { format: Some(format), bad_data: schema.bad_data.clone(), framing: schema.framing.clone(), + enable_metadata: None, + metadata_fields: None, }; Ok(Connection { @@ -268,6 +272,8 @@ impl Connector for WebsocketConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, + _enable_metadata: Option, + _metadata_fields: Option>, ) -> anyhow::Result { let endpoint = pull_opt("endpoint", options)?; let headers = options.remove("headers"); @@ -304,6 +310,8 @@ impl Connector for WebsocketConnector { subscription_messages, }, schema, + None, + None, ) } diff --git a/crates/arroyo-connectors/src/websocket/operator.rs b/crates/arroyo-connectors/src/websocket/operator.rs index f4c96dd68..394ff97f6 100644 --- a/crates/arroyo-connectors/src/websocket/operator.rs +++ b/crates/arroyo-connectors/src/websocket/operator.rs @@ -122,7 +122,7 @@ impl WebsocketSourceFunc { msg: &[u8], ctx: &mut ArrowContext, ) -> Result<(), UserError> { - ctx.deserialize_slice(msg, SystemTime::now(), (false, 0, 0, "".to_string())) + ctx.deserialize_slice(msg, SystemTime::now(), (false, 0, 0, "".to_string()), None) .await?; if ctx.should_flush() { diff --git a/crates/arroyo-formats/src/avro/de.rs b/crates/arroyo-formats/src/avro/de.rs index 7967dc8d0..f1bd54975 100644 --- a/crates/arroyo-formats/src/avro/de.rs +++ b/crates/arroyo-formats/src/avro/de.rs @@ -286,6 +286,7 @@ mod tests { message, SystemTime::now(), (false, 0, 0, "".to_string()), + None, ) .await; assert_eq!(errors, vec![]); diff --git a/crates/arroyo-formats/src/de.rs b/crates/arroyo-formats/src/de.rs index 7c70b6336..1c5265b12 100644 --- a/crates/arroyo-formats/src/de.rs +++ b/crates/arroyo-formats/src/de.rs @@ -84,6 +84,7 @@ pub struct ArrowDeserializer { proto_pool: DescriptorPool, schema_resolver: Arc, kafka_metadata_builder: Option<(Int64Builder, Int32Builder, StringBuilder)>, + metadata_fields: Option>, } impl ArrowDeserializer { @@ -161,6 +162,7 @@ impl ArrowDeserializer { buffered_count: 0, buffered_since: Instant::now(), kafka_metadata_builder: None, + metadata_fields: None, } } @@ -170,11 +172,12 @@ impl ArrowDeserializer { msg: &[u8], timestamp: SystemTime, kafka_metadata: (bool, i64, i32, String), + metadata_fields: Option> ) -> Vec { match &*self.format { Format::Avro(_) => self.deserialize_slice_avro(buffer, msg, timestamp).await, _ => FramingIterator::new(self.framing.clone(), msg) - .map(|t| self.deserialize_single(buffer, t, timestamp, kafka_metadata.clone())) + .map(|t| self.deserialize_single(buffer, t, timestamp, kafka_metadata.clone(), metadata_fields.clone())) .filter_map(|t| t.err()) .collect(), } @@ -200,28 +203,33 @@ impl ArrowDeserializer { let mut columns = batch.columns().to_vec(); columns.insert(self.schema.timestamp_index, Arc::new(timestamp.finish())); - if let Some((offset_builder, partition_builder, topic_builder)) = - &mut self.kafka_metadata_builder - { - if let Some((topic_idx, _)) = - self.schema.schema.column_with_name("topic") - { - columns.remove(topic_idx); - columns.insert(topic_idx, Arc::new(topic_builder.finish())); - } - - if let Some((partition_idx, _)) = - self.schema.schema.column_with_name("partition") - { - columns.remove(partition_idx); - columns.insert(partition_idx, Arc::new(partition_builder.finish())); - } - - if let Some((offset_idx, _)) = - self.schema.schema.column_with_name("offset") - { - columns.remove(offset_idx); - columns.insert(offset_idx, Arc::new(offset_builder.finish())); + if let Some((offset_builder, partition_builder, topic_builder)) = &mut self.kafka_metadata_builder { + if let Some(fields) = &self.metadata_fields { + for (field_name, argument_name) in fields.iter() { + match argument_name.as_str() { + "topic" => { + if let Some((idx, _)) = self.schema.schema.column_with_name(field_name) { + columns.remove(idx); + columns.insert(idx, Arc::new(topic_builder.finish())); + } + } + "partition" => { + if let Some((idx, _)) = self.schema.schema.column_with_name(field_name) { + columns.remove(idx); + columns.insert(idx, Arc::new(partition_builder.finish())); + } + } + "offset_id" => { + if let Some((idx, _)) = self.schema.schema.column_with_name(field_name) { + columns.remove(idx); + columns.insert(idx, Arc::new(offset_builder.finish())); + } + } + _ => { + // Handle unexpected argument names or log a message if necessary + } + } + } } } RecordBatch::try_new(self.schema.schema.clone(), columns).unwrap() @@ -255,7 +263,9 @@ impl ArrowDeserializer { msg: &[u8], timestamp: SystemTime, kafka_metadata: (bool, i64, i32, String), + metadata_fields: Option> ) -> Result<(), SourceError> { + self.metadata_fields = metadata_fields; match &*self.format { Format::RawString(_) | Format::Json(JsonFormat { @@ -270,6 +280,7 @@ impl ArrowDeserializer { &kafka_metadata.3, kafka_metadata.2, kafka_metadata.1, + self.metadata_fields.clone(), ); } } @@ -283,6 +294,7 @@ impl ArrowDeserializer { &kafka_metadata.3, kafka_metadata.2, kafka_metadata.1, + self.metadata_fields.clone(), ); } } @@ -343,6 +355,7 @@ impl ArrowDeserializer { &kafka_metadata.3, kafka_metadata.2, kafka_metadata.1, + self.metadata_fields.clone(), ); } self.buffered_count += 1; @@ -518,22 +531,33 @@ pub(crate) fn add_kafka_metadata( topic: &str, partition: i32, offset: i64, + metadata_fields: Option>, ) { - let (topic_idx, _) = schema - .schema - .column_with_name("topic") - .expect("No column named 'topic' in the schema"); - let (partition_idx, _) = schema - .schema - .column_with_name("partition") - .expect("No column named 'partition' in the schema"); - let (offset_idx, _) = schema - .schema - .column_with_name("offset") - .expect("No column named 'offset' in the schema"); - add_kafka_metadata_topic(builder, topic_idx, topic); - add_kafka_metadata_partition(builder, partition_idx, partition); - add_kafka_metadata_offset(builder, offset_idx, offset); + if let Some(fields) = metadata_fields { + for (field_name, argument_name) in fields.iter() { + // Match each argument name and add the corresponding metadata if found + match argument_name.as_str() { + "topic" => { + if let Some((idx, _)) = schema.schema.column_with_name(field_name) { + add_kafka_metadata_topic(builder, idx, topic); + } + } + "partition" => { + if let Some((idx, _)) = schema.schema.column_with_name(field_name) { + add_kafka_metadata_partition(builder, idx, partition); + } + } + "offset_id" => { + if let Some((idx, _)) = schema.schema.column_with_name(field_name) { + add_kafka_metadata_offset(builder, idx, offset); + } + } + _ => { + // Handle any unexpected argument names or log a message if necessary + } + } + } + } } #[cfg(test)] @@ -671,7 +695,8 @@ mod tests { &mut arrays[..], json!({ "x": 5 }).to_string().as_bytes(), now, - (false, 0, 0, "".to_string()) + (false, 0, 0, "".to_string()), + None ) .await, vec![] @@ -682,7 +707,8 @@ mod tests { &mut arrays[..], json!({ "x": "hello" }).to_string().as_bytes(), now, - (false, 0, 0, "".to_string()) + (false, 0, 0, "".to_string()), + None ) .await, vec![] @@ -709,7 +735,8 @@ mod tests { &mut arrays[..], json!({ "x": 5 }).to_string().as_bytes(), SystemTime::now(), - (false, 0, 0, "".to_string()) + (false, 0, 0, "".to_string()), + None ) .await, vec![] @@ -720,7 +747,8 @@ mod tests { &mut arrays[..], json!({ "x": "hello" }).to_string().as_bytes(), SystemTime::now(), - (false, 0, 0, "".to_string()) + (false, 0, 0, "".to_string()), + None ) .await, vec![] @@ -764,6 +792,7 @@ mod tests { &[0, 1, 2, 3, 4, 5], time, (false, 0, 0, "".to_string()), + None ) .await; assert!(result.is_empty()); diff --git a/crates/arroyo-operator/src/connector.rs b/crates/arroyo-operator/src/connector.rs index 5b8f82b08..11e150ed6 100644 --- a/crates/arroyo-operator/src/connector.rs +++ b/crates/arroyo-operator/src/connector.rs @@ -89,6 +89,8 @@ pub trait Connector: Send { options: &mut HashMap, schema: Option<&ConnectionSchema>, profile: Option<&ConnectionProfile>, + enable_metadata: Option, + metadata_fields: Option>, ) -> anyhow::Result; fn from_config( @@ -98,6 +100,8 @@ pub trait Connector: Send { config: Self::ProfileT, table: Self::TableT, schema: Option<&ConnectionSchema>, + enable_metadata: Option, + metadata_fields: Option>, ) -> anyhow::Result; #[allow(unused)] @@ -162,6 +166,8 @@ pub trait ErasedConnector: Send { options: &mut HashMap, schema: Option<&ConnectionSchema>, profile: Option<&ConnectionProfile>, + enable_metadata: Option, + metadata_fields: Option>, ) -> anyhow::Result; fn from_config( @@ -171,6 +177,8 @@ pub trait ErasedConnector: Send { config: &serde_json::Value, table: &serde_json::Value, schema: Option<&ConnectionSchema>, + enable_metadata: Option, + metadata_fields: Option>, ) -> anyhow::Result; fn make_operator(&self, config: OperatorConfig) -> anyhow::Result; @@ -256,8 +264,10 @@ impl ErasedConnector for C { options: &mut HashMap, schema: Option<&ConnectionSchema>, profile: Option<&ConnectionProfile>, + enable_metadata: Option, + metadata_fields: Option>, ) -> anyhow::Result { - self.from_options(name, options, schema, profile) + self.from_options(name, options, schema, profile, enable_metadata, metadata_fields) } fn from_config( @@ -267,6 +277,8 @@ impl ErasedConnector for C { config: &serde_json::Value, table: &serde_json::Value, schema: Option<&ConnectionSchema>, + enable_metadata: Option, + metadata_fields: Option>, ) -> anyhow::Result { self.from_config( id, @@ -274,6 +286,8 @@ impl ErasedConnector for C { self.parse_config(config)?, self.parse_table(table)?, schema, + enable_metadata, + metadata_fields, ) } diff --git a/crates/arroyo-operator/src/context.rs b/crates/arroyo-operator/src/context.rs index 4e1668d81..383c6ad55 100644 --- a/crates/arroyo-operator/src/context.rs +++ b/crates/arroyo-operator/src/context.rs @@ -671,6 +671,7 @@ impl ArrowContext { msg: &[u8], time: SystemTime, kafka_metadata: (bool, i64, i32, String), + metadata_fields: Option> ) -> Result<(), UserError> { let deserializer = self .deserializer @@ -690,6 +691,7 @@ impl ArrowContext { msg, time, kafka_metadata, + metadata_fields, ) .await; self.collect_source_errors(errors).await?; diff --git a/crates/arroyo-planner/src/tables.rs b/crates/arroyo-planner/src/tables.rs index 31bd36579..c6ff5c7a3 100644 --- a/crates/arroyo-planner/src/tables.rs +++ b/crates/arroyo-planner/src/tables.rs @@ -222,6 +222,8 @@ impl ConnectorTable { primary_keys: Vec, options: &mut HashMap, connection_profile: Option<&ConnectionProfile>, + enable_metadata: Option, + connector_metadata_columns: Option>, ) -> Result { // TODO: a more principled way of letting connectors dictate types to use if "delta" == connector { @@ -297,7 +299,7 @@ impl ConnectorTable { .map_err(|e| DataFusionError::Plan(format!("could not create connection schema: {}", e)))?; let connection = connector - .from_options(name, options, Some(&schema), connection_profile) + .from_options(name, options, Some(&schema), connection_profile, enable_metadata, connector_metadata_columns) .map_err(|e| DataFusionError::Plan(e.to_string()))?; let mut table: ConnectorTable = connection.into(); @@ -507,6 +509,7 @@ impl Table { fn schema_from_columns( columns: &[ColumnDef], schema_provider: &ArroyoSchemaProvider, + connector_metadata_columns: &mut HashMap, ) -> Result> { let struct_field_pairs = columns .iter() @@ -518,7 +521,7 @@ impl Table { .iter() .any(|option| matches!(option.option, ColumnOption::NotNull)); - let mut struct_field = ArroyoExtensionType::add_metadata( + let struct_field = ArroyoExtensionType::add_metadata( extension, Field::new(name, data_type, nullable), ); @@ -550,13 +553,7 @@ impl Table { ), ), )) => { - let mut metadata: HashMap = - Default::default(); - metadata.insert( - "metadata_argument_".to_string(), - value.to_string(), - ); - struct_field.set_metadata(metadata); + connector_metadata_columns.insert(column.name.value.to_string(), value.to_string()); } _ => { "Unsupported argument format.".to_string(); @@ -650,7 +647,8 @@ impl Table { } let connector = with_map.remove("connector"); - let fields = Self::schema_from_columns(columns, schema_provider)?; + let mut connector_metadata_columns = Some(HashMap::new()); + let fields = Self::schema_from_columns(columns, schema_provider, connector_metadata_columns.as_mut().unwrap())?; let primary_keys = columns .iter() @@ -714,6 +712,8 @@ impl Table { primary_keys, &mut with_map, connection_profile, + Some(true), + connector_metadata_columns, ) .map_err(|e| e.context(format!("Failed to create table {}", name)))?, ))) diff --git a/crates/arroyo-planner/src/test/mod.rs b/crates/arroyo-planner/src/test/mod.rs index d2ec51f9d..b48d0f96d 100644 --- a/crates/arroyo-planner/src/test/mod.rs +++ b/crates/arroyo-planner/src/test/mod.rs @@ -24,6 +24,8 @@ fn get_test_schema_provider() -> ArroyoSchemaProvider { runtime: Some(10.0 * 1_000_000.0), }, None, + Some(false), + None, ) .unwrap(); diff --git a/crates/arroyo-rpc/src/lib.rs b/crates/arroyo-rpc/src/lib.rs index a6013f4e0..1c75367af 100644 --- a/crates/arroyo-rpc/src/lib.rs +++ b/crates/arroyo-rpc/src/lib.rs @@ -191,6 +191,8 @@ pub struct OperatorConfig { pub bad_data: Option, pub framing: Option, pub rate_limit: Option, + pub enable_metadata: Option, + pub metadata_fields: Option>, } impl Default for OperatorConfig { @@ -202,6 +204,8 @@ impl Default for OperatorConfig { bad_data: None, framing: None, rate_limit: None, + enable_metadata: None, + metadata_fields: None, } } } From 4f754a61daecf1b35ef58512bb6c314859ea114e Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw Date: Fri, 25 Oct 2024 22:55:27 +0530 Subject: [PATCH 15/36] adding enable_metadata flag to deser --- crates/arroyo-formats/src/de.rs | 49 +++++++++++++++++++-------------- 1 file changed, 28 insertions(+), 21 deletions(-) diff --git a/crates/arroyo-formats/src/de.rs b/crates/arroyo-formats/src/de.rs index 1c5265b12..a513c5571 100644 --- a/crates/arroyo-formats/src/de.rs +++ b/crates/arroyo-formats/src/de.rs @@ -83,6 +83,7 @@ pub struct ArrowDeserializer { schema_registry: Arc>>, proto_pool: DescriptorPool, schema_resolver: Arc, + enable_metadata: Option, kafka_metadata_builder: Option<(Int64Builder, Int32Builder, StringBuilder)>, metadata_fields: Option>, } @@ -161,6 +162,7 @@ impl ArrowDeserializer { proto_pool, buffered_count: 0, buffered_since: Instant::now(), + enable_metadata: Some(false), kafka_metadata_builder: None, metadata_fields: None, } @@ -203,30 +205,32 @@ impl ArrowDeserializer { let mut columns = batch.columns().to_vec(); columns.insert(self.schema.timestamp_index, Arc::new(timestamp.finish())); - if let Some((offset_builder, partition_builder, topic_builder)) = &mut self.kafka_metadata_builder { - if let Some(fields) = &self.metadata_fields { - for (field_name, argument_name) in fields.iter() { - match argument_name.as_str() { - "topic" => { - if let Some((idx, _)) = self.schema.schema.column_with_name(field_name) { - columns.remove(idx); - columns.insert(idx, Arc::new(topic_builder.finish())); + if self.enable_metadata.unwrap_or(false) { + if let Some((offset_builder, partition_builder, topic_builder)) = &mut self.kafka_metadata_builder { + if let Some(fields) = &self.metadata_fields { + for (field_name, argument_name) in fields.iter() { + match argument_name.as_str() { + "topic" => { + if let Some((idx, _)) = self.schema.schema.column_with_name(field_name) { + columns.remove(idx); + columns.insert(idx, Arc::new(topic_builder.finish())); + } } - } - "partition" => { - if let Some((idx, _)) = self.schema.schema.column_with_name(field_name) { - columns.remove(idx); - columns.insert(idx, Arc::new(partition_builder.finish())); + "partition" => { + if let Some((idx, _)) = self.schema.schema.column_with_name(field_name) { + columns.remove(idx); + columns.insert(idx, Arc::new(partition_builder.finish())); + } } - } - "offset_id" => { - if let Some((idx, _)) = self.schema.schema.column_with_name(field_name) { - columns.remove(idx); - columns.insert(idx, Arc::new(offset_builder.finish())); + "offset_id" => { + if let Some((idx, _)) = self.schema.schema.column_with_name(field_name) { + columns.remove(idx); + columns.insert(idx, Arc::new(offset_builder.finish())); + } + } + _ => { + // Handle unexpected argument names or log a message if necessary } - } - _ => { - // Handle unexpected argument names or log a message if necessary } } } @@ -274,6 +278,7 @@ impl ArrowDeserializer { self.deserialize_raw_string(buffer, msg); add_timestamp(buffer, self.schema.timestamp_index, timestamp); if kafka_metadata.0 { + self.enable_metadata = Some(true); add_kafka_metadata( buffer, &self.schema, @@ -288,6 +293,7 @@ impl ArrowDeserializer { self.deserialize_raw_bytes(buffer, msg); add_timestamp(buffer, self.schema.timestamp_index, timestamp); if kafka_metadata.0 { + self.enable_metadata = Some(true); add_kafka_metadata( buffer, &self.schema, @@ -310,6 +316,7 @@ impl ArrowDeserializer { }; if kafka_metadata.0 { + self.enable_metadata = Some(true); self.kafka_metadata_builder.get_or_insert_with(|| { ( Int64Builder::new(), From 53b6b83a2507081c0ac7d77a5cc688c39a85223a Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw Date: Fri, 25 Oct 2024 23:14:53 +0530 Subject: [PATCH 16/36] fmt and clippy fixes --- crates/arroyo-formats/src/de.rs | 47 +++++++++++++++++++++++++-------- 1 file changed, 36 insertions(+), 11 deletions(-) diff --git a/crates/arroyo-formats/src/de.rs b/crates/arroyo-formats/src/de.rs index a513c5571..904f21848 100644 --- a/crates/arroyo-formats/src/de.rs +++ b/crates/arroyo-formats/src/de.rs @@ -174,12 +174,20 @@ impl ArrowDeserializer { msg: &[u8], timestamp: SystemTime, kafka_metadata: (bool, i64, i32, String), - metadata_fields: Option> + metadata_fields: Option>, ) -> Vec { match &*self.format { Format::Avro(_) => self.deserialize_slice_avro(buffer, msg, timestamp).await, _ => FramingIterator::new(self.framing.clone(), msg) - .map(|t| self.deserialize_single(buffer, t, timestamp, kafka_metadata.clone(), metadata_fields.clone())) + .map(|t| { + self.deserialize_single( + buffer, + t, + timestamp, + kafka_metadata.clone(), + metadata_fields.clone(), + ) + }) .filter_map(|t| t.err()) .collect(), } @@ -206,26 +214,43 @@ impl ArrowDeserializer { columns.insert(self.schema.timestamp_index, Arc::new(timestamp.finish())); if self.enable_metadata.unwrap_or(false) { - if let Some((offset_builder, partition_builder, topic_builder)) = &mut self.kafka_metadata_builder { + if let Some((offset_builder, partition_builder, topic_builder)) = + &mut self.kafka_metadata_builder + { if let Some(fields) = &self.metadata_fields { for (field_name, argument_name) in fields.iter() { match argument_name.as_str() { "topic" => { - if let Some((idx, _)) = self.schema.schema.column_with_name(field_name) { + if let Some((idx, _)) = + self.schema.schema.column_with_name(field_name) + { columns.remove(idx); - columns.insert(idx, Arc::new(topic_builder.finish())); + columns.insert( + idx, + Arc::new(topic_builder.finish()), + ); } } "partition" => { - if let Some((idx, _)) = self.schema.schema.column_with_name(field_name) { + if let Some((idx, _)) = + self.schema.schema.column_with_name(field_name) + { columns.remove(idx); - columns.insert(idx, Arc::new(partition_builder.finish())); + columns.insert( + idx, + Arc::new(partition_builder.finish()), + ); } } "offset_id" => { - if let Some((idx, _)) = self.schema.schema.column_with_name(field_name) { + if let Some((idx, _)) = + self.schema.schema.column_with_name(field_name) + { columns.remove(idx); - columns.insert(idx, Arc::new(offset_builder.finish())); + columns.insert( + idx, + Arc::new(offset_builder.finish()), + ); } } _ => { @@ -267,7 +292,7 @@ impl ArrowDeserializer { msg: &[u8], timestamp: SystemTime, kafka_metadata: (bool, i64, i32, String), - metadata_fields: Option> + metadata_fields: Option>, ) -> Result<(), SourceError> { self.metadata_fields = metadata_fields; match &*self.format { @@ -799,7 +824,7 @@ mod tests { &[0, 1, 2, 3, 4, 5], time, (false, 0, 0, "".to_string()), - None + None, ) .await; assert!(result.is_empty()); From fc8b8cfc9f9af2b07ccff684e5a6d6dcc52fc751 Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw Date: Fri, 25 Oct 2024 23:17:05 +0530 Subject: [PATCH 17/36] fmt fixes --- crates/arroyo-operator/src/connector.rs | 9 ++++++++- crates/arroyo-operator/src/context.rs | 2 +- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/crates/arroyo-operator/src/connector.rs b/crates/arroyo-operator/src/connector.rs index 11e150ed6..d3e67e692 100644 --- a/crates/arroyo-operator/src/connector.rs +++ b/crates/arroyo-operator/src/connector.rs @@ -267,7 +267,14 @@ impl ErasedConnector for C { enable_metadata: Option, metadata_fields: Option>, ) -> anyhow::Result { - self.from_options(name, options, schema, profile, enable_metadata, metadata_fields) + self.from_options( + name, + options, + schema, + profile, + enable_metadata, + metadata_fields, + ) } fn from_config( diff --git a/crates/arroyo-operator/src/context.rs b/crates/arroyo-operator/src/context.rs index 383c6ad55..2186631c0 100644 --- a/crates/arroyo-operator/src/context.rs +++ b/crates/arroyo-operator/src/context.rs @@ -671,7 +671,7 @@ impl ArrowContext { msg: &[u8], time: SystemTime, kafka_metadata: (bool, i64, i32, String), - metadata_fields: Option> + metadata_fields: Option>, ) -> Result<(), UserError> { let deserializer = self .deserializer From b4c48152bb6d5dbd506582642ef05ed5c5af517f Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw Date: Fri, 25 Oct 2024 23:20:22 +0530 Subject: [PATCH 18/36] fmt fixes in planner --- crates/arroyo-planner/src/tables.rs | 73 +++++++++++++++++------------ 1 file changed, 44 insertions(+), 29 deletions(-) diff --git a/crates/arroyo-planner/src/tables.rs b/crates/arroyo-planner/src/tables.rs index c6ff5c7a3..5c604dd9f 100644 --- a/crates/arroyo-planner/src/tables.rs +++ b/crates/arroyo-planner/src/tables.rs @@ -299,7 +299,14 @@ impl ConnectorTable { .map_err(|e| DataFusionError::Plan(format!("could not create connection schema: {}", e)))?; let connection = connector - .from_options(name, options, Some(&schema), connection_profile, enable_metadata, connector_metadata_columns) + .from_options( + name, + options, + Some(&schema), + connection_profile, + enable_metadata, + connector_metadata_columns, + ) .map_err(|e| DataFusionError::Plan(e.to_string()))?; let mut table: ConnectorTable = connection.into(); @@ -526,24 +533,25 @@ impl Table { Field::new(name, data_type, nullable), ); - let generating_expression = - column.options.iter().find_map(|option| { - if let ColumnOption::Generated { - generation_expr, .. - } = &option.option + let generating_expression = column.options.iter().find_map(|option| { + if let ColumnOption::Generated { + generation_expr, .. + } = &option.option + { + if let Some(sqlparser::ast::Expr::Function(sqlparser::ast::Function { + name, + args, + .. + })) = generation_expr { - if let Some(sqlparser::ast::Expr::Function( - sqlparser::ast::Function { name, args, .. }, - )) = generation_expr + if name + .0 + .iter() + .any(|ident| ident.value.to_lowercase() == "metadata") { - if name - .0 - .iter() - .any(|ident| ident.value.to_lowercase() == "metadata") - { - match args { - FunctionArguments::List(arg_list) => { - match arg_list.args.first() { + match args { + FunctionArguments::List(arg_list) => { + match arg_list.args.first() { Some(FunctionArg::Unnamed( sqlparser::ast::FunctionArgExpr::Expr( sqlparser::ast::Expr::Value( @@ -553,25 +561,28 @@ impl Table { ), ), )) => { - connector_metadata_columns.insert(column.name.value.to_string(), value.to_string()); + connector_metadata_columns.insert( + column.name.value.to_string(), + value.to_string(), + ); } _ => { - "Unsupported argument format.".to_string(); + "Unsupported argument format.".to_string(); } } - } - _ => { - "Unsupported argument format.".to_string(); - } } - return None; + _ => { + "Unsupported argument format.".to_string(); + } } + return None; } - generation_expr.clone() - } else { - None } - }); + generation_expr.clone() + } else { + None + } + }); Ok((struct_field, generating_expression)) }) .collect::>>()?; @@ -648,7 +659,11 @@ impl Table { let connector = with_map.remove("connector"); let mut connector_metadata_columns = Some(HashMap::new()); - let fields = Self::schema_from_columns(columns, schema_provider, connector_metadata_columns.as_mut().unwrap())?; + let fields = Self::schema_from_columns( + columns, + schema_provider, + connector_metadata_columns.as_mut().unwrap(), + )?; let primary_keys = columns .iter() From c5af394f12a6b9c2eed76b1f396a1c49e76ea290 Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw Date: Fri, 25 Oct 2024 23:25:28 +0530 Subject: [PATCH 19/36] fmt fixes in tables.rs --- crates/arroyo-planner/src/tables.rs | 51 ++++++++++++++--------------- 1 file changed, 25 insertions(+), 26 deletions(-) diff --git a/crates/arroyo-planner/src/tables.rs b/crates/arroyo-planner/src/tables.rs index 5c604dd9f..c7faf1654 100644 --- a/crates/arroyo-planner/src/tables.rs +++ b/crates/arroyo-planner/src/tables.rs @@ -533,25 +533,24 @@ impl Table { Field::new(name, data_type, nullable), ); - let generating_expression = column.options.iter().find_map(|option| { - if let ColumnOption::Generated { - generation_expr, .. - } = &option.option - { - if let Some(sqlparser::ast::Expr::Function(sqlparser::ast::Function { - name, - args, - .. - })) = generation_expr + let generating_expression = + column.options.iter().find_map(|option| { + if let ColumnOption::Generated { + generation_expr, .. + } = &option.option { - if name - .0 - .iter() - .any(|ident| ident.value.to_lowercase() == "metadata") + if let Some(sqlparser::ast::Expr::Function( + sqlparser::ast::Function { name, args, .. }, + )) = generation_expr { - match args { - FunctionArguments::List(arg_list) => { - match arg_list.args.first() { + if name + .0 + .iter() + .any(|ident| ident.value.to_lowercase() == "metadata") + { + match args { + FunctionArguments::List(arg_list) => { + match arg_list.args.first() { Some(FunctionArg::Unnamed( sqlparser::ast::FunctionArgExpr::Expr( sqlparser::ast::Expr::Value( @@ -570,19 +569,19 @@ impl Table { "Unsupported argument format.".to_string(); } } + } + _ => { + "Unsupported argument format.".to_string(); + } } - _ => { - "Unsupported argument format.".to_string(); - } + return None; } - return None; } + generation_expr.clone() + } else { + None } - generation_expr.clone() - } else { - None - } - }); + }); Ok((struct_field, generating_expression)) }) .collect::>>()?; From 9cd7c0445707111cdfd586e27ab11e501341df95 Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw Date: Fri, 25 Oct 2024 23:41:50 +0530 Subject: [PATCH 20/36] clippy fixes --- crates/arroyo-operator/src/connector.rs | 3 ++- crates/arroyo-planner/src/tables.rs | 1 + 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/crates/arroyo-operator/src/connector.rs b/crates/arroyo-operator/src/connector.rs index d3e67e692..da2ff708e 100644 --- a/crates/arroyo-operator/src/connector.rs +++ b/crates/arroyo-operator/src/connector.rs @@ -22,7 +22,7 @@ pub struct Connection { pub description: String, } -#[allow(clippy::wrong_self_convention)] +#[allow(clippy::wrong_self_convention, clippy::too_many_arguments)] pub trait Connector: Send { type ProfileT: DeserializeOwned + Serialize; type TableT: DeserializeOwned + Serialize; @@ -170,6 +170,7 @@ pub trait ErasedConnector: Send { metadata_fields: Option>, ) -> anyhow::Result; + #[allow(clippy::too_many_arguments)] fn from_config( &self, id: Option, diff --git a/crates/arroyo-planner/src/tables.rs b/crates/arroyo-planner/src/tables.rs index c7faf1654..bfe12d0f6 100644 --- a/crates/arroyo-planner/src/tables.rs +++ b/crates/arroyo-planner/src/tables.rs @@ -214,6 +214,7 @@ impl From for ConnectorTable { } } +#[allow(clippy::too_many_arguments)] impl ConnectorTable { fn from_options( name: &str, From fca90ce5faa43172cea0c7c2fff2a47a1e641d1a Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw Date: Sun, 27 Oct 2024 22:58:14 +0530 Subject: [PATCH 21/36] adding structs for better readability --- .../src/filesystem/source.rs | 11 ++++++++-- crates/arroyo-connectors/src/fluvio/source.rs | 11 ++++++++-- .../arroyo-connectors/src/kafka/source/mod.rs | 12 ++++++++--- .../arroyo-connectors/src/kinesis/source.rs | 13 ++++++++---- .../arroyo-connectors/src/mqtt/source/mod.rs | 11 ++++++++-- .../arroyo-connectors/src/nats/source/mod.rs | 20 +++++++++++++++--- .../src/polling_http/operator.rs | 11 +++++++++- .../src/single_file/source.rs | 21 ++++++++++--------- crates/arroyo-connectors/src/sse/operator.rs | 11 ++++++++-- .../src/websocket/operator.rs | 11 ++++++++-- crates/arroyo-operator/src/context.rs | 20 ++++++++++++++++-- 11 files changed, 119 insertions(+), 33 deletions(-) diff --git a/crates/arroyo-connectors/src/filesystem/source.rs b/crates/arroyo-connectors/src/filesystem/source.rs index 48f547b23..b50a0f365 100644 --- a/crates/arroyo-connectors/src/filesystem/source.rs +++ b/crates/arroyo-connectors/src/filesystem/source.rs @@ -18,7 +18,7 @@ use futures::StreamExt; use parquet::arrow::async_reader::ParquetObjectReader; use parquet::arrow::ParquetRecordBatchStreamBuilder; -use arroyo_operator::context::ArrowContext; +use arroyo_operator::context::{ArrowContext, ConnectorMetadata}; use regex::Regex; use tokio::io::{AsyncBufReadExt, AsyncRead, BufReader}; use tokio::select; @@ -367,7 +367,14 @@ impl FileSystemSourceFunc { line = line_reader.next() => { match line.transpose()? { Some(line) => { - ctx.deserialize_slice(line.as_bytes(), SystemTime::now(), (false, 0, 0, "".to_string()), None).await?; + let connector_metadata = ConnectorMetadata { + enable_metadata: false, + message_offset: 0, + message_partition: 0, + message_topic: "".to_string(), + metadata_fields: None, + }; + ctx.deserialize_slice(line.as_bytes(), SystemTime::now(), connector_metadata).await?; records_read += 1; if ctx.should_flush() { ctx.flush_buffer().await?; diff --git a/crates/arroyo-connectors/src/fluvio/source.rs b/crates/arroyo-connectors/src/fluvio/source.rs index c11a78feb..fd06b2b5b 100644 --- a/crates/arroyo-connectors/src/fluvio/source.rs +++ b/crates/arroyo-connectors/src/fluvio/source.rs @@ -1,5 +1,5 @@ use anyhow::anyhow; -use arroyo_operator::context::ArrowContext; +use arroyo_operator::context::{ArrowContext, ConnectorMetadata}; use arroyo_operator::operator::SourceOperator; use arroyo_operator::SourceFinishType; use arroyo_rpc::formats::{BadData, Format, Framing}; @@ -166,7 +166,14 @@ impl FluvioSourceFunc { match message { Some((_, Ok(msg))) => { let timestamp = from_millis(msg.timestamp().max(0) as u64); - ctx.deserialize_slice(msg.value(), timestamp, (false, 0, 0, "".to_string()), None).await?; + let connector_metadata = ConnectorMetadata { + enable_metadata: false, + message_offset: 0, + message_partition: 0, + message_topic: "".to_string(), + metadata_fields: None, + }; + ctx.deserialize_slice(msg.value(), timestamp, connector_metadata).await?; if ctx.should_flush() { ctx.flush_buffer().await?; diff --git a/crates/arroyo-connectors/src/kafka/source/mod.rs b/crates/arroyo-connectors/src/kafka/source/mod.rs index 7c7ca8ff1..3993800df 100644 --- a/crates/arroyo-connectors/src/kafka/source/mod.rs +++ b/crates/arroyo-connectors/src/kafka/source/mod.rs @@ -3,7 +3,7 @@ use arroyo_rpc::grpc::rpc::TableConfig; use arroyo_rpc::schema_resolver::SchemaResolver; use arroyo_rpc::{grpc::rpc::StopMode, ControlMessage, ControlResp}; -use arroyo_operator::context::ArrowContext; +use arroyo_operator::context::{ArrowContext, ConnectorMetadata}; use arroyo_operator::operator::SourceOperator; use arroyo_operator::SourceFinishType; use arroyo_types::*; @@ -180,9 +180,15 @@ impl KafkaSourceFunc { .ok_or_else(|| UserError::new("Failed to read timestamp from Kafka record", "The message read from Kafka did not contain a message timestamp"))?; - let kafka_metadata = (self.enable_metadata.unwrap_or(false), msg.offset(), msg.partition(), self.topic.clone()); + let connector_metadata = ConnectorMetadata { + enable_metadata: self.enable_metadata.unwrap_or(false), + message_offset: msg.offset(), + message_partition: msg.partition(), + message_topic: self.topic.clone(), + metadata_fields: self.metadata_fields.clone(), + }; - ctx.deserialize_slice(v, from_millis(timestamp as u64), kafka_metadata, self.metadata_fields.clone()).await?; + ctx.deserialize_slice(v, from_millis(timestamp as u64), connector_metadata).await?; if ctx.should_flush() { ctx.flush_buffer().await?; diff --git a/crates/arroyo-connectors/src/kinesis/source.rs b/crates/arroyo-connectors/src/kinesis/source.rs index 240410f55..00299b1fc 100644 --- a/crates/arroyo-connectors/src/kinesis/source.rs +++ b/crates/arroyo-connectors/src/kinesis/source.rs @@ -7,7 +7,7 @@ use std::{ }; use anyhow::{anyhow, bail, Context as AnyhowContext, Result}; -use arroyo_operator::context::ArrowContext; +use arroyo_operator::context::{ArrowContext, ConnectorMetadata}; use arroyo_operator::operator::SourceOperator; use arroyo_operator::SourceFinishType; use arroyo_rpc::formats::{BadData, Format, Framing}; @@ -434,12 +434,17 @@ impl KinesisSourceFunc { for record in records { let data = record.data.into_inner(); let timestamp = record.approximate_arrival_timestamp.unwrap(); - + let connector_metadata = ConnectorMetadata { + enable_metadata: false, + message_offset: 0, + message_partition: 0, + message_topic: "".to_string(), + metadata_fields: None, + }; ctx.deserialize_slice( &data, from_nanos(timestamp.as_nanos() as u128), - (false, 0, 0, "".to_string()), - None, + connector_metadata, ) .await?; diff --git a/crates/arroyo-connectors/src/mqtt/source/mod.rs b/crates/arroyo-connectors/src/mqtt/source/mod.rs index 6da21e7db..fd90f02e4 100644 --- a/crates/arroyo-connectors/src/mqtt/source/mod.rs +++ b/crates/arroyo-connectors/src/mqtt/source/mod.rs @@ -14,7 +14,7 @@ use rumqttc::v5::{ConnectionError, Event as MqttEvent, Incoming}; use rumqttc::Outgoing; use crate::mqtt::{create_connection, MqttConfig}; -use arroyo_operator::context::ArrowContext; +use arroyo_operator::context::{ArrowContext, ConnectorMetadata}; use arroyo_operator::operator::SourceOperator; use arroyo_operator::SourceFinishType; use arroyo_rpc::grpc::rpc::TableConfig; @@ -143,7 +143,14 @@ impl MqttSourceFunc { event = eventloop.poll() => { match event { Ok(MqttEvent::Incoming(Incoming::Publish(p))) => { - ctx.deserialize_slice(&p.payload, SystemTime::now(), (false, 0, 0, "".to_string()), None).await?; + let connector_metadata = ConnectorMetadata { + enable_metadata: false, + message_offset: 0, + message_partition: 0, + message_topic: "".to_string(), + metadata_fields: None, + }; + ctx.deserialize_slice(&p.payload, SystemTime::now(), connector_metadata).await?; rate_limiter.until_ready().await; } Ok(MqttEvent::Outgoing(Outgoing::Subscribe(_))) => { diff --git a/crates/arroyo-connectors/src/nats/source/mod.rs b/crates/arroyo-connectors/src/nats/source/mod.rs index 5a246c0c4..cebb6c9cc 100644 --- a/crates/arroyo-connectors/src/nats/source/mod.rs +++ b/crates/arroyo-connectors/src/nats/source/mod.rs @@ -6,6 +6,7 @@ use super::NatsTable; use super::ReplayPolicy; use super::{get_nats_client, SourceType}; use arroyo_operator::context::ArrowContext; +use arroyo_operator::context::ConnectorMetadata; use arroyo_operator::operator::SourceOperator; use arroyo_operator::SourceFinishType; use arroyo_rpc::formats::BadData; @@ -366,8 +367,14 @@ impl NatsSourceFunc { let payload = msg.payload.as_ref(); let message_info = msg.info().expect("Couldn't get message information"); let timestamp = message_info.published.into() ; - - ctx.deserialize_slice(payload, timestamp, (false, 0, 0, "".to_string()), None).await?; + let connector_metadata = ConnectorMetadata { + enable_metadata: false, + message_offset: 0, + message_partition: 0, + message_topic: "".to_string(), + metadata_fields: None, + }; + ctx.deserialize_slice(payload, timestamp, connector_metadata).await?; debug!("---------------------------------------------->"); debug!( @@ -493,7 +500,14 @@ impl NatsSourceFunc { Some(msg) => { let payload = msg.payload.as_ref(); let timestamp = SystemTime::now(); - ctx.deserialize_slice(payload, timestamp, (false, 0, 0, "".to_string()), None).await?; + let connector_metadata = ConnectorMetadata { + enable_metadata: false, + message_offset: 0, + message_partition: 0, + message_topic: "".to_string(), + metadata_fields: None, + }; + ctx.deserialize_slice(payload, timestamp, connector_metadata).await?; if ctx.should_flush() { ctx.flush_buffer().await?; } diff --git a/crates/arroyo-connectors/src/polling_http/operator.rs b/crates/arroyo-connectors/src/polling_http/operator.rs index 733c18cc4..a7cb689a0 100644 --- a/crates/arroyo-connectors/src/polling_http/operator.rs +++ b/crates/arroyo-connectors/src/polling_http/operator.rs @@ -1,3 +1,4 @@ +use arroyo_operator::context::ConnectorMetadata; use async_trait::async_trait; use bincode::{Decode, Encode}; use bytes::Bytes; @@ -215,7 +216,15 @@ impl PollingHttpSourceFunc { continue; } - ctx.deserialize_slice(&buf, SystemTime::now(), (false, 0, 0, "".to_string()), None).await?; + let connector_metadata = ConnectorMetadata { + enable_metadata: false, + message_offset: 0, + message_partition: 0, + message_topic: "".to_string(), + metadata_fields: None, + }; + + ctx.deserialize_slice(&buf, SystemTime::now(), connector_metadata).await?; if ctx.should_flush() { ctx.flush_buffer().await?; diff --git a/crates/arroyo-connectors/src/single_file/source.rs b/crates/arroyo-connectors/src/single_file/source.rs index 690d19e11..0e0100a64 100644 --- a/crates/arroyo-connectors/src/single_file/source.rs +++ b/crates/arroyo-connectors/src/single_file/source.rs @@ -1,6 +1,6 @@ use std::{collections::HashMap, time::SystemTime}; -use arroyo_operator::context::ArrowContext; +use arroyo_operator::context::{ArrowContext, ConnectorMetadata}; use arroyo_operator::operator::SourceOperator; use arroyo_operator::SourceFinishType; use arroyo_rpc::{ @@ -51,15 +51,16 @@ impl SingleFileSourceFunc { i += 1; continue; } - - ctx.deserialize_slice( - s.as_bytes(), - SystemTime::now(), - (false, 0, 0, "".to_string()), - None, - ) - .await - .unwrap(); + let connector_metadata = ConnectorMetadata { + enable_metadata: false, + message_offset: 0, + message_partition: 0, + message_topic: "".to_string(), + metadata_fields: None, + }; + ctx.deserialize_slice(s.as_bytes(), SystemTime::now(), connector_metadata) + .await + .unwrap(); if ctx.should_flush() { ctx.flush_buffer().await.unwrap(); } diff --git a/crates/arroyo-connectors/src/sse/operator.rs b/crates/arroyo-connectors/src/sse/operator.rs index 2c9e233f2..40ba31cfc 100644 --- a/crates/arroyo-connectors/src/sse/operator.rs +++ b/crates/arroyo-connectors/src/sse/operator.rs @@ -1,5 +1,5 @@ use crate::sse::SseTable; -use arroyo_operator::context::ArrowContext; +use arroyo_operator::context::{ArrowContext, ConnectorMetadata}; use arroyo_operator::operator::{OperatorNode, SourceOperator}; use arroyo_operator::SourceFinishType; use arroyo_rpc::formats::{BadData, Format, Framing}; @@ -171,8 +171,15 @@ impl SSESourceFunc { } if events.is_empty() || events.contains(&event.event_type) { + let connector_metadata = ConnectorMetadata { + enable_metadata: false, + message_offset: 0, + message_partition: 0, + message_topic: "".to_string(), + metadata_fields: None, + }; ctx.deserialize_slice( - event.data.as_bytes(), SystemTime::now(), (false, 0, 0, "".to_string()), None).await?; + event.data.as_bytes(), SystemTime::now(), connector_metadata).await?; if ctx.should_flush() { ctx.flush_buffer().await?; diff --git a/crates/arroyo-connectors/src/websocket/operator.rs b/crates/arroyo-connectors/src/websocket/operator.rs index 394ff97f6..02e62c557 100644 --- a/crates/arroyo-connectors/src/websocket/operator.rs +++ b/crates/arroyo-connectors/src/websocket/operator.rs @@ -3,7 +3,7 @@ use std::collections::HashMap; use std::str::FromStr; use std::time::SystemTime; -use arroyo_operator::context::ArrowContext; +use arroyo_operator::context::{ArrowContext, ConnectorMetadata}; use arroyo_operator::operator::SourceOperator; use arroyo_operator::SourceFinishType; use arroyo_rpc::formats::{BadData, Format, Framing}; @@ -122,7 +122,14 @@ impl WebsocketSourceFunc { msg: &[u8], ctx: &mut ArrowContext, ) -> Result<(), UserError> { - ctx.deserialize_slice(msg, SystemTime::now(), (false, 0, 0, "".to_string()), None) + let connector_metadata = ConnectorMetadata { + enable_metadata: false, + message_offset: 0, + message_partition: 0, + message_topic: "".to_string(), + metadata_fields: None, + }; + ctx.deserialize_slice(msg, SystemTime::now(), connector_metadata) .await?; if ctx.should_flush() { diff --git a/crates/arroyo-operator/src/context.rs b/crates/arroyo-operator/src/context.rs index 2186631c0..fec8f1d44 100644 --- a/crates/arroyo-operator/src/context.rs +++ b/crates/arroyo-operator/src/context.rs @@ -258,6 +258,14 @@ pub struct ArrowContext { pub table_manager: TableManager, } +pub struct ConnectorMetadata { + pub enable_metadata: bool, + pub message_offset: i64, + pub message_partition: i32, + pub message_topic: String, + pub metadata_fields: Option>, +} + #[derive(Clone)] pub struct ErrorReporter { pub tx: Sender, @@ -670,8 +678,7 @@ impl ArrowContext { &mut self, msg: &[u8], time: SystemTime, - kafka_metadata: (bool, i64, i32, String), - metadata_fields: Option>, + connector_metadata: ConnectorMetadata, ) -> Result<(), UserError> { let deserializer = self .deserializer @@ -685,6 +692,15 @@ impl ArrowContext { .map(|t| ContextBuffer::new(t.schema.clone())); } + let kafka_metadata = ( + connector_metadata.enable_metadata, + connector_metadata.message_offset, + connector_metadata.message_partition, + connector_metadata.message_topic.clone(), + ); + + let metadata_fields = connector_metadata.metadata_fields; + let errors = deserializer .deserialize_slice( &mut self.buffer.as_mut().expect("no out schema").buffer, From 111e67c45d688402c06acc8a47e0e6da929b6a1d Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw Date: Sun, 27 Oct 2024 23:20:07 +0530 Subject: [PATCH 22/36] code refactor --- crates/arroyo-operator/src/context.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/crates/arroyo-operator/src/context.rs b/crates/arroyo-operator/src/context.rs index fec8f1d44..a2f5fefe9 100644 --- a/crates/arroyo-operator/src/context.rs +++ b/crates/arroyo-operator/src/context.rs @@ -698,7 +698,6 @@ impl ArrowContext { connector_metadata.message_partition, connector_metadata.message_topic.clone(), ); - let metadata_fields = connector_metadata.metadata_fields; let errors = deserializer From efb3fbbd6828df8e188dfbbe31bba88222887f2c Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw Date: Mon, 28 Oct 2024 17:08:17 +0530 Subject: [PATCH 23/36] code reformatting to address generic functionality --- crates/arroyo-api/src/pipelines.rs | 1 - crates/arroyo-connectors/src/blackhole/mod.rs | 15 +- crates/arroyo-connectors/src/confluent/mod.rs | 6 +- .../arroyo-connectors/src/filesystem/delta.rs | 7 +- .../arroyo-connectors/src/filesystem/mod.rs | 8 +- .../src/filesystem/source.rs | 11 +- crates/arroyo-connectors/src/fluvio/mod.rs | 7 +- crates/arroyo-connectors/src/fluvio/source.rs | 11 +- crates/arroyo-connectors/src/impulse/mod.rs | 6 +- crates/arroyo-connectors/src/kafka/mod.rs | 43 +-- .../arroyo-connectors/src/kafka/source/mod.rs | 30 +- .../src/kafka/source/test.rs | 1 - crates/arroyo-connectors/src/kinesis/mod.rs | 7 +- .../arroyo-connectors/src/kinesis/source.rs | 17 +- crates/arroyo-connectors/src/mqtt/mod.rs | 7 +- .../arroyo-connectors/src/mqtt/source/mod.rs | 11 +- crates/arroyo-connectors/src/nats/mod.rs | 7 +- .../arroyo-connectors/src/nats/source/mod.rs | 19 +- crates/arroyo-connectors/src/nexmark/mod.rs | 6 +- .../arroyo-connectors/src/polling_http/mod.rs | 6 +- .../src/polling_http/operator.rs | 11 +- crates/arroyo-connectors/src/preview/mod.rs | 5 +- crates/arroyo-connectors/src/redis/mod.rs | 6 +- .../arroyo-connectors/src/single_file/mod.rs | 6 +- .../src/single_file/source.rs | 11 +- crates/arroyo-connectors/src/sse/mod.rs | 6 +- crates/arroyo-connectors/src/sse/operator.rs | 11 +- crates/arroyo-connectors/src/stdout/mod.rs | 15 +- crates/arroyo-connectors/src/webhook/mod.rs | 7 +- crates/arroyo-connectors/src/websocket/mod.rs | 6 +- .../src/websocket/operator.rs | 12 +- crates/arroyo-formats/src/avro/de.rs | 8 +- crates/arroyo-formats/src/de.rs | 332 ++++++++---------- crates/arroyo-operator/src/connector.rs | 18 +- crates/arroyo-operator/src/context.rs | 23 +- crates/arroyo-planner/src/tables.rs | 79 ++--- crates/arroyo-planner/src/test/mod.rs | 1 - crates/arroyo-rpc/src/lib.rs | 6 +- 38 files changed, 258 insertions(+), 531 deletions(-) diff --git a/crates/arroyo-api/src/pipelines.rs b/crates/arroyo-api/src/pipelines.rs index a42d30ba7..29d501297 100644 --- a/crates/arroyo-api/src/pipelines.rs +++ b/crates/arroyo-api/src/pipelines.rs @@ -159,7 +159,6 @@ async fn compile_sql<'a>( .unwrap_or(json!({})), &table.config, Some(&table.schema), - Some(true), None, ) .map_err(log_and_map)?; diff --git a/crates/arroyo-connectors/src/blackhole/mod.rs b/crates/arroyo-connectors/src/blackhole/mod.rs index f9b8f16e3..f186128b5 100644 --- a/crates/arroyo-connectors/src/blackhole/mod.rs +++ b/crates/arroyo-connectors/src/blackhole/mod.rs @@ -78,18 +78,9 @@ impl Connector for BlackholeConnector { _options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, - _enable_metadata: Option, _metadata_fields: Option>, ) -> anyhow::Result { - self.from_config( - None, - name, - EmptyConfig {}, - EmptyConfig {}, - schema, - None, - None, - ) + self.from_config(None, name, EmptyConfig {}, EmptyConfig {}, schema, None) } fn from_config( @@ -99,7 +90,6 @@ impl Connector for BlackholeConnector { config: Self::ProfileT, table: Self::TableT, s: Option<&ConnectionSchema>, - _enable_metadata: Option, _metadata_fields: Option>, ) -> anyhow::Result { let description = "Blackhole".to_string(); @@ -111,8 +101,7 @@ impl Connector for BlackholeConnector { format: None, bad_data: None, framing: None, - enable_metadata: None, - metadata_fields: None, + additional_fields: None, }; Ok(Connection { diff --git a/crates/arroyo-connectors/src/confluent/mod.rs b/crates/arroyo-connectors/src/confluent/mod.rs index fdd215804..c3886cd92 100644 --- a/crates/arroyo-connectors/src/confluent/mod.rs +++ b/crates/arroyo-connectors/src/confluent/mod.rs @@ -161,7 +161,6 @@ impl Connector for ConfluentConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, profile: Option<&ConnectionProfile>, - _enable_metadata: Option, _metadata_fields: Option>, ) -> anyhow::Result { let connection = profile @@ -174,7 +173,7 @@ impl Connector for ConfluentConnector { let table = KafkaConnector::table_from_options(options)?; - self.from_config(None, name, connection, table, schema, None, None) + self.from_config(None, name, connection, table, schema, None) } fn from_config( @@ -184,13 +183,12 @@ impl Connector for ConfluentConnector { config: Self::ProfileT, mut table: Self::TableT, schema: Option<&ConnectionSchema>, - _enable_metadata: Option, _metadata_fields: Option>, ) -> anyhow::Result { table .client_configs .insert("client.id".to_string(), CLIENT_ID.to_string()); - KafkaConnector {}.from_config(id, name, config.into(), table, schema, None, None) + KafkaConnector {}.from_config(id, name, config.into(), table, schema, None) } fn make_operator( diff --git a/crates/arroyo-connectors/src/filesystem/delta.rs b/crates/arroyo-connectors/src/filesystem/delta.rs index c2c98a881..750ad1f45 100644 --- a/crates/arroyo-connectors/src/filesystem/delta.rs +++ b/crates/arroyo-connectors/src/filesystem/delta.rs @@ -77,7 +77,6 @@ impl Connector for DeltaLakeConnector { config: Self::ProfileT, table: Self::TableT, schema: Option<&ConnectionSchema>, - _enable_metadata: Option, _metadata_fields: Option>, ) -> anyhow::Result { let TableType::Sink { @@ -125,8 +124,7 @@ impl Connector for DeltaLakeConnector { format: Some(format), bad_data: schema.bad_data.clone(), framing: schema.framing.clone(), - enable_metadata: None, - metadata_fields: None, + additional_fields: None, }; Ok(Connection { @@ -146,12 +144,11 @@ impl Connector for DeltaLakeConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, - _enable_metadata: Option, _metadata_fields: Option>, ) -> anyhow::Result { let table = file_system_sink_from_options(options, schema, CommitStyle::DeltaLake)?; - self.from_config(None, name, EmptyConfig {}, table, schema, None, None) + self.from_config(None, name, EmptyConfig {}, table, schema, None) } fn make_operator( diff --git a/crates/arroyo-connectors/src/filesystem/mod.rs b/crates/arroyo-connectors/src/filesystem/mod.rs index 626ea3032..fdd66b37c 100644 --- a/crates/arroyo-connectors/src/filesystem/mod.rs +++ b/crates/arroyo-connectors/src/filesystem/mod.rs @@ -114,7 +114,6 @@ impl Connector for FileSystemConnector { config: Self::ProfileT, table: Self::TableT, schema: Option<&ConnectionSchema>, - _enable_metadata: Option, _metadata_fields: Option>, ) -> anyhow::Result { let (description, connection_type) = match table.table_type { @@ -170,8 +169,7 @@ impl Connector for FileSystemConnector { format: Some(format), bad_data: schema.bad_data.clone(), framing: schema.framing.clone(), - enable_metadata: None, - metadata_fields: None, + additional_fields: None, }; Ok(Connection { @@ -191,7 +189,6 @@ impl Connector for FileSystemConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, - _enable_metadata: Option, _metadata_fields: Option>, ) -> anyhow::Result { match options.remove("type") { @@ -217,13 +214,12 @@ impl Connector for FileSystemConnector { }, schema, None, - None, ) } Some(t) if t == "sink" => { let table = file_system_sink_from_options(options, schema, CommitStyle::Direct)?; - self.from_config(None, name, EmptyConfig {}, table, schema, None, None) + self.from_config(None, name, EmptyConfig {}, table, schema, None) } Some(t) => bail!("unknown type: {}", t), None => bail!("must have type set"), diff --git a/crates/arroyo-connectors/src/filesystem/source.rs b/crates/arroyo-connectors/src/filesystem/source.rs index b50a0f365..3bf43665d 100644 --- a/crates/arroyo-connectors/src/filesystem/source.rs +++ b/crates/arroyo-connectors/src/filesystem/source.rs @@ -18,7 +18,7 @@ use futures::StreamExt; use parquet::arrow::async_reader::ParquetObjectReader; use parquet::arrow::ParquetRecordBatchStreamBuilder; -use arroyo_operator::context::{ArrowContext, ConnectorMetadata}; +use arroyo_operator::context::ArrowContext; use regex::Regex; use tokio::io::{AsyncBufReadExt, AsyncRead, BufReader}; use tokio::select; @@ -367,14 +367,7 @@ impl FileSystemSourceFunc { line = line_reader.next() => { match line.transpose()? { Some(line) => { - let connector_metadata = ConnectorMetadata { - enable_metadata: false, - message_offset: 0, - message_partition: 0, - message_topic: "".to_string(), - metadata_fields: None, - }; - ctx.deserialize_slice(line.as_bytes(), SystemTime::now(), connector_metadata).await?; + ctx.deserialize_slice(line.as_bytes(), SystemTime::now(), None).await?; records_read += 1; if ctx.should_flush() { ctx.flush_buffer().await?; diff --git a/crates/arroyo-connectors/src/fluvio/mod.rs b/crates/arroyo-connectors/src/fluvio/mod.rs index 006f1ebf2..911197368 100644 --- a/crates/arroyo-connectors/src/fluvio/mod.rs +++ b/crates/arroyo-connectors/src/fluvio/mod.rs @@ -88,7 +88,6 @@ impl Connector for FluvioConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, - _enable_metadata: Option, _metadata_fields: Option>, ) -> anyhow::Result { let endpoint = options.remove("endpoint"); @@ -118,7 +117,7 @@ impl Connector for FluvioConnector { type_: table_type, }; - Self::from_config(self, None, name, EmptyConfig {}, table, schema, None, None) + Self::from_config(self, None, name, EmptyConfig {}, table, schema, None) } fn from_config( @@ -128,7 +127,6 @@ impl Connector for FluvioConnector { config: EmptyConfig, table: FluvioTable, schema: Option<&ConnectionSchema>, - _enable_metadata: Option, _metadata_fields: Option>, ) -> anyhow::Result { let (typ, desc) = match table.type_ { @@ -158,8 +156,7 @@ impl Connector for FluvioConnector { format: Some(format), bad_data: schema.bad_data.clone(), framing: schema.framing.clone(), - enable_metadata: None, - metadata_fields: None, + additional_fields: None, }; Ok(Connection { diff --git a/crates/arroyo-connectors/src/fluvio/source.rs b/crates/arroyo-connectors/src/fluvio/source.rs index fd06b2b5b..4e7468a78 100644 --- a/crates/arroyo-connectors/src/fluvio/source.rs +++ b/crates/arroyo-connectors/src/fluvio/source.rs @@ -1,5 +1,5 @@ use anyhow::anyhow; -use arroyo_operator::context::{ArrowContext, ConnectorMetadata}; +use arroyo_operator::context::ArrowContext; use arroyo_operator::operator::SourceOperator; use arroyo_operator::SourceFinishType; use arroyo_rpc::formats::{BadData, Format, Framing}; @@ -166,14 +166,7 @@ impl FluvioSourceFunc { match message { Some((_, Ok(msg))) => { let timestamp = from_millis(msg.timestamp().max(0) as u64); - let connector_metadata = ConnectorMetadata { - enable_metadata: false, - message_offset: 0, - message_partition: 0, - message_topic: "".to_string(), - metadata_fields: None, - }; - ctx.deserialize_slice(msg.value(), timestamp, connector_metadata).await?; + ctx.deserialize_slice(msg.value(), timestamp, None).await?; if ctx.should_flush() { ctx.flush_buffer().await?; diff --git a/crates/arroyo-connectors/src/impulse/mod.rs b/crates/arroyo-connectors/src/impulse/mod.rs index 94739ae68..b1163de08 100644 --- a/crates/arroyo-connectors/src/impulse/mod.rs +++ b/crates/arroyo-connectors/src/impulse/mod.rs @@ -101,7 +101,6 @@ impl Connector for ImpulseConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, - _enable_metadata: Option, _metadata_fields: Option>, ) -> anyhow::Result { let event_rate = f64::from_str(&pull_opt("event_rate", options)?) @@ -137,7 +136,6 @@ impl Connector for ImpulseConnector { }, None, None, - None, ) } @@ -148,7 +146,6 @@ impl Connector for ImpulseConnector { config: Self::ProfileT, table: Self::TableT, _: Option<&ConnectionSchema>, - _enable_metadata: Option, _metadata_fields: Option>, ) -> anyhow::Result { let description = format!( @@ -172,8 +169,7 @@ impl Connector for ImpulseConnector { format: None, bad_data: None, framing: None, - enable_metadata: None, - metadata_fields: None, + additional_fields: None, }; Ok(Connection { diff --git a/crates/arroyo-connectors/src/kafka/mod.rs b/crates/arroyo-connectors/src/kafka/mod.rs index d19439e90..db7f001bc 100644 --- a/crates/arroyo-connectors/src/kafka/mod.rs +++ b/crates/arroyo-connectors/src/kafka/mod.rs @@ -188,7 +188,6 @@ impl Connector for KafkaConnector { config: KafkaConfig, table: KafkaTable, schema: Option<&ConnectionSchema>, - enable_metadata: Option, metadata_fields: Option>, ) -> anyhow::Result { let (typ, desc) = match table.type_ { @@ -216,8 +215,7 @@ impl Connector for KafkaConnector { format: Some(format), bad_data: schema.bad_data.clone(), framing: schema.framing.clone(), - enable_metadata, - metadata_fields, + additional_fields: metadata_fields, }; Ok(Connection { @@ -316,7 +314,6 @@ impl Connector for KafkaConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, profile: Option<&ConnectionProfile>, - enable_metadata: Option, metadata_fields: Option>, ) -> anyhow::Result { let connection = profile @@ -329,16 +326,7 @@ impl Connector for KafkaConnector { let table = Self::table_from_options(options)?; - Self::from_config( - self, - None, - name, - connection, - table, - schema, - enable_metadata, - metadata_fields, - ) + Self::from_config(self, None, name, connection, table, schema, metadata_fields) } fn make_operator( @@ -398,8 +386,7 @@ impl Connector for KafkaConnector { .unwrap_or(u32::MAX), ) .unwrap(), - enable_metadata: config.enable_metadata, - metadata_fields: config.metadata_fields, + metadata_fields: config.additional_fields, }))) } TableType::Sink { @@ -639,13 +626,7 @@ impl KafkaTester { let mut builders = aschema.builders(); let mut error = deserializer - .deserialize_slice( - &mut builders, - &msg, - SystemTime::now(), - (false, 0, 0, "".to_string()), - None, - ) + .deserialize_slice(&mut builders, &msg, SystemTime::now(), None) .await .into_iter() .next(); @@ -667,13 +648,7 @@ impl KafkaTester { let mut builders = aschema.builders(); let mut error = deserializer - .deserialize_slice( - &mut builders, - &msg, - SystemTime::now(), - (false, 0, 0, "".to_string()), - None, - ) + .deserialize_slice(&mut builders, &msg, SystemTime::now(), None) .await .into_iter() .next(); @@ -707,13 +682,7 @@ impl KafkaTester { let mut builders = aschema.builders(); let mut error = deserializer - .deserialize_slice( - &mut builders, - &msg, - SystemTime::now(), - (false, 0, 0, "".to_string()), - None, - ) + .deserialize_slice(&mut builders, &msg, SystemTime::now(), None) .await .into_iter() .next(); diff --git a/crates/arroyo-connectors/src/kafka/source/mod.rs b/crates/arroyo-connectors/src/kafka/source/mod.rs index 3993800df..9d2a4fe2f 100644 --- a/crates/arroyo-connectors/src/kafka/source/mod.rs +++ b/crates/arroyo-connectors/src/kafka/source/mod.rs @@ -1,9 +1,10 @@ +use arroyo_formats::de::FieldValueType; use arroyo_rpc::formats::{BadData, Format, Framing}; use arroyo_rpc::grpc::rpc::TableConfig; use arroyo_rpc::schema_resolver::SchemaResolver; use arroyo_rpc::{grpc::rpc::StopMode, ControlMessage, ControlResp}; -use arroyo_operator::context::{ArrowContext, ConnectorMetadata}; +use arroyo_operator::context::ArrowContext; use arroyo_operator::operator::SourceOperator; use arroyo_operator::SourceFinishType; use arroyo_types::*; @@ -35,7 +36,6 @@ pub struct KafkaSourceFunc { pub schema_resolver: Option>, pub client_configs: HashMap, pub messages_per_second: NonZeroU32, - pub enable_metadata: Option, pub metadata_fields: Option>, } @@ -180,16 +180,30 @@ impl KafkaSourceFunc { .ok_or_else(|| UserError::new("Failed to read timestamp from Kafka record", "The message read from Kafka did not contain a message timestamp"))?; - let connector_metadata = ConnectorMetadata { - enable_metadata: self.enable_metadata.unwrap_or(false), - message_offset: msg.offset(), - message_partition: msg.partition(), - message_topic: self.topic.clone(), - metadata_fields: self.metadata_fields.clone(), + let connector_metadata = if self.metadata_fields.is_some() { + let mut connector_metadata = HashMap::new(); + for (key, value) in self.metadata_fields.as_ref().unwrap() { + match value.as_str() { + "offset_id" => { + connector_metadata.insert(key, FieldValueType::Int64(msg.offset())); + } + "partition_id" => { + connector_metadata.insert(key, FieldValueType::Int32(msg.partition())); + } + "topic" => { + connector_metadata.insert(key, FieldValueType::String(&self.topic)); + } + _ => {} + } + } + Some(connector_metadata) + } else { + None }; ctx.deserialize_slice(v, from_millis(timestamp as u64), connector_metadata).await?; + if ctx.should_flush() { ctx.flush_buffer().await?; } diff --git a/crates/arroyo-connectors/src/kafka/source/test.rs b/crates/arroyo-connectors/src/kafka/source/test.rs index 31b8187e7..19453ecc4 100644 --- a/crates/arroyo-connectors/src/kafka/source/test.rs +++ b/crates/arroyo-connectors/src/kafka/source/test.rs @@ -87,7 +87,6 @@ impl KafkaTopicTester { schema_resolver: None, client_configs: HashMap::new(), messages_per_second: NonZeroU32::new(100).unwrap(), - enable_metadata: Some(false), metadata_fields: None, }); diff --git a/crates/arroyo-connectors/src/kinesis/mod.rs b/crates/arroyo-connectors/src/kinesis/mod.rs index ae01f23ec..af7f8fdd5 100644 --- a/crates/arroyo-connectors/src/kinesis/mod.rs +++ b/crates/arroyo-connectors/src/kinesis/mod.rs @@ -83,7 +83,6 @@ impl Connector for KinesisConnector { config: Self::ProfileT, table: Self::TableT, schema: Option<&ConnectionSchema>, - _enable_metadata: Option, _metadata_fields: Option>, ) -> anyhow::Result { let (connection_type, description) = match table.type_ { @@ -113,8 +112,7 @@ impl Connector for KinesisConnector { format: Some(format), bad_data: schema.bad_data.clone(), framing: schema.framing.clone(), - enable_metadata: None, - metadata_fields: None, + additional_fields: None, }; Ok(Connection { @@ -134,7 +132,6 @@ impl Connector for KinesisConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, - _enable_metadata: Option, _metadata_fields: Option>, ) -> anyhow::Result { let typ = pull_opt("type", options)?; @@ -172,7 +169,7 @@ impl Connector for KinesisConnector { aws_region: options.remove("aws_region").map(|s| s.to_string()), }; - Self::from_config(self, None, name, EmptyConfig {}, table, schema, None, None) + Self::from_config(self, None, name, EmptyConfig {}, table, schema, None) } fn make_operator( diff --git a/crates/arroyo-connectors/src/kinesis/source.rs b/crates/arroyo-connectors/src/kinesis/source.rs index 00299b1fc..39890d575 100644 --- a/crates/arroyo-connectors/src/kinesis/source.rs +++ b/crates/arroyo-connectors/src/kinesis/source.rs @@ -7,7 +7,7 @@ use std::{ }; use anyhow::{anyhow, bail, Context as AnyhowContext, Result}; -use arroyo_operator::context::{ArrowContext, ConnectorMetadata}; +use arroyo_operator::context::ArrowContext; use arroyo_operator::operator::SourceOperator; use arroyo_operator::SourceFinishType; use arroyo_rpc::formats::{BadData, Format, Framing}; @@ -434,19 +434,8 @@ impl KinesisSourceFunc { for record in records { let data = record.data.into_inner(); let timestamp = record.approximate_arrival_timestamp.unwrap(); - let connector_metadata = ConnectorMetadata { - enable_metadata: false, - message_offset: 0, - message_partition: 0, - message_topic: "".to_string(), - metadata_fields: None, - }; - ctx.deserialize_slice( - &data, - from_nanos(timestamp.as_nanos() as u128), - connector_metadata, - ) - .await?; + ctx.deserialize_slice(&data, from_nanos(timestamp.as_nanos() as u128), None) + .await?; if ctx.should_flush() { ctx.flush_buffer().await? diff --git a/crates/arroyo-connectors/src/mqtt/mod.rs b/crates/arroyo-connectors/src/mqtt/mod.rs index 592c87854..9f77108bc 100644 --- a/crates/arroyo-connectors/src/mqtt/mod.rs +++ b/crates/arroyo-connectors/src/mqtt/mod.rs @@ -156,7 +156,6 @@ impl Connector for MqttConnector { config: MqttConfig, table: MqttTable, schema: Option<&ConnectionSchema>, - _enable_metadata: Option, _metadata_fields: Option>, ) -> anyhow::Result { let (typ, desc) = match table.type_ { @@ -184,8 +183,7 @@ impl Connector for MqttConnector { format: Some(format), bad_data: schema.bad_data.clone(), framing: schema.framing.clone(), - enable_metadata: None, - metadata_fields: None, + additional_fields: None, }; Ok(Connection { @@ -246,7 +244,6 @@ impl Connector for MqttConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, profile: Option<&ConnectionProfile>, - _enable_metadata: Option, _metadata_fields: Option>, ) -> anyhow::Result { let connection = profile @@ -259,7 +256,7 @@ impl Connector for MqttConnector { let table = Self::table_from_options(options)?; - Self::from_config(self, None, name, connection, table, schema, None, None) + Self::from_config(self, None, name, connection, table, schema, None) } fn make_operator( diff --git a/crates/arroyo-connectors/src/mqtt/source/mod.rs b/crates/arroyo-connectors/src/mqtt/source/mod.rs index fd90f02e4..485ee5560 100644 --- a/crates/arroyo-connectors/src/mqtt/source/mod.rs +++ b/crates/arroyo-connectors/src/mqtt/source/mod.rs @@ -14,7 +14,7 @@ use rumqttc::v5::{ConnectionError, Event as MqttEvent, Incoming}; use rumqttc::Outgoing; use crate::mqtt::{create_connection, MqttConfig}; -use arroyo_operator::context::{ArrowContext, ConnectorMetadata}; +use arroyo_operator::context::ArrowContext; use arroyo_operator::operator::SourceOperator; use arroyo_operator::SourceFinishType; use arroyo_rpc::grpc::rpc::TableConfig; @@ -143,14 +143,7 @@ impl MqttSourceFunc { event = eventloop.poll() => { match event { Ok(MqttEvent::Incoming(Incoming::Publish(p))) => { - let connector_metadata = ConnectorMetadata { - enable_metadata: false, - message_offset: 0, - message_partition: 0, - message_topic: "".to_string(), - metadata_fields: None, - }; - ctx.deserialize_slice(&p.payload, SystemTime::now(), connector_metadata).await?; + ctx.deserialize_slice(&p.payload, SystemTime::now(), None).await?; rate_limiter.until_ready().await; } Ok(MqttEvent::Outgoing(Outgoing::Subscribe(_))) => { diff --git a/crates/arroyo-connectors/src/nats/mod.rs b/crates/arroyo-connectors/src/nats/mod.rs index 71b7c12b7..603aedd05 100644 --- a/crates/arroyo-connectors/src/nats/mod.rs +++ b/crates/arroyo-connectors/src/nats/mod.rs @@ -246,7 +246,6 @@ impl Connector for NatsConnector { config: NatsConfig, table: NatsTable, schema: Option<&ConnectionSchema>, - _enable_metadata: Option, _metadata_fields: Option>, ) -> anyhow::Result { let stream_or_subject = match &table.connector_type { @@ -297,8 +296,7 @@ impl Connector for NatsConnector { format: Some(format), bad_data: schema.bad_data.clone(), framing: schema.framing.clone(), - enable_metadata: None, - metadata_fields: None, + additional_fields: None, }; Ok(Connection { @@ -318,7 +316,6 @@ impl Connector for NatsConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, profile: Option<&ConnectionProfile>, - _enable_metadata: Option, _metadata_fields: Option>, ) -> anyhow::Result { let connection = profile @@ -331,7 +328,7 @@ impl Connector for NatsConnector { let table = Self::table_from_options(options)?; - Self::from_config(self, None, name, connection, table, schema, None, None) + Self::from_config(self, None, name, connection, table, schema, None) } fn make_operator( diff --git a/crates/arroyo-connectors/src/nats/source/mod.rs b/crates/arroyo-connectors/src/nats/source/mod.rs index cebb6c9cc..9a81aee41 100644 --- a/crates/arroyo-connectors/src/nats/source/mod.rs +++ b/crates/arroyo-connectors/src/nats/source/mod.rs @@ -6,7 +6,6 @@ use super::NatsTable; use super::ReplayPolicy; use super::{get_nats_client, SourceType}; use arroyo_operator::context::ArrowContext; -use arroyo_operator::context::ConnectorMetadata; use arroyo_operator::operator::SourceOperator; use arroyo_operator::SourceFinishType; use arroyo_rpc::formats::BadData; @@ -367,14 +366,7 @@ impl NatsSourceFunc { let payload = msg.payload.as_ref(); let message_info = msg.info().expect("Couldn't get message information"); let timestamp = message_info.published.into() ; - let connector_metadata = ConnectorMetadata { - enable_metadata: false, - message_offset: 0, - message_partition: 0, - message_topic: "".to_string(), - metadata_fields: None, - }; - ctx.deserialize_slice(payload, timestamp, connector_metadata).await?; + ctx.deserialize_slice(payload, timestamp, None).await?; debug!("---------------------------------------------->"); debug!( @@ -500,14 +492,7 @@ impl NatsSourceFunc { Some(msg) => { let payload = msg.payload.as_ref(); let timestamp = SystemTime::now(); - let connector_metadata = ConnectorMetadata { - enable_metadata: false, - message_offset: 0, - message_partition: 0, - message_topic: "".to_string(), - metadata_fields: None, - }; - ctx.deserialize_slice(payload, timestamp, connector_metadata).await?; + ctx.deserialize_slice(payload, timestamp, None).await?; if ctx.should_flush() { ctx.flush_buffer().await?; } diff --git a/crates/arroyo-connectors/src/nexmark/mod.rs b/crates/arroyo-connectors/src/nexmark/mod.rs index c3d9f4e21..17029d144 100644 --- a/crates/arroyo-connectors/src/nexmark/mod.rs +++ b/crates/arroyo-connectors/src/nexmark/mod.rs @@ -158,7 +158,6 @@ impl Connector for NexmarkConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, - _enable_metadata: Option, _metadata_fields: Option>, ) -> anyhow::Result { let event_rate = f64::from_str(&pull_opt("event_rate", options)?) @@ -186,7 +185,6 @@ impl Connector for NexmarkConnector { }, None, None, - None, ) } @@ -197,7 +195,6 @@ impl Connector for NexmarkConnector { config: Self::ProfileT, table: Self::TableT, _: Option<&ConnectionSchema>, - _enable_metadata: Option, _metadata_fields: Option>, ) -> anyhow::Result { let description = format!( @@ -217,8 +214,7 @@ impl Connector for NexmarkConnector { format: None, bad_data: None, framing: None, - enable_metadata: None, - metadata_fields: None, + additional_fields: None, }; Ok(Connection { diff --git a/crates/arroyo-connectors/src/polling_http/mod.rs b/crates/arroyo-connectors/src/polling_http/mod.rs index 3fc53f190..cb17a2ccf 100644 --- a/crates/arroyo-connectors/src/polling_http/mod.rs +++ b/crates/arroyo-connectors/src/polling_http/mod.rs @@ -152,7 +152,6 @@ impl Connector for PollingHTTPConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, - _enable_metadata: Option, _metadata_fields: Option>, ) -> anyhow::Result { let endpoint = pull_opt("endpoint", options)?; @@ -186,7 +185,6 @@ impl Connector for PollingHTTPConnector { }, schema, None, - None, ) } @@ -197,7 +195,6 @@ impl Connector for PollingHTTPConnector { config: Self::ProfileT, table: Self::TableT, schema: Option<&ConnectionSchema>, - _enable_metadata: Option, _metadata_fields: Option>, ) -> anyhow::Result { let description = format!("PollingHTTPSource<{}>", table.endpoint); @@ -228,8 +225,7 @@ impl Connector for PollingHTTPConnector { format: Some(format), bad_data: schema.bad_data.clone(), framing: schema.framing.clone(), - enable_metadata: None, - metadata_fields: None, + additional_fields: None, }; Ok(Connection { diff --git a/crates/arroyo-connectors/src/polling_http/operator.rs b/crates/arroyo-connectors/src/polling_http/operator.rs index a7cb689a0..ec150cebd 100644 --- a/crates/arroyo-connectors/src/polling_http/operator.rs +++ b/crates/arroyo-connectors/src/polling_http/operator.rs @@ -1,4 +1,3 @@ -use arroyo_operator::context::ConnectorMetadata; use async_trait::async_trait; use bincode::{Decode, Encode}; use bytes::Bytes; @@ -216,15 +215,7 @@ impl PollingHttpSourceFunc { continue; } - let connector_metadata = ConnectorMetadata { - enable_metadata: false, - message_offset: 0, - message_partition: 0, - message_topic: "".to_string(), - metadata_fields: None, - }; - - ctx.deserialize_slice(&buf, SystemTime::now(), connector_metadata).await?; + ctx.deserialize_slice(&buf, SystemTime::now(), None).await?; if ctx.should_flush() { ctx.flush_buffer().await?; diff --git a/crates/arroyo-connectors/src/preview/mod.rs b/crates/arroyo-connectors/src/preview/mod.rs index 4675d09f0..0a25be91c 100644 --- a/crates/arroyo-connectors/src/preview/mod.rs +++ b/crates/arroyo-connectors/src/preview/mod.rs @@ -71,7 +71,6 @@ impl Connector for PreviewConnector { _: &mut HashMap, _: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, - _enable_metadata: Option, _metadata_fields: Option>, ) -> anyhow::Result { bail!("Preview connector cannot be created in SQL"); @@ -84,7 +83,6 @@ impl Connector for PreviewConnector { config: Self::ProfileT, table: Self::TableT, schema: Option<&ConnectionSchema>, - _enable_metadata: Option, _metadata_fields: Option>, ) -> anyhow::Result { let description = "PreviewSink".to_string(); @@ -100,8 +98,7 @@ impl Connector for PreviewConnector { format: None, bad_data: schema.bad_data.clone(), framing: schema.framing.clone(), - enable_metadata: None, - metadata_fields: None, + additional_fields: None, }; Ok(Connection { diff --git a/crates/arroyo-connectors/src/redis/mod.rs b/crates/arroyo-connectors/src/redis/mod.rs index ceb4f00f6..eb9addf07 100644 --- a/crates/arroyo-connectors/src/redis/mod.rs +++ b/crates/arroyo-connectors/src/redis/mod.rs @@ -227,7 +227,6 @@ impl Connector for RedisConnector { options: &mut HashMap, s: Option<&ConnectionSchema>, profile: Option<&ConnectionProfile>, - _enable_metadata: Option, _metadata_fields: Option>, ) -> anyhow::Result { let connection_config = match profile { @@ -351,7 +350,6 @@ impl Connector for RedisConnector { }, s, None, - None, ) } @@ -362,7 +360,6 @@ impl Connector for RedisConnector { config: Self::ProfileT, table: Self::TableT, schema: Option<&ConnectionSchema>, - _enable_metadata: Option, _metadata_fields: Option>, ) -> anyhow::Result { let schema = schema @@ -384,8 +381,7 @@ impl Connector for RedisConnector { format: Some(format), bad_data: schema.bad_data.clone(), framing: schema.framing.clone(), - enable_metadata: None, - metadata_fields: None, + additional_fields: None, }; Ok(Connection { diff --git a/crates/arroyo-connectors/src/single_file/mod.rs b/crates/arroyo-connectors/src/single_file/mod.rs index 01793ca24..cb25b9d64 100644 --- a/crates/arroyo-connectors/src/single_file/mod.rs +++ b/crates/arroyo-connectors/src/single_file/mod.rs @@ -84,7 +84,6 @@ impl Connector for SingleFileConnector { config: Self::ProfileT, table: Self::TableT, schema: Option<&ConnectionSchema>, - _enable_metadata: Option, _metadata_fields: Option>, ) -> anyhow::Result { let schema = schema @@ -105,8 +104,7 @@ impl Connector for SingleFileConnector { format: Some(format), bad_data: schema.bad_data.clone(), framing: schema.framing.clone(), - enable_metadata: None, - metadata_fields: None, + additional_fields: None, }; Ok(Connection { @@ -126,7 +124,6 @@ impl Connector for SingleFileConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, - _enable_metadata: Option, _metadata_fields: Option>, ) -> anyhow::Result { let path = pull_opt("path", options)?; @@ -153,7 +150,6 @@ impl Connector for SingleFileConnector { }, schema, None, - None, ) } diff --git a/crates/arroyo-connectors/src/single_file/source.rs b/crates/arroyo-connectors/src/single_file/source.rs index 0e0100a64..3ea8063a5 100644 --- a/crates/arroyo-connectors/src/single_file/source.rs +++ b/crates/arroyo-connectors/src/single_file/source.rs @@ -1,6 +1,6 @@ use std::{collections::HashMap, time::SystemTime}; -use arroyo_operator::context::{ArrowContext, ConnectorMetadata}; +use arroyo_operator::context::ArrowContext; use arroyo_operator::operator::SourceOperator; use arroyo_operator::SourceFinishType; use arroyo_rpc::{ @@ -51,14 +51,7 @@ impl SingleFileSourceFunc { i += 1; continue; } - let connector_metadata = ConnectorMetadata { - enable_metadata: false, - message_offset: 0, - message_partition: 0, - message_topic: "".to_string(), - metadata_fields: None, - }; - ctx.deserialize_slice(s.as_bytes(), SystemTime::now(), connector_metadata) + ctx.deserialize_slice(s.as_bytes(), SystemTime::now(), None) .await .unwrap(); if ctx.should_flush() { diff --git a/crates/arroyo-connectors/src/sse/mod.rs b/crates/arroyo-connectors/src/sse/mod.rs index fa1a3502a..88a29142b 100644 --- a/crates/arroyo-connectors/src/sse/mod.rs +++ b/crates/arroyo-connectors/src/sse/mod.rs @@ -78,7 +78,6 @@ impl Connector for SSEConnector { config: Self::ProfileT, table: Self::TableT, schema: Option<&ConnectionSchema>, - _enable_metadata: Option, _metadata_fields: Option>, ) -> anyhow::Result { let description = format!("SSESource<{}>", table.endpoint); @@ -109,8 +108,7 @@ impl Connector for SSEConnector { format: Some(format), bad_data: schema.bad_data.clone(), framing: schema.framing.clone(), - enable_metadata: None, - metadata_fields: None, + additional_fields: None, }; Ok(Connection { @@ -130,7 +128,6 @@ impl Connector for SSEConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, - _enable_metadata: Option, _metadata_fields: Option>, ) -> anyhow::Result { let endpoint = pull_opt("endpoint", options)?; @@ -148,7 +145,6 @@ impl Connector for SSEConnector { }, schema, None, - None, ) } diff --git a/crates/arroyo-connectors/src/sse/operator.rs b/crates/arroyo-connectors/src/sse/operator.rs index 40ba31cfc..80bbebdd1 100644 --- a/crates/arroyo-connectors/src/sse/operator.rs +++ b/crates/arroyo-connectors/src/sse/operator.rs @@ -1,5 +1,5 @@ use crate::sse::SseTable; -use arroyo_operator::context::{ArrowContext, ConnectorMetadata}; +use arroyo_operator::context::ArrowContext; use arroyo_operator::operator::{OperatorNode, SourceOperator}; use arroyo_operator::SourceFinishType; use arroyo_rpc::formats::{BadData, Format, Framing}; @@ -171,15 +171,8 @@ impl SSESourceFunc { } if events.is_empty() || events.contains(&event.event_type) { - let connector_metadata = ConnectorMetadata { - enable_metadata: false, - message_offset: 0, - message_partition: 0, - message_topic: "".to_string(), - metadata_fields: None, - }; ctx.deserialize_slice( - event.data.as_bytes(), SystemTime::now(), connector_metadata).await?; + event.data.as_bytes(), SystemTime::now(), None).await?; if ctx.should_flush() { ctx.flush_buffer().await?; diff --git a/crates/arroyo-connectors/src/stdout/mod.rs b/crates/arroyo-connectors/src/stdout/mod.rs index 3e0a34d3c..1460c9e92 100644 --- a/crates/arroyo-connectors/src/stdout/mod.rs +++ b/crates/arroyo-connectors/src/stdout/mod.rs @@ -76,18 +76,9 @@ impl Connector for StdoutConnector { _options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, - _enable_metadata: Option, _metadata_fields: Option>, ) -> anyhow::Result { - self.from_config( - None, - name, - EmptyConfig {}, - EmptyConfig {}, - schema, - None, - None, - ) + self.from_config(None, name, EmptyConfig {}, EmptyConfig {}, schema, None) } fn from_config( @@ -97,7 +88,6 @@ impl Connector for StdoutConnector { config: Self::ProfileT, table: Self::TableT, schema: Option<&ConnectionSchema>, - _enable_metadata: Option, _metadata_fields: Option>, ) -> anyhow::Result { let description = "StdoutSink".to_string(); @@ -119,8 +109,7 @@ impl Connector for StdoutConnector { format: Some(format), bad_data: schema.bad_data.clone(), framing: schema.framing.clone(), - enable_metadata: None, - metadata_fields: None, + additional_fields: None, }; Ok(Connection { diff --git a/crates/arroyo-connectors/src/webhook/mod.rs b/crates/arroyo-connectors/src/webhook/mod.rs index c8718b9f9..e8d513251 100644 --- a/crates/arroyo-connectors/src/webhook/mod.rs +++ b/crates/arroyo-connectors/src/webhook/mod.rs @@ -142,7 +142,6 @@ impl Connector for WebhookConnector { config: Self::ProfileT, table: Self::TableT, schema: Option<&ConnectionSchema>, - _enable_metadata: Option, _metadata_fields: Option>, ) -> anyhow::Result { let description = format!("WebhookSink<{}>", table.endpoint.sub_env_vars()?); @@ -164,8 +163,7 @@ impl Connector for WebhookConnector { format: Some(format), bad_data: schema.bad_data.clone(), framing: schema.framing.clone(), - enable_metadata: None, - metadata_fields: None, + additional_fields: None, }; Ok(Connection { @@ -185,7 +183,6 @@ impl Connector for WebhookConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, - _enable_metadata: Option, _metadata_fields: Option>, ) -> anyhow::Result { let endpoint = pull_opt("endpoint", options)?; @@ -207,7 +204,7 @@ impl Connector for WebhookConnector { )?; let _ = Self::construct_test_request(&client, &table)?; - self.from_config(None, name, EmptyConfig {}, table, schema, None, None) + self.from_config(None, name, EmptyConfig {}, table, schema, None) } fn make_operator( diff --git a/crates/arroyo-connectors/src/websocket/mod.rs b/crates/arroyo-connectors/src/websocket/mod.rs index 425445755..e2e5f65c8 100644 --- a/crates/arroyo-connectors/src/websocket/mod.rs +++ b/crates/arroyo-connectors/src/websocket/mod.rs @@ -220,7 +220,6 @@ impl Connector for WebsocketConnector { config: Self::ProfileT, table: Self::TableT, schema: Option<&ConnectionSchema>, - _enable_metadata: Option, _metadata_fields: Option>, ) -> anyhow::Result { let description = format!("WebsocketSource<{}>", table.endpoint); @@ -251,8 +250,7 @@ impl Connector for WebsocketConnector { format: Some(format), bad_data: schema.bad_data.clone(), framing: schema.framing.clone(), - enable_metadata: None, - metadata_fields: None, + additional_fields: None, }; Ok(Connection { @@ -272,7 +270,6 @@ impl Connector for WebsocketConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, - _enable_metadata: Option, _metadata_fields: Option>, ) -> anyhow::Result { let endpoint = pull_opt("endpoint", options)?; @@ -311,7 +308,6 @@ impl Connector for WebsocketConnector { }, schema, None, - None, ) } diff --git a/crates/arroyo-connectors/src/websocket/operator.rs b/crates/arroyo-connectors/src/websocket/operator.rs index 02e62c557..c9c41b073 100644 --- a/crates/arroyo-connectors/src/websocket/operator.rs +++ b/crates/arroyo-connectors/src/websocket/operator.rs @@ -3,7 +3,7 @@ use std::collections::HashMap; use std::str::FromStr; use std::time::SystemTime; -use arroyo_operator::context::{ArrowContext, ConnectorMetadata}; +use arroyo_operator::context::ArrowContext; use arroyo_operator::operator::SourceOperator; use arroyo_operator::SourceFinishType; use arroyo_rpc::formats::{BadData, Format, Framing}; @@ -122,15 +122,7 @@ impl WebsocketSourceFunc { msg: &[u8], ctx: &mut ArrowContext, ) -> Result<(), UserError> { - let connector_metadata = ConnectorMetadata { - enable_metadata: false, - message_offset: 0, - message_partition: 0, - message_topic: "".to_string(), - metadata_fields: None, - }; - ctx.deserialize_slice(msg, SystemTime::now(), connector_metadata) - .await?; + ctx.deserialize_slice(msg, SystemTime::now(), None).await?; if ctx.should_flush() { ctx.flush_buffer().await?; diff --git a/crates/arroyo-formats/src/avro/de.rs b/crates/arroyo-formats/src/avro/de.rs index f1bd54975..f371bf70c 100644 --- a/crates/arroyo-formats/src/avro/de.rs +++ b/crates/arroyo-formats/src/avro/de.rs @@ -281,13 +281,7 @@ mod tests { deserializer_with_schema(format.clone(), writer_schema); let errors = deserializer - .deserialize_slice( - &mut builders, - message, - SystemTime::now(), - (false, 0, 0, "".to_string()), - None, - ) + .deserialize_slice(&mut builders, message, SystemTime::now(), None) .await; assert_eq!(errors, vec![]); diff --git a/crates/arroyo-formats/src/de.rs b/crates/arroyo-formats/src/de.rs index 904f21848..186585750 100644 --- a/crates/arroyo-formats/src/de.rs +++ b/crates/arroyo-formats/src/de.rs @@ -21,6 +21,14 @@ use std::sync::Arc; use std::time::{Instant, SystemTime}; use tokio::sync::Mutex; +#[derive(Debug, Clone)] +pub enum FieldValueType<'a> { + Int64(i64), + Int32(i32), + String(&'a String), + // Extend with more types as needed +} + pub struct FramingIterator<'a> { framing: Option>, buf: &'a [u8], @@ -83,9 +91,7 @@ pub struct ArrowDeserializer { schema_registry: Arc>>, proto_pool: DescriptorPool, schema_resolver: Arc, - enable_metadata: Option, - kafka_metadata_builder: Option<(Int64Builder, Int32Builder, StringBuilder)>, - metadata_fields: Option>, + additional_fields_builder: Option>>, } impl ArrowDeserializer { @@ -162,9 +168,7 @@ impl ArrowDeserializer { proto_pool, buffered_count: 0, buffered_since: Instant::now(), - enable_metadata: Some(false), - kafka_metadata_builder: None, - metadata_fields: None, + additional_fields_builder: None, } } @@ -173,21 +177,12 @@ impl ArrowDeserializer { buffer: &mut [Box], msg: &[u8], timestamp: SystemTime, - kafka_metadata: (bool, i64, i32, String), - metadata_fields: Option>, + additional_fields: Option>>, ) -> Vec { match &*self.format { Format::Avro(_) => self.deserialize_slice_avro(buffer, msg, timestamp).await, _ => FramingIterator::new(self.framing.clone(), msg) - .map(|t| { - self.deserialize_single( - buffer, - t, - timestamp, - kafka_metadata.clone(), - metadata_fields.clone(), - ) - }) + .map(|t| self.deserialize_single(buffer, t, timestamp, additional_fields.clone())) .filter_map(|t| t.err()) .collect(), } @@ -213,52 +208,14 @@ impl ArrowDeserializer { let mut columns = batch.columns().to_vec(); columns.insert(self.schema.timestamp_index, Arc::new(timestamp.finish())); - if self.enable_metadata.unwrap_or(false) { - if let Some((offset_builder, partition_builder, topic_builder)) = - &mut self.kafka_metadata_builder + if self.additional_fields_builder.is_some() { + for (field_name, mut builder) in + self.additional_fields_builder.take().unwrap() { - if let Some(fields) = &self.metadata_fields { - for (field_name, argument_name) in fields.iter() { - match argument_name.as_str() { - "topic" => { - if let Some((idx, _)) = - self.schema.schema.column_with_name(field_name) - { - columns.remove(idx); - columns.insert( - idx, - Arc::new(topic_builder.finish()), - ); - } - } - "partition" => { - if let Some((idx, _)) = - self.schema.schema.column_with_name(field_name) - { - columns.remove(idx); - columns.insert( - idx, - Arc::new(partition_builder.finish()), - ); - } - } - "offset_id" => { - if let Some((idx, _)) = - self.schema.schema.column_with_name(field_name) - { - columns.remove(idx); - columns.insert( - idx, - Arc::new(offset_builder.finish()), - ); - } - } - _ => { - // Handle unexpected argument names or log a message if necessary - } - } - } - } + let (idx, _) = + self.schema.schema.column_with_name(&field_name).unwrap(); + columns.remove(idx); + columns.insert(idx, Arc::new(builder.as_mut().finish())); } } RecordBatch::try_new(self.schema.schema.clone(), columns).unwrap() @@ -291,10 +248,8 @@ impl ArrowDeserializer { buffer: &mut [Box], msg: &[u8], timestamp: SystemTime, - kafka_metadata: (bool, i64, i32, String), - metadata_fields: Option>, + additional_fields: Option>, ) -> Result<(), SourceError> { - self.metadata_fields = metadata_fields; match &*self.format { Format::RawString(_) | Format::Json(JsonFormat { @@ -302,31 +257,19 @@ impl ArrowDeserializer { }) => { self.deserialize_raw_string(buffer, msg); add_timestamp(buffer, self.schema.timestamp_index, timestamp); - if kafka_metadata.0 { - self.enable_metadata = Some(true); - add_kafka_metadata( - buffer, - &self.schema, - &kafka_metadata.3, - kafka_metadata.2, - kafka_metadata.1, - self.metadata_fields.clone(), - ); + if additional_fields.is_some() { + for (k, v) in additional_fields.unwrap().iter() { + add_additional_fields(buffer, &self.schema, k, v); + } } } Format::RawBytes(_) => { self.deserialize_raw_bytes(buffer, msg); add_timestamp(buffer, self.schema.timestamp_index, timestamp); - if kafka_metadata.0 { - self.enable_metadata = Some(true); - add_kafka_metadata( - buffer, - &self.schema, - &kafka_metadata.3, - kafka_metadata.2, - kafka_metadata.1, - self.metadata_fields.clone(), - ); + if additional_fields.is_some() { + for (k, v) in additional_fields.unwrap().iter() { + add_additional_fields(buffer, &self.schema, k, v); + } } } Format::Json(json) => { @@ -340,28 +283,61 @@ impl ArrowDeserializer { panic!("json decoder not initialized"); }; - if kafka_metadata.0 { - self.enable_metadata = Some(true); - self.kafka_metadata_builder.get_or_insert_with(|| { - ( - Int64Builder::new(), - Int32Builder::new(), - StringBuilder::new(), - ) - }); + if self.additional_fields_builder.is_none() { + if let Some(fields) = additional_fields.as_ref() { + let mut builders = HashMap::new(); + for (key, value) in fields.iter() { + let builder: Box = match value { + FieldValueType::Int32(_) => Box::new(Int32Builder::new()), + FieldValueType::Int64(_) => Box::new(Int64Builder::new()), + FieldValueType::String(_) => Box::new(StringBuilder::new()), + }; + builders.insert(key, builder); + } + self.additional_fields_builder = Some( + builders + .into_iter() + .map(|(k, v)| ((*k).clone(), v)) + .collect(), + ); + } } decoder .decode(msg) .map_err(|e| SourceError::bad_data(format!("invalid JSON: {:?}", e)))?; timestamp_builder.append_value(to_nanos(timestamp) as i64); - if kafka_metadata.0 { - if let Some((offset_builder, partition_builder, topic_builder)) = - &mut self.kafka_metadata_builder - { - offset_builder.append_value(kafka_metadata.1); - partition_builder.append_value(kafka_metadata.2); - topic_builder.append_value(kafka_metadata.3.clone()); + if let Some(additional_fields) = additional_fields { + for (k, v) in additional_fields.iter() { + if let Some(builder) = self + .additional_fields_builder + .as_mut() + .and_then(|b| b.get_mut(k.to_owned())) + { + match v { + FieldValueType::Int32(i) => { + builder + .as_any_mut() + .downcast_mut::() + .expect("additional field has incorrect type") + .append_value(*i); + } + FieldValueType::Int64(i) => { + builder + .as_any_mut() + .downcast_mut::() + .expect("additional field has incorrect type") + .append_value(*i); + } + FieldValueType::String(s) => { + builder + .as_any_mut() + .downcast_mut::() + .expect("additional field has incorrect type") + .append_value(s); + } + } + } } } self.buffered_count += 1; @@ -380,15 +356,39 @@ impl ArrowDeserializer { .decode(json.to_string().as_bytes()) .map_err(|e| SourceError::bad_data(format!("invalid JSON: {:?}", e)))?; timestamp_builder.append_value(to_nanos(timestamp) as i64); - if kafka_metadata.0 { - add_kafka_metadata( - buffer, - &self.schema, - &kafka_metadata.3, - kafka_metadata.2, - kafka_metadata.1, - self.metadata_fields.clone(), - ); + + if self.additional_fields_builder.is_some() { + for (k, v) in additional_fields.unwrap().iter() { + if let Some(builder) = self + .additional_fields_builder + .as_mut() + .and_then(|b| b.get_mut(k.to_owned())) + { + match v { + FieldValueType::Int32(i) => { + builder + .as_any_mut() + .downcast_mut::() + .expect("additional field has incorrect type") + .append_value(*i); + } + FieldValueType::Int64(i) => { + builder + .as_any_mut() + .downcast_mut::() + .expect("additional field has incorrect type") + .append_value(*i); + } + FieldValueType::String(s) => { + builder + .as_any_mut() + .downcast_mut::() + .expect("additional field has incorrect type") + .append_value(s); + } + } + } + } } self.buffered_count += 1; } @@ -521,73 +521,37 @@ pub(crate) fn add_timestamp( .append_value(to_nanos(timestamp) as i64); } -pub(crate) fn add_kafka_metadata_partition( - builder: &mut [Box], - idx: usize, - partition: i32, -) { - builder[idx] - .as_any_mut() - .downcast_mut::() - .expect("kafka_metadata.partition column has incorrect type") - .append_value(partition); -} - -pub(crate) fn add_kafka_metadata_offset( - builder: &mut [Box], - idx: usize, - offset: i64, -) { - builder[idx] - .as_any_mut() - .downcast_mut::() - .expect("kafka_metadata.offset column has incorrect type") - .append_value(offset); -} - -pub(crate) fn add_kafka_metadata_topic( - builder: &mut [Box], - idx: usize, - topic: &str, -) { - builder[idx] - .as_any_mut() - .downcast_mut::() - .expect("kafka_metadata.topic column has incorrect type") - .append_value(topic) -} - -pub(crate) fn add_kafka_metadata( +pub(crate) fn add_additional_fields( builder: &mut [Box], schema: &ArroyoSchema, - topic: &str, - partition: i32, - offset: i64, - metadata_fields: Option>, + key: &str, + value: &FieldValueType<'_>, ) { - if let Some(fields) = metadata_fields { - for (field_name, argument_name) in fields.iter() { - // Match each argument name and add the corresponding metadata if found - match argument_name.as_str() { - "topic" => { - if let Some((idx, _)) = schema.schema.column_with_name(field_name) { - add_kafka_metadata_topic(builder, idx, topic); - } - } - "partition" => { - if let Some((idx, _)) = schema.schema.column_with_name(field_name) { - add_kafka_metadata_partition(builder, idx, partition); - } - } - "offset_id" => { - if let Some((idx, _)) = schema.schema.column_with_name(field_name) { - add_kafka_metadata_offset(builder, idx, offset); - } - } - _ => { - // Handle any unexpected argument names or log a message if necessary - } - } + let (idx, _) = schema + .schema + .column_with_name(key) + .expect("no 'value' column for additional fields"); + match value { + FieldValueType::Int32(i) => { + builder[idx] + .as_any_mut() + .downcast_mut::() + .expect("additional field has incorrect type") + .append_value(*i); + } + FieldValueType::Int64(i) => { + builder[idx] + .as_any_mut() + .downcast_mut::() + .expect("additional field has incorrect type") + .append_value(*i); + } + FieldValueType::String(s) => { + builder[idx] + .as_any_mut() + .downcast_mut::() + .expect("additional field has incorrect type") + .append_value(s); } } } @@ -727,8 +691,7 @@ mod tests { &mut arrays[..], json!({ "x": 5 }).to_string().as_bytes(), now, - (false, 0, 0, "".to_string()), - None + None, ) .await, vec![] @@ -739,8 +702,7 @@ mod tests { &mut arrays[..], json!({ "x": "hello" }).to_string().as_bytes(), now, - (false, 0, 0, "".to_string()), - None + None, ) .await, vec![] @@ -767,8 +729,7 @@ mod tests { &mut arrays[..], json!({ "x": 5 }).to_string().as_bytes(), SystemTime::now(), - (false, 0, 0, "".to_string()), - None + None, ) .await, vec![] @@ -779,8 +740,7 @@ mod tests { &mut arrays[..], json!({ "x": "hello" }).to_string().as_bytes(), SystemTime::now(), - (false, 0, 0, "".to_string()), - None + None, ) .await, vec![] @@ -819,13 +779,7 @@ mod tests { let time = SystemTime::now(); let result = deserializer - .deserialize_slice( - &mut arrays, - &[0, 1, 2, 3, 4, 5], - time, - (false, 0, 0, "".to_string()), - None, - ) + .deserialize_slice(&mut arrays, &[0, 1, 2, 3, 4, 5], time, None) .await; assert!(result.is_empty()); diff --git a/crates/arroyo-operator/src/connector.rs b/crates/arroyo-operator/src/connector.rs index da2ff708e..5247e1040 100644 --- a/crates/arroyo-operator/src/connector.rs +++ b/crates/arroyo-operator/src/connector.rs @@ -22,7 +22,7 @@ pub struct Connection { pub description: String, } -#[allow(clippy::wrong_self_convention, clippy::too_many_arguments)] +#[allow(clippy::wrong_self_convention)] pub trait Connector: Send { type ProfileT: DeserializeOwned + Serialize; type TableT: DeserializeOwned + Serialize; @@ -89,7 +89,6 @@ pub trait Connector: Send { options: &mut HashMap, schema: Option<&ConnectionSchema>, profile: Option<&ConnectionProfile>, - enable_metadata: Option, metadata_fields: Option>, ) -> anyhow::Result; @@ -100,7 +99,6 @@ pub trait Connector: Send { config: Self::ProfileT, table: Self::TableT, schema: Option<&ConnectionSchema>, - enable_metadata: Option, metadata_fields: Option>, ) -> anyhow::Result; @@ -166,7 +164,6 @@ pub trait ErasedConnector: Send { options: &mut HashMap, schema: Option<&ConnectionSchema>, profile: Option<&ConnectionProfile>, - enable_metadata: Option, metadata_fields: Option>, ) -> anyhow::Result; @@ -178,7 +175,6 @@ pub trait ErasedConnector: Send { config: &serde_json::Value, table: &serde_json::Value, schema: Option<&ConnectionSchema>, - enable_metadata: Option, metadata_fields: Option>, ) -> anyhow::Result; @@ -265,17 +261,9 @@ impl ErasedConnector for C { options: &mut HashMap, schema: Option<&ConnectionSchema>, profile: Option<&ConnectionProfile>, - enable_metadata: Option, metadata_fields: Option>, ) -> anyhow::Result { - self.from_options( - name, - options, - schema, - profile, - enable_metadata, - metadata_fields, - ) + self.from_options(name, options, schema, profile, metadata_fields) } fn from_config( @@ -285,7 +273,6 @@ impl ErasedConnector for C { config: &serde_json::Value, table: &serde_json::Value, schema: Option<&ConnectionSchema>, - enable_metadata: Option, metadata_fields: Option>, ) -> anyhow::Result { self.from_config( @@ -294,7 +281,6 @@ impl ErasedConnector for C { self.parse_config(config)?, self.parse_table(table)?, schema, - enable_metadata, metadata_fields, ) } diff --git a/crates/arroyo-operator/src/context.rs b/crates/arroyo-operator/src/context.rs index a2f5fefe9..cf0534e32 100644 --- a/crates/arroyo-operator/src/context.rs +++ b/crates/arroyo-operator/src/context.rs @@ -2,7 +2,7 @@ use crate::{server_for_hash_array, RateLimiter}; use arrow::array::{make_builder, Array, ArrayBuilder, PrimitiveArray, RecordBatch}; use arrow::compute::{partition, sort_to_indices, take}; use arrow::datatypes::{SchemaRef, UInt64Type}; -use arroyo_formats::de::ArrowDeserializer; +use arroyo_formats::de::{ArrowDeserializer, FieldValueType}; use arroyo_formats::should_flush; use arroyo_metrics::{register_queue_gauge, QueueGauges, TaskCounters}; use arroyo_rpc::config::config; @@ -258,14 +258,6 @@ pub struct ArrowContext { pub table_manager: TableManager, } -pub struct ConnectorMetadata { - pub enable_metadata: bool, - pub message_offset: i64, - pub message_partition: i32, - pub message_topic: String, - pub metadata_fields: Option>, -} - #[derive(Clone)] pub struct ErrorReporter { pub tx: Sender, @@ -678,7 +670,7 @@ impl ArrowContext { &mut self, msg: &[u8], time: SystemTime, - connector_metadata: ConnectorMetadata, + additional_fields: Option>>, ) -> Result<(), UserError> { let deserializer = self .deserializer @@ -692,21 +684,12 @@ impl ArrowContext { .map(|t| ContextBuffer::new(t.schema.clone())); } - let kafka_metadata = ( - connector_metadata.enable_metadata, - connector_metadata.message_offset, - connector_metadata.message_partition, - connector_metadata.message_topic.clone(), - ); - let metadata_fields = connector_metadata.metadata_fields; - let errors = deserializer .deserialize_slice( &mut self.buffer.as_mut().expect("no out schema").buffer, msg, time, - kafka_metadata, - metadata_fields, + additional_fields, ) .await; self.collect_source_errors(errors).await?; diff --git a/crates/arroyo-planner/src/tables.rs b/crates/arroyo-planner/src/tables.rs index bfe12d0f6..cd9de51df 100644 --- a/crates/arroyo-planner/src/tables.rs +++ b/crates/arroyo-planner/src/tables.rs @@ -223,7 +223,6 @@ impl ConnectorTable { primary_keys: Vec, options: &mut HashMap, connection_profile: Option<&ConnectionProfile>, - enable_metadata: Option, connector_metadata_columns: Option>, ) -> Result { // TODO: a more principled way of letting connectors dictate types to use @@ -305,7 +304,6 @@ impl ConnectorTable { options, Some(&schema), connection_profile, - enable_metadata, connector_metadata_columns, ) .map_err(|e| DataFusionError::Plan(e.to_string()))?; @@ -534,55 +532,45 @@ impl Table { Field::new(name, data_type, nullable), ); - let generating_expression = - column.options.iter().find_map(|option| { - if let ColumnOption::Generated { - generation_expr, .. - } = &option.option + let generating_expression = column.options.iter().find_map(|option| { + if let ColumnOption::Generated { + generation_expr, .. + } = &option.option + { + if let Some(sqlparser::ast::Expr::Function(sqlparser::ast::Function { + name, + args, + .. + })) = generation_expr { - if let Some(sqlparser::ast::Expr::Function( - sqlparser::ast::Function { name, args, .. }, - )) = generation_expr + if name + .0 + .iter() + .any(|ident| ident.value.to_lowercase() == "metadata") { - if name - .0 - .iter() - .any(|ident| ident.value.to_lowercase() == "metadata") - { - match args { - FunctionArguments::List(arg_list) => { - match arg_list.args.first() { - Some(FunctionArg::Unnamed( - sqlparser::ast::FunctionArgExpr::Expr( - sqlparser::ast::Expr::Value( - sqlparser::ast::Value::SingleQuotedString( - value, - ), - ), - ), - )) => { - connector_metadata_columns.insert( - column.name.value.to_string(), - value.to_string(), - ); - } - _ => { - "Unsupported argument format.".to_string(); - } - } - } - _ => { - "Unsupported argument format.".to_string(); - } + if let FunctionArguments::List(arg_list) = args { + if let Some(FunctionArg::Unnamed( + sqlparser::ast::FunctionArgExpr::Expr( + sqlparser::ast::Expr::Value( + sqlparser::ast::Value::SingleQuotedString(value), + ), + ), + )) = arg_list.args.first() + { + connector_metadata_columns.insert( + column.name.value.to_string(), + value.to_string(), + ); + return None; } - return None; } } - generation_expr.clone() - } else { - None } - }); + generation_expr.clone() + } else { + None + } + }); Ok((struct_field, generating_expression)) }) .collect::>>()?; @@ -727,7 +715,6 @@ impl Table { primary_keys, &mut with_map, connection_profile, - Some(true), connector_metadata_columns, ) .map_err(|e| e.context(format!("Failed to create table {}", name)))?, diff --git a/crates/arroyo-planner/src/test/mod.rs b/crates/arroyo-planner/src/test/mod.rs index b48d0f96d..111a59f00 100644 --- a/crates/arroyo-planner/src/test/mod.rs +++ b/crates/arroyo-planner/src/test/mod.rs @@ -24,7 +24,6 @@ fn get_test_schema_provider() -> ArroyoSchemaProvider { runtime: Some(10.0 * 1_000_000.0), }, None, - Some(false), None, ) .unwrap(); diff --git a/crates/arroyo-rpc/src/lib.rs b/crates/arroyo-rpc/src/lib.rs index 1c75367af..21058b0c1 100644 --- a/crates/arroyo-rpc/src/lib.rs +++ b/crates/arroyo-rpc/src/lib.rs @@ -191,8 +191,7 @@ pub struct OperatorConfig { pub bad_data: Option, pub framing: Option, pub rate_limit: Option, - pub enable_metadata: Option, - pub metadata_fields: Option>, + pub additional_fields: Option>, } impl Default for OperatorConfig { @@ -204,8 +203,7 @@ impl Default for OperatorConfig { bad_data: None, framing: None, rate_limit: None, - enable_metadata: None, - metadata_fields: None, + additional_fields: None, } } } From 3f3fe6197dbd98fc3b737ae2658aa7e51ff571db Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw Date: Mon, 28 Oct 2024 17:44:45 +0530 Subject: [PATCH 24/36] bug fix --- crates/arroyo-connectors/src/kafka/source/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/arroyo-connectors/src/kafka/source/mod.rs b/crates/arroyo-connectors/src/kafka/source/mod.rs index 9d2a4fe2f..6b14d6e13 100644 --- a/crates/arroyo-connectors/src/kafka/source/mod.rs +++ b/crates/arroyo-connectors/src/kafka/source/mod.rs @@ -187,7 +187,7 @@ impl KafkaSourceFunc { "offset_id" => { connector_metadata.insert(key, FieldValueType::Int64(msg.offset())); } - "partition_id" => { + "partition" => { connector_metadata.insert(key, FieldValueType::Int32(msg.partition())); } "topic" => { From 65ea8968f6920873fc1255a60db410f702149a09 Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw Date: Mon, 28 Oct 2024 20:51:41 +0530 Subject: [PATCH 25/36] remove clippy warn hints --- crates/arroyo-operator/src/connector.rs | 1 - crates/arroyo-planner/src/tables.rs | 1 - 2 files changed, 2 deletions(-) diff --git a/crates/arroyo-operator/src/connector.rs b/crates/arroyo-operator/src/connector.rs index 5247e1040..dfc6cf2d4 100644 --- a/crates/arroyo-operator/src/connector.rs +++ b/crates/arroyo-operator/src/connector.rs @@ -167,7 +167,6 @@ pub trait ErasedConnector: Send { metadata_fields: Option>, ) -> anyhow::Result; - #[allow(clippy::too_many_arguments)] fn from_config( &self, id: Option, diff --git a/crates/arroyo-planner/src/tables.rs b/crates/arroyo-planner/src/tables.rs index cd9de51df..54b1f4897 100644 --- a/crates/arroyo-planner/src/tables.rs +++ b/crates/arroyo-planner/src/tables.rs @@ -214,7 +214,6 @@ impl From for ConnectorTable { } } -#[allow(clippy::too_many_arguments)] impl ConnectorTable { fn from_options( name: &str, From 737536af2abfa8ba1e9cfae2b14345827e3c16d3 Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw Date: Mon, 28 Oct 2024 21:47:18 +0530 Subject: [PATCH 26/36] adding validations during planning --- crates/arroyo-connectors/src/kafka/mod.rs | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/crates/arroyo-connectors/src/kafka/mod.rs b/crates/arroyo-connectors/src/kafka/mod.rs index db7f001bc..aa07f364f 100644 --- a/crates/arroyo-connectors/src/kafka/mod.rs +++ b/crates/arroyo-connectors/src/kafka/mod.rs @@ -326,6 +326,18 @@ impl Connector for KafkaConnector { let table = Self::table_from_options(options)?; + let allowed_metadata_udf_args = ["offset_id", "partition", "topic"]; + if let Some(fields) = &metadata_fields { + for key in fields.keys() { + if !allowed_metadata_udf_args.contains(&key.as_str()) { + return Err(anyhow!( + "Invalid metadata field key for kafka connector: '{}'", + key + )); + } + } + } + Self::from_config(self, None, name, connection, table, schema, metadata_fields) } From 6e047e8336e7c96aac9fc1abfd070115773e8bf6 Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw Date: Tue, 29 Oct 2024 11:03:38 +0530 Subject: [PATCH 27/36] checking metadata columns have only a metadata function in themC --- crates/arroyo-connectors/src/kafka/mod.rs | 8 ++++---- crates/arroyo-planner/src/tables.rs | 13 ++++++++----- 2 files changed, 12 insertions(+), 9 deletions(-) diff --git a/crates/arroyo-connectors/src/kafka/mod.rs b/crates/arroyo-connectors/src/kafka/mod.rs index aa07f364f..bc2b479a8 100644 --- a/crates/arroyo-connectors/src/kafka/mod.rs +++ b/crates/arroyo-connectors/src/kafka/mod.rs @@ -328,11 +328,11 @@ impl Connector for KafkaConnector { let allowed_metadata_udf_args = ["offset_id", "partition", "topic"]; if let Some(fields) = &metadata_fields { - for key in fields.keys() { - if !allowed_metadata_udf_args.contains(&key.as_str()) { + for val in fields.values() { + if !allowed_metadata_udf_args.contains(&val.as_str()) { return Err(anyhow!( - "Invalid metadata field key for kafka connector: '{}'", - key + "Invalid metadata field val for kafka connector: '{}'", + val )); } } diff --git a/crates/arroyo-planner/src/tables.rs b/crates/arroyo-planner/src/tables.rs index 54b1f4897..0808beb81 100644 --- a/crates/arroyo-planner/src/tables.rs +++ b/crates/arroyo-planner/src/tables.rs @@ -101,15 +101,18 @@ impl FieldSpec { } fn is_metadata_virtual(&self) -> bool { match self { - FieldSpec::VirtualField { expression, .. } => { - if let Expr::ScalarFunction(ScalarFunction { func, .. }) = expression { - return func.name() == "metadata"; - } - false + FieldSpec::VirtualField { + expression: Expr::ScalarFunction(ScalarFunction { func, args, .. }), + .. + } => { + func.name() == "metadata" + && args.len() == 1 + && matches!(args.first(), Some(Expr::Literal(_))) } _ => false, } } + } impl From for FieldSpec { From c468d00fbba0cb4d1f148fcf074923cbbc97aebc Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw Date: Tue, 29 Oct 2024 11:12:22 +0530 Subject: [PATCH 28/36] simplifying using if let in kafka connector --- crates/arroyo-connectors/src/kafka/source/mod.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/crates/arroyo-connectors/src/kafka/source/mod.rs b/crates/arroyo-connectors/src/kafka/source/mod.rs index 6b14d6e13..f95c4a479 100644 --- a/crates/arroyo-connectors/src/kafka/source/mod.rs +++ b/crates/arroyo-connectors/src/kafka/source/mod.rs @@ -180,9 +180,9 @@ impl KafkaSourceFunc { .ok_or_else(|| UserError::new("Failed to read timestamp from Kafka record", "The message read from Kafka did not contain a message timestamp"))?; - let connector_metadata = if self.metadata_fields.is_some() { + let connector_metadata = if let Some(metadata_fields) = &self.metadata_fields { let mut connector_metadata = HashMap::new(); - for (key, value) in self.metadata_fields.as_ref().unwrap() { + for (key, value) in metadata_fields { match value.as_str() { "offset_id" => { connector_metadata.insert(key, FieldValueType::Int64(msg.offset())); @@ -201,6 +201,7 @@ impl KafkaSourceFunc { None }; + ctx.deserialize_slice(v, from_millis(timestamp as u64), connector_metadata).await?; From 2dfe40419a14f59ad9098772f89ec6ab083e7e50 Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw Date: Tue, 29 Oct 2024 11:18:27 +0530 Subject: [PATCH 29/36] fix clippy errors --- crates/arroyo-planner/src/tables.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/crates/arroyo-planner/src/tables.rs b/crates/arroyo-planner/src/tables.rs index 0808beb81..017c29557 100644 --- a/crates/arroyo-planner/src/tables.rs +++ b/crates/arroyo-planner/src/tables.rs @@ -112,7 +112,6 @@ impl FieldSpec { _ => false, } } - } impl From for FieldSpec { From 16e76efb24e6b4060480320302f2946c65a05a81 Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw Date: Tue, 29 Oct 2024 11:23:50 +0530 Subject: [PATCH 30/36] code reformating for if let deserializer --- crates/arroyo-formats/src/de.rs | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/crates/arroyo-formats/src/de.rs b/crates/arroyo-formats/src/de.rs index 186585750..7edceace2 100644 --- a/crates/arroyo-formats/src/de.rs +++ b/crates/arroyo-formats/src/de.rs @@ -208,16 +208,16 @@ impl ArrowDeserializer { let mut columns = batch.columns().to_vec(); columns.insert(self.schema.timestamp_index, Arc::new(timestamp.finish())); - if self.additional_fields_builder.is_some() { - for (field_name, mut builder) in - self.additional_fields_builder.take().unwrap() - { - let (idx, _) = - self.schema.schema.column_with_name(&field_name).unwrap(); - columns.remove(idx); - columns.insert(idx, Arc::new(builder.as_mut().finish())); + if let Some(additional_fields) = self.additional_fields_builder.take() { + for (field_name, mut builder) in additional_fields { + if let Some((idx, _)) = + self.schema.schema.column_with_name(&field_name) + { + columns[idx] = Arc::new(builder.as_mut().finish()); + } } } + RecordBatch::try_new(self.schema.schema.clone(), columns).unwrap() }), ), From b3cd088a06df56a4b2ea7c4bdc6e01f338e09c1e Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw Date: Tue, 29 Oct 2024 11:27:41 +0530 Subject: [PATCH 31/36] adding if let in deserialize_single fn --- crates/arroyo-formats/src/de.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/crates/arroyo-formats/src/de.rs b/crates/arroyo-formats/src/de.rs index 7edceace2..2de65ed42 100644 --- a/crates/arroyo-formats/src/de.rs +++ b/crates/arroyo-formats/src/de.rs @@ -257,8 +257,8 @@ impl ArrowDeserializer { }) => { self.deserialize_raw_string(buffer, msg); add_timestamp(buffer, self.schema.timestamp_index, timestamp); - if additional_fields.is_some() { - for (k, v) in additional_fields.unwrap().iter() { + if let Some(fields) = additional_fields { + for (k, v) in fields.iter() { add_additional_fields(buffer, &self.schema, k, v); } } @@ -266,8 +266,8 @@ impl ArrowDeserializer { Format::RawBytes(_) => { self.deserialize_raw_bytes(buffer, msg); add_timestamp(buffer, self.schema.timestamp_index, timestamp); - if additional_fields.is_some() { - for (k, v) in additional_fields.unwrap().iter() { + if let Some(fields) = additional_fields { + for (k, v) in fields.iter() { add_additional_fields(buffer, &self.schema, k, v); } } From bcff0f0c1778cbad285b85f0bea49b0c22c7fdcd Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw Date: Tue, 29 Oct 2024 11:57:17 +0530 Subject: [PATCH 32/36] code reformat --- crates/arroyo-formats/src/de.rs | 114 ++++++++++++++------------------ 1 file changed, 48 insertions(+), 66 deletions(-) diff --git a/crates/arroyo-formats/src/de.rs b/crates/arroyo-formats/src/de.rs index 2de65ed42..4c7f50f0e 100644 --- a/crates/arroyo-formats/src/de.rs +++ b/crates/arroyo-formats/src/de.rs @@ -307,39 +307,11 @@ impl ArrowDeserializer { .decode(msg) .map_err(|e| SourceError::bad_data(format!("invalid JSON: {:?}", e)))?; timestamp_builder.append_value(to_nanos(timestamp) as i64); - if let Some(additional_fields) = additional_fields { - for (k, v) in additional_fields.iter() { - if let Some(builder) = self - .additional_fields_builder - .as_mut() - .and_then(|b| b.get_mut(k.to_owned())) - { - match v { - FieldValueType::Int32(i) => { - builder - .as_any_mut() - .downcast_mut::() - .expect("additional field has incorrect type") - .append_value(*i); - } - FieldValueType::Int64(i) => { - builder - .as_any_mut() - .downcast_mut::() - .expect("additional field has incorrect type") - .append_value(*i); - } - FieldValueType::String(s) => { - builder - .as_any_mut() - .downcast_mut::() - .expect("additional field has incorrect type") - .append_value(s); - } - } - } - } - } + + add_additional_fields_using_builder( + additional_fields, + &mut self.additional_fields_builder, + ); self.buffered_count += 1; } Format::Protobuf(proto) => { @@ -357,39 +329,11 @@ impl ArrowDeserializer { .map_err(|e| SourceError::bad_data(format!("invalid JSON: {:?}", e)))?; timestamp_builder.append_value(to_nanos(timestamp) as i64); - if self.additional_fields_builder.is_some() { - for (k, v) in additional_fields.unwrap().iter() { - if let Some(builder) = self - .additional_fields_builder - .as_mut() - .and_then(|b| b.get_mut(k.to_owned())) - { - match v { - FieldValueType::Int32(i) => { - builder - .as_any_mut() - .downcast_mut::() - .expect("additional field has incorrect type") - .append_value(*i); - } - FieldValueType::Int64(i) => { - builder - .as_any_mut() - .downcast_mut::() - .expect("additional field has incorrect type") - .append_value(*i); - } - FieldValueType::String(s) => { - builder - .as_any_mut() - .downcast_mut::() - .expect("additional field has incorrect type") - .append_value(s); - } - } - } - } - } + add_additional_fields_using_builder( + additional_fields, + &mut self.additional_fields_builder, + ); + self.buffered_count += 1; } } @@ -556,6 +500,44 @@ pub(crate) fn add_additional_fields( } } +pub(crate) fn add_additional_fields_using_builder( + additional_fields: Option>>, + additional_fields_builder: &mut Option>>, +) { + if let Some(fields) = additional_fields { + for (k, v) in fields.iter() { + if let Some(builder) = additional_fields_builder + .as_mut() + .and_then(|b| b.get_mut(*k)) + { + match v { + FieldValueType::Int32(i) => { + builder + .as_any_mut() + .downcast_mut::() + .expect("additional field has incorrect type") + .append_value(*i); + } + FieldValueType::Int64(i) => { + builder + .as_any_mut() + .downcast_mut::() + .expect("additional field has incorrect type") + .append_value(*i); + } + FieldValueType::String(s) => { + builder + .as_any_mut() + .downcast_mut::() + .expect("additional field has incorrect type") + .append_value(s); + } + } + } + } + } +} + #[cfg(test)] mod tests { use crate::de::{ArrowDeserializer, FramingIterator}; From 174835bc5cbdbdbf9b869b07580f784874bf69c8 Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw Date: Tue, 29 Oct 2024 13:39:09 +0530 Subject: [PATCH 33/36] adding tests --- .../src/kafka/source/test.rs | 98 +++++++++++++++++++ crates/arroyo-formats/src/de.rs | 80 ++++++++++++++- .../src/test/queries/kafka_metadata_udf.sql | 15 +++ 3 files changed, 191 insertions(+), 2 deletions(-) create mode 100644 crates/arroyo-planner/src/test/queries/kafka_metadata_udf.sql diff --git a/crates/arroyo-connectors/src/kafka/source/test.rs b/crates/arroyo-connectors/src/kafka/source/test.rs index 19453ecc4..2b2fe1408 100644 --- a/crates/arroyo-connectors/src/kafka/source/test.rs +++ b/crates/arroyo-connectors/src/kafka/source/test.rs @@ -343,3 +343,101 @@ async fn test_kafka() { ) .await; } + +#[tokio::test] +async fn test_kafka_with_metadata_fields() { + let mut kafka_topic_tester = KafkaTopicTester { + topic: "__arroyo-source-test".to_string(), + server: "0.0.0.0:9092".to_string(), + group_id: Some("test-consumer-group".to_string()), + }; + + let mut task_info = arroyo_types::get_test_task_info(); + task_info.job_id = format!("kafka-job-{}", random::()); + + kafka_topic_tester.create_topic().await; + + // Prepare metadata fields + let metadata_fields = Some(HashMap::from([( + "offset".to_string(), + "offset_id".to_string(), + )])); + + // Set metadata fields in KafkaSourceFunc + let mut kafka = KafkaSourceFunc { + bootstrap_servers: kafka_topic_tester.server.clone(), + topic: kafka_topic_tester.topic.clone(), + group_id: kafka_topic_tester.group_id.clone(), + group_id_prefix: None, + offset_mode: SourceOffset::Earliest, + format: Format::RawString(RawStringFormat {}), + framing: None, + bad_data: None, + schema_resolver: None, + client_configs: HashMap::new(), + messages_per_second: NonZeroU32::new(100).unwrap(), + metadata_fields, + }; + + let (_to_control_tx, control_rx) = channel(128); + let (command_tx, _from_control_rx) = channel(128); + let (data_tx, _recv) = batch_bounded(128); + + let checkpoint_metadata = None; + + let mut ctx = ArrowContext::new( + task_info.clone(), + checkpoint_metadata, + control_rx, + command_tx, + 1, + vec![], + Some(ArroyoSchema::new_unkeyed( + Arc::new(Schema::new(vec![ + Field::new( + "_timestamp", + DataType::Timestamp(TimeUnit::Nanosecond, None), + false, + ), + Field::new("value", DataType::Utf8, false), + Field::new("offset", DataType::Int64, false), + ])), + 0, + )), + None, + vec![vec![data_tx]], + kafka.tables(), + ) + .await; + + tokio::spawn(async move { + kafka.run(&mut ctx).await; + }); + + let mut reader = kafka_topic_tester + .get_source_with_reader(task_info.clone(), None) + .await; + let mut producer = kafka_topic_tester.get_producer(); + + // Send test data + let expected_messages: Vec<_> = (1u64..=21) + .map(|i| { + let data = TestData { i }; + producer.send_data(data.clone()); + serde_json::to_string(&data).unwrap() + }) + .collect(); + + // Verify received messages + reader + .assert_next_message_record_values(expected_messages.into()) + .await; + + reader + .to_control_tx + .send(ControlMessage::Stop { + mode: arroyo_rpc::grpc::rpc::StopMode::Graceful, + }) + .await + .unwrap(); +} diff --git a/crates/arroyo-formats/src/de.rs b/crates/arroyo-formats/src/de.rs index 4c7f50f0e..5f37ed911 100644 --- a/crates/arroyo-formats/src/de.rs +++ b/crates/arroyo-formats/src/de.rs @@ -237,6 +237,15 @@ impl ArrowDeserializer { kernels::filter::filter(×tamp.finish(), &mask).unwrap(); columns.insert(self.schema.timestamp_index, Arc::new(timestamp)); + if let Some(additional_fields) = self.additional_fields_builder.take() { + for (field_name, mut builder) in additional_fields { + if let Some((idx, _)) = + self.schema.schema.column_with_name(&field_name) + { + columns[idx] = Arc::new(builder.as_mut().finish()); + } + } + } RecordBatch::try_new(self.schema.schema.clone(), columns).unwrap() }), ), @@ -474,7 +483,7 @@ pub(crate) fn add_additional_fields( let (idx, _) = schema .schema .column_with_name(key) - .expect("no 'value' column for additional fields"); + .unwrap_or_else(|| panic!("no '{}' column for additional fields", key)); match value { FieldValueType::Int32(i) => { builder[idx] @@ -540,7 +549,8 @@ pub(crate) fn add_additional_fields_using_builder( #[cfg(test)] mod tests { - use crate::de::{ArrowDeserializer, FramingIterator}; + use crate::de::{ArrowDeserializer, FieldValueType, FramingIterator}; + use arrow::datatypes::Int32Type; use arrow_array::builder::{make_builder, ArrayBuilder}; use arrow_array::cast::AsArray; use arrow_array::types::{GenericBinaryType, Int64Type, TimestampNanosecondType}; @@ -782,4 +792,70 @@ mod tests { to_nanos(time) as i64 ); } + + #[tokio::test] + async fn test_additional_fields_deserialisation() { + let schema = Arc::new(Schema::new(vec![ + arrow_schema::Field::new("x", arrow_schema::DataType::Int64, true), + arrow_schema::Field::new("y", arrow_schema::DataType::Int32, true), + arrow_schema::Field::new("z", arrow_schema::DataType::Utf8, true), + arrow_schema::Field::new( + "_timestamp", + arrow_schema::DataType::Timestamp(TimeUnit::Nanosecond, None), + false, + ), + ])); + + let mut arrays: Vec<_> = schema + .fields + .iter() + .map(|f| make_builder(f.data_type(), 16)) + .collect(); + + let arroyo_schema = ArroyoSchema::from_schema_unkeyed(schema.clone()).unwrap(); + + let mut deserializer = ArrowDeserializer::new( + Format::Json(JsonFormat { + confluent_schema_registry: false, + schema_id: None, + include_schema: false, + debezium: false, + unstructured: false, + timestamp_format: Default::default(), + }), + arroyo_schema, + None, + BadData::Drop {}, + ); + + let time = SystemTime::now(); + let mut additional_fields = std::collections::HashMap::new(); + let binding = "y".to_string(); + additional_fields.insert(&binding, FieldValueType::Int32(5)); + let z_value = "hello".to_string(); + let binding = "z".to_string(); + additional_fields.insert(&binding, FieldValueType::String(&z_value)); + + let result = deserializer + .deserialize_slice( + &mut arrays, + json!({ "x": 5 }).to_string().as_bytes(), + time, + Some(additional_fields), + ) + .await; + assert!(result.is_empty()); + + let batch = deserializer.flush_buffer().unwrap().unwrap(); + assert_eq!(batch.num_rows(), 1); + assert_eq!(batch.columns()[0].as_primitive::().value(0), 5); + assert_eq!(batch.columns()[1].as_primitive::().value(0), 5); + assert_eq!(batch.columns()[2].as_string::().value(0), "hello"); + assert_eq!( + batch.columns()[3] + .as_primitive::() + .value(0), + to_nanos(time) as i64 + ); + } } diff --git a/crates/arroyo-planner/src/test/queries/kafka_metadata_udf.sql b/crates/arroyo-planner/src/test/queries/kafka_metadata_udf.sql new file mode 100644 index 000000000..c88c503f9 --- /dev/null +++ b/crates/arroyo-planner/src/test/queries/kafka_metadata_udf.sql @@ -0,0 +1,15 @@ +create table users ( + id TEXT, + name TEXT, + offset BIGINT GENERATED ALWAYS AS (metadata('offset_id')) STORED, + topic TEXT GENERATED ALWAYS AS (metadata('topic')) STORED, + partition INT GENERATED ALWAYS AS (metadata('partition')) STORED +) with ( + connector = 'kafka', + topic = 'order_topic', + format='json', + bootstrap_servers = '0.0.0.0:9092', + type='source' +); + +SELECT * FROM users; \ No newline at end of file From 77f6a90bcba675a3be8ddc71a4109d41ba0dea6c Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw Date: Tue, 29 Oct 2024 13:57:30 +0530 Subject: [PATCH 34/36] adding validations for datatypes --- crates/arroyo-formats/src/de.rs | 55 +++++++++++++++++++++------------ 1 file changed, 35 insertions(+), 20 deletions(-) diff --git a/crates/arroyo-formats/src/de.rs b/crates/arroyo-formats/src/de.rs index 5f37ed911..0198b1193 100644 --- a/crates/arroyo-formats/src/de.rs +++ b/crates/arroyo-formats/src/de.rs @@ -207,17 +207,11 @@ impl ArrowDeserializer { .map(|batch| { let mut columns = batch.columns().to_vec(); columns.insert(self.schema.timestamp_index, Arc::new(timestamp.finish())); - - if let Some(additional_fields) = self.additional_fields_builder.take() { - for (field_name, mut builder) in additional_fields { - if let Some((idx, _)) = - self.schema.schema.column_with_name(&field_name) - { - columns[idx] = Arc::new(builder.as_mut().finish()); - } - } - } - + flush_additional_fields_builders( + &mut self.additional_fields_builder, + &self.schema, + &mut columns, + ); RecordBatch::try_new(self.schema.schema.clone(), columns).unwrap() }), ), @@ -237,15 +231,11 @@ impl ArrowDeserializer { kernels::filter::filter(×tamp.finish(), &mask).unwrap(); columns.insert(self.schema.timestamp_index, Arc::new(timestamp)); - if let Some(additional_fields) = self.additional_fields_builder.take() { - for (field_name, mut builder) in additional_fields { - if let Some((idx, _)) = - self.schema.schema.column_with_name(&field_name) - { - columns[idx] = Arc::new(builder.as_mut().finish()); - } - } - } + flush_additional_fields_builders( + &mut self.additional_fields_builder, + &self.schema, + &mut columns, + ); RecordBatch::try_new(self.schema.schema.clone(), columns).unwrap() }), ), @@ -547,6 +537,31 @@ pub(crate) fn add_additional_fields_using_builder( } } +pub(crate) fn flush_additional_fields_builders( + additional_fields_builder: &mut Option>>, + schema: &ArroyoSchema, + columns: &mut [Arc], +) { + if let Some(additional_fields) = additional_fields_builder.take() { + for (field_name, mut builder) in additional_fields { + if let Some((idx, _)) = schema.schema.column_with_name(&field_name) { + let expected_type = schema.schema.fields[idx].data_type(); + let built_column = builder.as_mut().finish(); + let actual_type = built_column.data_type(); + if expected_type != actual_type { + panic!( + "Type mismatch for column '{}': expected {:?}, got {:?}", + field_name, expected_type, actual_type + ); + } + columns[idx] = Arc::new(built_column); + } else { + panic!("Field '{}' not found in schema", field_name); + } + } + } +} + #[cfg(test)] mod tests { use crate::de::{ArrowDeserializer, FieldValueType, FramingIterator}; From 66e996a481f388d1055734888d8ae71167fc23f0 Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw Date: Tue, 29 Oct 2024 14:14:12 +0530 Subject: [PATCH 35/36] bug fix in test --- crates/arroyo-connectors/src/kafka/source/test.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/arroyo-connectors/src/kafka/source/test.rs b/crates/arroyo-connectors/src/kafka/source/test.rs index 2b2fe1408..a8af9cb9d 100644 --- a/crates/arroyo-connectors/src/kafka/source/test.rs +++ b/crates/arroyo-connectors/src/kafka/source/test.rs @@ -347,7 +347,7 @@ async fn test_kafka() { #[tokio::test] async fn test_kafka_with_metadata_fields() { let mut kafka_topic_tester = KafkaTopicTester { - topic: "__arroyo-source-test".to_string(), + topic: "__arroyo-source-test_metadata".to_string(), server: "0.0.0.0:9092".to_string(), group_id: Some("test-consumer-group".to_string()), }; From fa2d089fbdbc67976c72906be3f713c89e8d33cd Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw Date: Wed, 30 Oct 2024 11:50:25 +0530 Subject: [PATCH 36/36] adding validations for dataypes in connector metadata udf --- crates/arroyo-connectors/src/blackhole/mod.rs | 5 ++- crates/arroyo-connectors/src/confluent/mod.rs | 5 ++- .../arroyo-connectors/src/filesystem/delta.rs | 5 ++- .../arroyo-connectors/src/filesystem/mod.rs | 5 ++- crates/arroyo-connectors/src/fluvio/mod.rs | 5 ++- crates/arroyo-connectors/src/impulse/mod.rs | 5 ++- crates/arroyo-connectors/src/kafka/mod.rs | 44 +++++++++++++++---- crates/arroyo-connectors/src/kinesis/mod.rs | 5 ++- crates/arroyo-connectors/src/mqtt/mod.rs | 5 ++- crates/arroyo-connectors/src/nats/mod.rs | 5 ++- crates/arroyo-connectors/src/nexmark/mod.rs | 6 +-- .../arroyo-connectors/src/polling_http/mod.rs | 5 ++- crates/arroyo-connectors/src/preview/mod.rs | 5 ++- crates/arroyo-connectors/src/redis/mod.rs | 5 ++- .../arroyo-connectors/src/single_file/mod.rs | 5 ++- crates/arroyo-connectors/src/sse/mod.rs | 5 ++- crates/arroyo-connectors/src/stdout/mod.rs | 5 ++- crates/arroyo-connectors/src/webhook/mod.rs | 5 ++- crates/arroyo-connectors/src/websocket/mod.rs | 5 ++- crates/arroyo-operator/src/connector.rs | 13 +++--- crates/arroyo-planner/src/tables.rs | 9 ++-- 21 files changed, 100 insertions(+), 57 deletions(-) diff --git a/crates/arroyo-connectors/src/blackhole/mod.rs b/crates/arroyo-connectors/src/blackhole/mod.rs index f186128b5..1b3f522e6 100644 --- a/crates/arroyo-connectors/src/blackhole/mod.rs +++ b/crates/arroyo-connectors/src/blackhole/mod.rs @@ -1,5 +1,6 @@ use crate::blackhole::operator::BlackholeSinkFunc; use anyhow::anyhow; +use arrow::datatypes::DataType; use arroyo_operator::connector::{Connection, Connector}; use arroyo_operator::operator::OperatorNode; use arroyo_rpc::api_types::connections::{ @@ -78,7 +79,7 @@ impl Connector for BlackholeConnector { _options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, - _metadata_fields: Option>, + _metadata_fields: Option>, ) -> anyhow::Result { self.from_config(None, name, EmptyConfig {}, EmptyConfig {}, schema, None) } @@ -90,7 +91,7 @@ impl Connector for BlackholeConnector { config: Self::ProfileT, table: Self::TableT, s: Option<&ConnectionSchema>, - _metadata_fields: Option>, + _metadata_fields: Option>, ) -> anyhow::Result { let description = "Blackhole".to_string(); diff --git a/crates/arroyo-connectors/src/confluent/mod.rs b/crates/arroyo-connectors/src/confluent/mod.rs index c3886cd92..aa991afbd 100644 --- a/crates/arroyo-connectors/src/confluent/mod.rs +++ b/crates/arroyo-connectors/src/confluent/mod.rs @@ -3,6 +3,7 @@ use crate::kafka::{ }; use crate::{kafka, pull_opt}; use anyhow::anyhow; +use arrow::datatypes::DataType; use arroyo_operator::connector::{Connection, Connector}; use arroyo_operator::operator::OperatorNode; use arroyo_rpc::api_types::connections::{ @@ -161,7 +162,7 @@ impl Connector for ConfluentConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, profile: Option<&ConnectionProfile>, - _metadata_fields: Option>, + _metadata_fields: Option>, ) -> anyhow::Result { let connection = profile .map(|p| { @@ -183,7 +184,7 @@ impl Connector for ConfluentConnector { config: Self::ProfileT, mut table: Self::TableT, schema: Option<&ConnectionSchema>, - _metadata_fields: Option>, + _metadata_fields: Option>, ) -> anyhow::Result { table .client_configs diff --git a/crates/arroyo-connectors/src/filesystem/delta.rs b/crates/arroyo-connectors/src/filesystem/delta.rs index 750ad1f45..2c62434a1 100644 --- a/crates/arroyo-connectors/src/filesystem/delta.rs +++ b/crates/arroyo-connectors/src/filesystem/delta.rs @@ -1,4 +1,5 @@ use anyhow::{anyhow, bail}; +use arrow::datatypes::DataType; use arroyo_operator::connector::Connection; use arroyo_storage::BackendConfig; use std::collections::HashMap; @@ -77,7 +78,7 @@ impl Connector for DeltaLakeConnector { config: Self::ProfileT, table: Self::TableT, schema: Option<&ConnectionSchema>, - _metadata_fields: Option>, + _metadata_fields: Option>, ) -> anyhow::Result { let TableType::Sink { write_path, @@ -144,7 +145,7 @@ impl Connector for DeltaLakeConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, - _metadata_fields: Option>, + _metadata_fields: Option>, ) -> anyhow::Result { let table = file_system_sink_from_options(options, schema, CommitStyle::DeltaLake)?; diff --git a/crates/arroyo-connectors/src/filesystem/mod.rs b/crates/arroyo-connectors/src/filesystem/mod.rs index fdd66b37c..d3b4706bd 100644 --- a/crates/arroyo-connectors/src/filesystem/mod.rs +++ b/crates/arroyo-connectors/src/filesystem/mod.rs @@ -3,6 +3,7 @@ mod sink; mod source; use anyhow::{anyhow, bail, Result}; +use arrow::datatypes::DataType; use arroyo_storage::BackendConfig; use regex::Regex; use std::collections::HashMap; @@ -114,7 +115,7 @@ impl Connector for FileSystemConnector { config: Self::ProfileT, table: Self::TableT, schema: Option<&ConnectionSchema>, - _metadata_fields: Option>, + _metadata_fields: Option>, ) -> anyhow::Result { let (description, connection_type) = match table.table_type { TableType::Source { .. } => ("FileSystem".to_string(), ConnectionType::Source), @@ -189,7 +190,7 @@ impl Connector for FileSystemConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, - _metadata_fields: Option>, + _metadata_fields: Option>, ) -> anyhow::Result { match options.remove("type") { Some(t) if t == "source" => { diff --git a/crates/arroyo-connectors/src/fluvio/mod.rs b/crates/arroyo-connectors/src/fluvio/mod.rs index 911197368..98f9aa4bb 100644 --- a/crates/arroyo-connectors/src/fluvio/mod.rs +++ b/crates/arroyo-connectors/src/fluvio/mod.rs @@ -1,4 +1,5 @@ use anyhow::{anyhow, bail}; +use arrow::datatypes::DataType; use arroyo_formats::ser::ArrowSerializer; use arroyo_operator::connector::{Connection, Connector}; use arroyo_operator::operator::OperatorNode; @@ -88,7 +89,7 @@ impl Connector for FluvioConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, - _metadata_fields: Option>, + _metadata_fields: Option>, ) -> anyhow::Result { let endpoint = options.remove("endpoint"); let topic = pull_opt("topic", options)?; @@ -127,7 +128,7 @@ impl Connector for FluvioConnector { config: EmptyConfig, table: FluvioTable, schema: Option<&ConnectionSchema>, - _metadata_fields: Option>, + _metadata_fields: Option>, ) -> anyhow::Result { let (typ, desc) = match table.type_ { TableType::Source { .. } => ( diff --git a/crates/arroyo-connectors/src/impulse/mod.rs b/crates/arroyo-connectors/src/impulse/mod.rs index b1163de08..c5382a369 100644 --- a/crates/arroyo-connectors/src/impulse/mod.rs +++ b/crates/arroyo-connectors/src/impulse/mod.rs @@ -1,6 +1,7 @@ mod operator; use anyhow::{anyhow, bail}; +use arrow::datatypes::DataType; use arroyo_operator::connector::{Connection, Connector}; use arroyo_operator::operator::OperatorNode; use arroyo_rpc::api_types::connections::FieldType::Primitive; @@ -101,7 +102,7 @@ impl Connector for ImpulseConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, - _metadata_fields: Option>, + _metadata_fields: Option>, ) -> anyhow::Result { let event_rate = f64::from_str(&pull_opt("event_rate", options)?) .map_err(|_| anyhow!("invalid value for event_rate; expected float"))?; @@ -146,7 +147,7 @@ impl Connector for ImpulseConnector { config: Self::ProfileT, table: Self::TableT, _: Option<&ConnectionSchema>, - _metadata_fields: Option>, + _metadata_fields: Option>, ) -> anyhow::Result { let description = format!( "{}Impulse<{} eps{}>", diff --git a/crates/arroyo-connectors/src/kafka/mod.rs b/crates/arroyo-connectors/src/kafka/mod.rs index bc2b479a8..b3faf20a2 100644 --- a/crates/arroyo-connectors/src/kafka/mod.rs +++ b/crates/arroyo-connectors/src/kafka/mod.rs @@ -1,4 +1,5 @@ use anyhow::{anyhow, bail}; +use arrow::datatypes::DataType; use arroyo_formats::de::ArrowDeserializer; use arroyo_formats::ser::ArrowSerializer; use arroyo_operator::connector::Connection; @@ -188,7 +189,7 @@ impl Connector for KafkaConnector { config: KafkaConfig, table: KafkaTable, schema: Option<&ConnectionSchema>, - metadata_fields: Option>, + metadata_fields: Option>, ) -> anyhow::Result { let (typ, desc) = match table.type_ { TableType::Source { .. } => ( @@ -208,6 +209,13 @@ impl Connector for KafkaConnector { .map(|t| t.to_owned()) .ok_or_else(|| anyhow!("'format' must be set for Kafka connection"))?; + let metadata_fields = metadata_fields.map(|fields| { + fields + .into_iter() + .map(|(k, (v, _))| (k, v)) + .collect::>() + }); + let config = OperatorConfig { connection: serde_json::to_value(config).unwrap(), table: serde_json::to_value(table).unwrap(), @@ -314,7 +322,7 @@ impl Connector for KafkaConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, profile: Option<&ConnectionProfile>, - metadata_fields: Option>, + metadata_fields: Option>, ) -> anyhow::Result { let connection = profile .map(|p| { @@ -326,14 +334,32 @@ impl Connector for KafkaConnector { let table = Self::table_from_options(options)?; - let allowed_metadata_udf_args = ["offset_id", "partition", "topic"]; + let allowed_metadata_udf_args: HashMap<&str, DataType> = [ + ("offset_id", DataType::Int64), + ("partition", DataType::Int32), + ("topic", DataType::Utf8), + ] + .iter() + .cloned() + .collect(); + if let Some(fields) = &metadata_fields { - for val in fields.values() { - if !allowed_metadata_udf_args.contains(&val.as_str()) { - return Err(anyhow!( - "Invalid metadata field val for kafka connector: '{}'", - val - )); + for (field_name, data_type) in fields.values() { + match allowed_metadata_udf_args.get(field_name.as_str()) { + Some(expected_type) => { + if expected_type != data_type { + return Err(anyhow!( + "Invalid datatype for metadata field '{}': expected '{:?}', found '{:?}'", + field_name, expected_type, data_type + )); + } + } + None => { + return Err(anyhow!( + "Invalid metadata field name for Kafka connector: '{}'", + field_name + )); + } } } } diff --git a/crates/arroyo-connectors/src/kinesis/mod.rs b/crates/arroyo-connectors/src/kinesis/mod.rs index af7f8fdd5..4014a7a75 100644 --- a/crates/arroyo-connectors/src/kinesis/mod.rs +++ b/crates/arroyo-connectors/src/kinesis/mod.rs @@ -1,4 +1,5 @@ use anyhow::{anyhow, bail, Result}; +use arrow::datatypes::DataType; use std::collections::HashMap; use typify::import_types; @@ -83,7 +84,7 @@ impl Connector for KinesisConnector { config: Self::ProfileT, table: Self::TableT, schema: Option<&ConnectionSchema>, - _metadata_fields: Option>, + _metadata_fields: Option>, ) -> anyhow::Result { let (connection_type, description) = match table.type_ { TableType::Source { .. } => ( @@ -132,7 +133,7 @@ impl Connector for KinesisConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, - _metadata_fields: Option>, + _metadata_fields: Option>, ) -> anyhow::Result { let typ = pull_opt("type", options)?; let table_type = match typ.as_str() { diff --git a/crates/arroyo-connectors/src/mqtt/mod.rs b/crates/arroyo-connectors/src/mqtt/mod.rs index e8d15c817..e0e54d3fd 100644 --- a/crates/arroyo-connectors/src/mqtt/mod.rs +++ b/crates/arroyo-connectors/src/mqtt/mod.rs @@ -8,6 +8,7 @@ use crate::mqtt::sink::MqttSinkFunc; use crate::mqtt::source::MqttSourceFunc; use crate::pull_opt; use anyhow::{anyhow, bail}; +use arrow::datatypes::DataType; use arroyo_formats::ser::ArrowSerializer; use arroyo_operator::connector::{Connection, Connector}; use arroyo_operator::operator::OperatorNode; @@ -157,7 +158,7 @@ impl Connector for MqttConnector { config: MqttConfig, table: MqttTable, schema: Option<&ConnectionSchema>, - _metadata_fields: Option>, + _metadata_fields: Option>, ) -> anyhow::Result { let (typ, desc) = match table.type_ { TableType::Source { .. } => ( @@ -245,7 +246,7 @@ impl Connector for MqttConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, profile: Option<&ConnectionProfile>, - _metadata_fields: Option>, + _metadata_fields: Option>, ) -> anyhow::Result { let connection = profile .map(|p| { diff --git a/crates/arroyo-connectors/src/nats/mod.rs b/crates/arroyo-connectors/src/nats/mod.rs index 603aedd05..095d8fce5 100644 --- a/crates/arroyo-connectors/src/nats/mod.rs +++ b/crates/arroyo-connectors/src/nats/mod.rs @@ -3,6 +3,7 @@ use crate::nats::source::NatsSourceFunc; use crate::pull_opt; use anyhow::anyhow; use anyhow::bail; +use arrow::datatypes::DataType; use arroyo_formats::ser::ArrowSerializer; use arroyo_operator::connector::{Connection, Connector}; use arroyo_operator::operator::OperatorNode; @@ -246,7 +247,7 @@ impl Connector for NatsConnector { config: NatsConfig, table: NatsTable, schema: Option<&ConnectionSchema>, - _metadata_fields: Option>, + _metadata_fields: Option>, ) -> anyhow::Result { let stream_or_subject = match &table.connector_type { ConnectorType::Source { source_type, .. } => { @@ -316,7 +317,7 @@ impl Connector for NatsConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, profile: Option<&ConnectionProfile>, - _metadata_fields: Option>, + _metadata_fields: Option>, ) -> anyhow::Result { let connection = profile .map(|p| { diff --git a/crates/arroyo-connectors/src/nexmark/mod.rs b/crates/arroyo-connectors/src/nexmark/mod.rs index 17029d144..5c110f936 100644 --- a/crates/arroyo-connectors/src/nexmark/mod.rs +++ b/crates/arroyo-connectors/src/nexmark/mod.rs @@ -3,7 +3,7 @@ mod operator; mod test; use anyhow::{anyhow, bail}; -use arrow::datatypes::{Field, Schema, TimeUnit}; +use arrow::datatypes::{DataType, Field, Schema, TimeUnit}; use arroyo_operator::connector::{Connection, Connector}; use arroyo_operator::operator::OperatorNode; use arroyo_rpc::api_types::connections::{ @@ -158,7 +158,7 @@ impl Connector for NexmarkConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, - _metadata_fields: Option>, + _metadata_fields: Option>, ) -> anyhow::Result { let event_rate = f64::from_str(&pull_opt("event_rate", options)?) .map_err(|_| anyhow!("invalid value for event_rate; expected float"))?; @@ -195,7 +195,7 @@ impl Connector for NexmarkConnector { config: Self::ProfileT, table: Self::TableT, _: Option<&ConnectionSchema>, - _metadata_fields: Option>, + _metadata_fields: Option>, ) -> anyhow::Result { let description = format!( "{}Nexmark<{} eps>", diff --git a/crates/arroyo-connectors/src/polling_http/mod.rs b/crates/arroyo-connectors/src/polling_http/mod.rs index cb17a2ccf..ea8ec421b 100644 --- a/crates/arroyo-connectors/src/polling_http/mod.rs +++ b/crates/arroyo-connectors/src/polling_http/mod.rs @@ -5,6 +5,7 @@ use std::str::FromStr; use std::time::Duration; use anyhow::anyhow; +use arrow::datatypes::DataType; use arroyo_rpc::{var_str::VarStr, OperatorConfig}; use arroyo_types::string_to_map; use reqwest::{Client, Request}; @@ -152,7 +153,7 @@ impl Connector for PollingHTTPConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, - _metadata_fields: Option>, + _metadata_fields: Option>, ) -> anyhow::Result { let endpoint = pull_opt("endpoint", options)?; let headers = options.remove("headers"); @@ -195,7 +196,7 @@ impl Connector for PollingHTTPConnector { config: Self::ProfileT, table: Self::TableT, schema: Option<&ConnectionSchema>, - _metadata_fields: Option>, + _metadata_fields: Option>, ) -> anyhow::Result { let description = format!("PollingHTTPSource<{}>", table.endpoint); diff --git a/crates/arroyo-connectors/src/preview/mod.rs b/crates/arroyo-connectors/src/preview/mod.rs index 0a25be91c..72dddd7f0 100644 --- a/crates/arroyo-connectors/src/preview/mod.rs +++ b/crates/arroyo-connectors/src/preview/mod.rs @@ -3,6 +3,7 @@ mod operator; use std::collections::HashMap; use anyhow::{anyhow, bail}; +use arrow::datatypes::DataType; use arroyo_rpc::OperatorConfig; use arroyo_operator::connector::Connection; @@ -71,7 +72,7 @@ impl Connector for PreviewConnector { _: &mut HashMap, _: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, - _metadata_fields: Option>, + _metadata_fields: Option>, ) -> anyhow::Result { bail!("Preview connector cannot be created in SQL"); } @@ -83,7 +84,7 @@ impl Connector for PreviewConnector { config: Self::ProfileT, table: Self::TableT, schema: Option<&ConnectionSchema>, - _metadata_fields: Option>, + _metadata_fields: Option>, ) -> anyhow::Result { let description = "PreviewSink".to_string(); diff --git a/crates/arroyo-connectors/src/redis/mod.rs b/crates/arroyo-connectors/src/redis/mod.rs index eb9addf07..84323762e 100644 --- a/crates/arroyo-connectors/src/redis/mod.rs +++ b/crates/arroyo-connectors/src/redis/mod.rs @@ -1,6 +1,7 @@ mod operator; use anyhow::{anyhow, bail}; +use arrow::datatypes::DataType; use arroyo_formats::ser::ArrowSerializer; use arroyo_operator::connector::{Connection, Connector}; use arroyo_operator::operator::OperatorNode; @@ -227,7 +228,7 @@ impl Connector for RedisConnector { options: &mut HashMap, s: Option<&ConnectionSchema>, profile: Option<&ConnectionProfile>, - _metadata_fields: Option>, + _metadata_fields: Option>, ) -> anyhow::Result { let connection_config = match profile { Some(connection_profile) => { @@ -360,7 +361,7 @@ impl Connector for RedisConnector { config: Self::ProfileT, table: Self::TableT, schema: Option<&ConnectionSchema>, - _metadata_fields: Option>, + _metadata_fields: Option>, ) -> anyhow::Result { let schema = schema .map(|s| s.to_owned()) diff --git a/crates/arroyo-connectors/src/single_file/mod.rs b/crates/arroyo-connectors/src/single_file/mod.rs index cb25b9d64..543e5ba13 100644 --- a/crates/arroyo-connectors/src/single_file/mod.rs +++ b/crates/arroyo-connectors/src/single_file/mod.rs @@ -1,4 +1,5 @@ use anyhow::{anyhow, bail, Result}; +use arrow::datatypes::DataType; use arroyo_formats::ser::ArrowSerializer; use std::collections::HashMap; use typify::import_types; @@ -84,7 +85,7 @@ impl Connector for SingleFileConnector { config: Self::ProfileT, table: Self::TableT, schema: Option<&ConnectionSchema>, - _metadata_fields: Option>, + _metadata_fields: Option>, ) -> anyhow::Result { let schema = schema .map(|s| s.to_owned()) @@ -124,7 +125,7 @@ impl Connector for SingleFileConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, - _metadata_fields: Option>, + _metadata_fields: Option>, ) -> anyhow::Result { let path = pull_opt("path", options)?; let Ok(table_type) = pull_opt("type", options)?.try_into() else { diff --git a/crates/arroyo-connectors/src/sse/mod.rs b/crates/arroyo-connectors/src/sse/mod.rs index 88a29142b..52b9b3be9 100644 --- a/crates/arroyo-connectors/src/sse/mod.rs +++ b/crates/arroyo-connectors/src/sse/mod.rs @@ -4,6 +4,7 @@ use std::collections::HashMap; use std::time::Duration; use anyhow::{anyhow, bail}; +use arrow::datatypes::DataType; use arroyo_rpc::{var_str::VarStr, OperatorConfig}; use arroyo_types::string_to_map; use eventsource_client::Client; @@ -78,7 +79,7 @@ impl Connector for SSEConnector { config: Self::ProfileT, table: Self::TableT, schema: Option<&ConnectionSchema>, - _metadata_fields: Option>, + _metadata_fields: Option>, ) -> anyhow::Result { let description = format!("SSESource<{}>", table.endpoint); @@ -128,7 +129,7 @@ impl Connector for SSEConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, - _metadata_fields: Option>, + _metadata_fields: Option>, ) -> anyhow::Result { let endpoint = pull_opt("endpoint", options)?; let headers = options.remove("headers"); diff --git a/crates/arroyo-connectors/src/stdout/mod.rs b/crates/arroyo-connectors/src/stdout/mod.rs index 1460c9e92..59db510a3 100644 --- a/crates/arroyo-connectors/src/stdout/mod.rs +++ b/crates/arroyo-connectors/src/stdout/mod.rs @@ -3,6 +3,7 @@ mod operator; use std::collections::HashMap; use anyhow::anyhow; +use arrow::datatypes::DataType; use arroyo_rpc::OperatorConfig; use tokio::io::BufWriter; @@ -76,7 +77,7 @@ impl Connector for StdoutConnector { _options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, - _metadata_fields: Option>, + _metadata_fields: Option>, ) -> anyhow::Result { self.from_config(None, name, EmptyConfig {}, EmptyConfig {}, schema, None) } @@ -88,7 +89,7 @@ impl Connector for StdoutConnector { config: Self::ProfileT, table: Self::TableT, schema: Option<&ConnectionSchema>, - _metadata_fields: Option>, + _metadata_fields: Option>, ) -> anyhow::Result { let description = "StdoutSink".to_string(); diff --git a/crates/arroyo-connectors/src/webhook/mod.rs b/crates/arroyo-connectors/src/webhook/mod.rs index e8d513251..d5e1a2a29 100644 --- a/crates/arroyo-connectors/src/webhook/mod.rs +++ b/crates/arroyo-connectors/src/webhook/mod.rs @@ -5,6 +5,7 @@ use std::sync::Arc; use std::time::SystemTime; use anyhow::anyhow; +use arrow::datatypes::DataType; use arroyo_rpc::OperatorConfig; use arroyo_formats::ser::ArrowSerializer; @@ -142,7 +143,7 @@ impl Connector for WebhookConnector { config: Self::ProfileT, table: Self::TableT, schema: Option<&ConnectionSchema>, - _metadata_fields: Option>, + _metadata_fields: Option>, ) -> anyhow::Result { let description = format!("WebhookSink<{}>", table.endpoint.sub_env_vars()?); @@ -183,7 +184,7 @@ impl Connector for WebhookConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, - _metadata_fields: Option>, + _metadata_fields: Option>, ) -> anyhow::Result { let endpoint = pull_opt("endpoint", options)?; diff --git a/crates/arroyo-connectors/src/websocket/mod.rs b/crates/arroyo-connectors/src/websocket/mod.rs index e2e5f65c8..36fefca1f 100644 --- a/crates/arroyo-connectors/src/websocket/mod.rs +++ b/crates/arroyo-connectors/src/websocket/mod.rs @@ -3,6 +3,7 @@ use std::str::FromStr; use std::time::Duration; use anyhow::anyhow; +use arrow::datatypes::DataType; use arroyo_operator::connector::Connection; use arroyo_rpc::api_types::connections::{ ConnectionProfile, ConnectionSchema, ConnectionType, TestSourceMessage, @@ -220,7 +221,7 @@ impl Connector for WebsocketConnector { config: Self::ProfileT, table: Self::TableT, schema: Option<&ConnectionSchema>, - _metadata_fields: Option>, + _metadata_fields: Option>, ) -> anyhow::Result { let description = format!("WebsocketSource<{}>", table.endpoint); @@ -270,7 +271,7 @@ impl Connector for WebsocketConnector { options: &mut HashMap, schema: Option<&ConnectionSchema>, _profile: Option<&ConnectionProfile>, - _metadata_fields: Option>, + _metadata_fields: Option>, ) -> anyhow::Result { let endpoint = pull_opt("endpoint", options)?; let headers = options.remove("headers"); diff --git a/crates/arroyo-operator/src/connector.rs b/crates/arroyo-operator/src/connector.rs index dfc6cf2d4..8606de728 100644 --- a/crates/arroyo-operator/src/connector.rs +++ b/crates/arroyo-operator/src/connector.rs @@ -1,5 +1,6 @@ use crate::operator::OperatorNode; use anyhow::anyhow; +use arrow::datatypes::DataType; use arroyo_rpc::api_types::connections::{ ConnectionProfile, ConnectionSchema, ConnectionType, TestSourceMessage, }; @@ -89,7 +90,7 @@ pub trait Connector: Send { options: &mut HashMap, schema: Option<&ConnectionSchema>, profile: Option<&ConnectionProfile>, - metadata_fields: Option>, + metadata_fields: Option>, ) -> anyhow::Result; fn from_config( @@ -99,7 +100,7 @@ pub trait Connector: Send { config: Self::ProfileT, table: Self::TableT, schema: Option<&ConnectionSchema>, - metadata_fields: Option>, + metadata_fields: Option>, ) -> anyhow::Result; #[allow(unused)] @@ -164,7 +165,7 @@ pub trait ErasedConnector: Send { options: &mut HashMap, schema: Option<&ConnectionSchema>, profile: Option<&ConnectionProfile>, - metadata_fields: Option>, + metadata_fields: Option>, ) -> anyhow::Result; fn from_config( @@ -174,7 +175,7 @@ pub trait ErasedConnector: Send { config: &serde_json::Value, table: &serde_json::Value, schema: Option<&ConnectionSchema>, - metadata_fields: Option>, + metadata_fields: Option>, ) -> anyhow::Result; fn make_operator(&self, config: OperatorConfig) -> anyhow::Result; @@ -260,7 +261,7 @@ impl ErasedConnector for C { options: &mut HashMap, schema: Option<&ConnectionSchema>, profile: Option<&ConnectionProfile>, - metadata_fields: Option>, + metadata_fields: Option>, ) -> anyhow::Result { self.from_options(name, options, schema, profile, metadata_fields) } @@ -272,7 +273,7 @@ impl ErasedConnector for C { config: &serde_json::Value, table: &serde_json::Value, schema: Option<&ConnectionSchema>, - metadata_fields: Option>, + metadata_fields: Option>, ) -> anyhow::Result { self.from_config( id, diff --git a/crates/arroyo-planner/src/tables.rs b/crates/arroyo-planner/src/tables.rs index 017c29557..b13bcb14a 100644 --- a/crates/arroyo-planner/src/tables.rs +++ b/crates/arroyo-planner/src/tables.rs @@ -224,7 +224,7 @@ impl ConnectorTable { primary_keys: Vec, options: &mut HashMap, connection_profile: Option<&ConnectionProfile>, - connector_metadata_columns: Option>, + connector_metadata_columns: Option>, ) -> Result { // TODO: a more principled way of letting connectors dictate types to use if "delta" == connector { @@ -516,7 +516,7 @@ impl Table { fn schema_from_columns( columns: &[ColumnDef], schema_provider: &ArroyoSchemaProvider, - connector_metadata_columns: &mut HashMap, + connector_metadata_columns: &mut HashMap, ) -> Result> { let struct_field_pairs = columns .iter() @@ -527,10 +527,9 @@ impl Table { .options .iter() .any(|option| matches!(option.option, ColumnOption::NotNull)); - let struct_field = ArroyoExtensionType::add_metadata( extension, - Field::new(name, data_type, nullable), + Field::new(name, data_type.clone(), nullable), ); let generating_expression = column.options.iter().find_map(|option| { @@ -560,7 +559,7 @@ impl Table { { connector_metadata_columns.insert( column.name.value.to_string(), - value.to_string(), + (value.to_string(), data_type.clone()), ); return None; }